From 6a831d98d9f19561ccf1a1c7a492ffd9b85ade86 Mon Sep 17 00:00:00 2001 From: Bo Du Date: Sat, 16 Nov 2019 23:23:48 -0500 Subject: [PATCH 01/11] [dbnode] Make index memory RSS usage flat block over block (#2037) * Free file backed mmaped mem pages every tick. --- src/dbnode/persist/fs/index_read.go | 17 ++- src/dbnode/storage/index/block.go | 14 +- .../storage/index/compaction/compactor.go | 6 +- .../storage/index/read_through_segment.go | 8 + src/m3ninx/index/segment/fst/segment.go | 32 ++++ src/m3ninx/index/segment/segment_mock.go | 138 ++++++++++++++++++ src/m3ninx/index/segment/types.go | 7 + src/x/instrument/invariant.go | 4 +- src/x/mmap/mmap_linux.go | 12 ++ src/x/mmap/mmap_other.go | 24 +++ 10 files changed, 251 insertions(+), 11 deletions(-) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index 6e0a19b263..c952236568 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -201,11 +201,16 @@ func (r *indexReader) ReadSegmentFileSet() ( segmentType: idxpersist.IndexSegmentType(segment.SegmentType), } ) - closeFiles := func() { + success := false + defer func() { + // Do not close opened files if read finishes sucessfully. + if success { + return + } for _, file := range result.files { file.Close() } - } + }() for _, file := range segment.Files { segFileType := idxpersist.IndexSegmentFileType(file.SegmentFileType) @@ -218,7 +223,6 @@ func (r *indexReader) ReadSegmentFileSet() ( filePath = filesetIndexSegmentFilePathFromTime(r.namespaceDir, r.start, r.volumeIndex, r.currIdx, segFileType) default: - closeFiles() return nil, fmt.Errorf("unknown fileset type: %s", r.fileSetType) } @@ -234,7 +238,6 @@ func (r *indexReader) ReadSegmentFileSet() ( }, }) if err != nil { - closeFiles() return nil, err } @@ -248,10 +251,16 @@ func (r *indexReader) ReadSegmentFileSet() ( segmentFileType: segFileType, digest: digest.Checksum(bytes), }) + + // NB(bodu): Free mmaped bytes after we take the checksum so we don't get memory spikes at bootstrap time. + if err := mmap.MadviseDontNeed(bytes); err != nil { + return nil, err + } } r.currIdx++ r.readDigests.segments = append(r.readDigests.segments, digests) + success = true return result, nil } diff --git a/src/dbnode/storage/index/block.go b/src/dbnode/storage/index/block.go index 0d18c6fb49..e8ad59bec1 100644 --- a/src/dbnode/storage/index/block.go +++ b/src/dbnode/storage/index/block.go @@ -1235,8 +1235,8 @@ func (b *block) AddResults( } func (b *block) Tick(c context.Cancellable) (BlockTickResult, error) { - b.RLock() - defer b.RUnlock() + b.Lock() + defer b.Unlock() result := BlockTickResult{} if b.state == blockStateClosed { return result, errUnableToTickBlockClosed @@ -1252,15 +1252,23 @@ func (b *block) Tick(c context.Cancellable) (BlockTickResult, error) { result.NumDocs += seg.Segment().Size() } + multiErr := xerrors.NewMultiError() + // Any segments covering persisted shard ranges. for _, group := range b.shardRangesSegments { for _, seg := range group.segments { result.NumSegments++ result.NumDocs += seg.Size() + // TODO(bodu): Revist this and implement a more sophisticated free strategy. + if immSeg, ok := seg.(segment.ImmutableSegment); ok { + if err := immSeg.FreeMmap(); err != nil { + multiErr = multiErr.Add(err) + } + } } } - return result, nil + return result, multiErr.FinalError() } func (b *block) Seal() error { diff --git a/src/dbnode/storage/index/compaction/compactor.go b/src/dbnode/storage/index/compaction/compactor.go index f838be75fd..8d8c0f5d5c 100644 --- a/src/dbnode/storage/index/compaction/compactor.go +++ b/src/dbnode/storage/index/compaction/compactor.go @@ -32,8 +32,8 @@ import ( "github.com/m3db/m3/src/m3ninx/index/segment/builder" "github.com/m3db/m3/src/m3ninx/index/segment/fst" "github.com/m3db/m3/src/m3ninx/index/segment/fst/encoding/docs" - "github.com/m3db/m3/src/x/mmap" xerrors "github.com/m3db/m3/src/x/errors" + "github.com/m3db/m3/src/x/mmap" ) var ( @@ -175,6 +175,10 @@ func (c *Compactor) CompactUsingBuilder( } // Reset docs batch for reuse + var empty doc.Document + for i := range batch { + batch[i] = empty + } batch = batch[:0] return nil } diff --git a/src/dbnode/storage/index/read_through_segment.go b/src/dbnode/storage/index/read_through_segment.go index 45bdc06d03..eed8e53108 100644 --- a/src/dbnode/storage/index/read_through_segment.go +++ b/src/dbnode/storage/index/read_through_segment.go @@ -140,6 +140,14 @@ func (r *ReadThroughSegment) ContainsField(field []byte) (bool, error) { return r.segment.ContainsField(field) } +// FreeMmap frees the mmapped data if any. +func (r *ReadThroughSegment) FreeMmap() error { + if immSeg, ok := r.segment.(segment.ImmutableSegment); ok { + return immSeg.FreeMmap() + } + return nil +} + // Size is a pass through call to the segment, since there's no // postings lists to cache for queries. func (r *ReadThroughSegment) Size() int64 { diff --git a/src/m3ninx/index/segment/fst/segment.go b/src/m3ninx/index/segment/fst/segment.go index e9d43078f6..16b91caf9d 100644 --- a/src/m3ninx/index/segment/fst/segment.go +++ b/src/m3ninx/index/segment/fst/segment.go @@ -37,6 +37,9 @@ import ( "github.com/m3db/m3/src/m3ninx/postings/roaring" "github.com/m3db/m3/src/m3ninx/x" "github.com/m3db/m3/src/x/context" + xerrors "github.com/m3db/m3/src/x/errors" + "github.com/m3db/m3/src/x/mmap" + pilosaroaring "github.com/m3db/pilosa/roaring" "github.com/m3db/vellum" ) @@ -292,6 +295,35 @@ func (r *fsSegment) TermsIterable() sgmt.TermsIterable { } } +func (r *fsSegment) FreeMmap() error { + multiErr := xerrors.NewMultiError() + + // NB(bodu): PostingsData, FSTTermsData and FSTFieldsData always present. + if err := mmap.MadviseDontNeed(r.data.PostingsData); err != nil { + multiErr = multiErr.Add(err) + } + if err := mmap.MadviseDontNeed(r.data.FSTTermsData); err != nil { + multiErr = multiErr.Add(err) + } + if err := mmap.MadviseDontNeed(r.data.FSTFieldsData); err != nil { + multiErr = multiErr.Add(err) + } + + // DocsData and DocsIdxData are not always present. + if r.data.DocsData != nil { + if err := mmap.MadviseDontNeed(r.data.DocsData); err != nil { + multiErr = multiErr.Add(err) + } + } + if r.data.DocsIdxData != nil { + if err := mmap.MadviseDontNeed(r.data.DocsIdxData); err != nil { + multiErr = multiErr.Add(err) + } + } + + return multiErr.FinalError() +} + // termsIterable allows multiple term lookups to share the same roaring // bitmap being unpacked for use when iterating over an entire segment type termsIterable struct { diff --git a/src/m3ninx/index/segment/segment_mock.go b/src/m3ninx/index/segment/segment_mock.go index e7295eb5b1..3325e03523 100644 --- a/src/m3ninx/index/segment/segment_mock.go +++ b/src/m3ninx/index/segment/segment_mock.go @@ -738,6 +738,144 @@ func (mr *MockMutableSegmentMockRecorder) IsSealed() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsSealed", reflect.TypeOf((*MockMutableSegment)(nil).IsSealed)) } +// MockImmutableSegment is a mock of ImmutableSegment interface +type MockImmutableSegment struct { + ctrl *gomock.Controller + recorder *MockImmutableSegmentMockRecorder +} + +// MockImmutableSegmentMockRecorder is the mock recorder for MockImmutableSegment +type MockImmutableSegmentMockRecorder struct { + mock *MockImmutableSegment +} + +// NewMockImmutableSegment creates a new mock instance +func NewMockImmutableSegment(ctrl *gomock.Controller) *MockImmutableSegment { + mock := &MockImmutableSegment{ctrl: ctrl} + mock.recorder = &MockImmutableSegmentMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockImmutableSegment) EXPECT() *MockImmutableSegmentMockRecorder { + return m.recorder +} + +// FieldsIterable mocks base method +func (m *MockImmutableSegment) FieldsIterable() FieldsIterable { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FieldsIterable") + ret0, _ := ret[0].(FieldsIterable) + return ret0 +} + +// FieldsIterable indicates an expected call of FieldsIterable +func (mr *MockImmutableSegmentMockRecorder) FieldsIterable() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FieldsIterable", reflect.TypeOf((*MockImmutableSegment)(nil).FieldsIterable)) +} + +// TermsIterable mocks base method +func (m *MockImmutableSegment) TermsIterable() TermsIterable { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TermsIterable") + ret0, _ := ret[0].(TermsIterable) + return ret0 +} + +// TermsIterable indicates an expected call of TermsIterable +func (mr *MockImmutableSegmentMockRecorder) TermsIterable() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TermsIterable", reflect.TypeOf((*MockImmutableSegment)(nil).TermsIterable)) +} + +// Size mocks base method +func (m *MockImmutableSegment) Size() int64 { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Size") + ret0, _ := ret[0].(int64) + return ret0 +} + +// Size indicates an expected call of Size +func (mr *MockImmutableSegmentMockRecorder) Size() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Size", reflect.TypeOf((*MockImmutableSegment)(nil).Size)) +} + +// ContainsID mocks base method +func (m *MockImmutableSegment) ContainsID(docID []byte) (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ContainsID", docID) + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ContainsID indicates an expected call of ContainsID +func (mr *MockImmutableSegmentMockRecorder) ContainsID(docID interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ContainsID", reflect.TypeOf((*MockImmutableSegment)(nil).ContainsID), docID) +} + +// ContainsField mocks base method +func (m *MockImmutableSegment) ContainsField(field []byte) (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ContainsField", field) + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ContainsField indicates an expected call of ContainsField +func (mr *MockImmutableSegmentMockRecorder) ContainsField(field interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ContainsField", reflect.TypeOf((*MockImmutableSegment)(nil).ContainsField), field) +} + +// Reader mocks base method +func (m *MockImmutableSegment) Reader() (index.Reader, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Reader") + ret0, _ := ret[0].(index.Reader) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Reader indicates an expected call of Reader +func (mr *MockImmutableSegmentMockRecorder) Reader() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reader", reflect.TypeOf((*MockImmutableSegment)(nil).Reader)) +} + +// Close mocks base method +func (m *MockImmutableSegment) 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 *MockImmutableSegmentMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockImmutableSegment)(nil).Close)) +} + +// FreeMmap mocks base method +func (m *MockImmutableSegment) FreeMmap() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FreeMmap") + ret0, _ := ret[0].(error) + return ret0 +} + +// FreeMmap indicates an expected call of FreeMmap +func (mr *MockImmutableSegmentMockRecorder) FreeMmap() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FreeMmap", reflect.TypeOf((*MockImmutableSegment)(nil).FreeMmap)) +} + // MockBuilder is a mock of Builder interface type MockBuilder struct { ctrl *gomock.Controller diff --git a/src/m3ninx/index/segment/types.go b/src/m3ninx/index/segment/types.go index 253235527d..c80bc3a8c1 100644 --- a/src/m3ninx/index/segment/types.go +++ b/src/m3ninx/index/segment/types.go @@ -144,6 +144,13 @@ type MutableSegment interface { IsSealed() bool } +// ImmutableSegment is segment that has been written to disk. +type ImmutableSegment interface { + Segment + + FreeMmap() error +} + // Builder is a builder that can be used to construct segments. type Builder interface { FieldsIterable diff --git a/src/x/instrument/invariant.go b/src/x/instrument/invariant.go index f4e551a1d0..baae4be76a 100644 --- a/src/x/instrument/invariant.go +++ b/src/x/instrument/invariant.go @@ -69,13 +69,11 @@ func EmitInvariantViolation(opts Options) { // with a supplied logger that is pre-configured with an invariant violated field. Optionally panics // if the ShouldPanicEnvironmentVariableName is set to "true". func EmitAndLogInvariantViolation(opts Options, f func(l *zap.Logger)) { - EmitInvariantViolation(opts) - logger := opts.Logger().With( zap.String(InvariantViolatedLogFieldName, InvariantViolatedLogFieldValue)) f(logger) - panicIfEnvSet() + EmitInvariantViolation(opts) } // InvariantErrorf constructs a new error, prefixed with a string indicating that an invariant diff --git a/src/x/mmap/mmap_linux.go b/src/x/mmap/mmap_linux.go index 803c5e4fd9..dd32608458 100644 --- a/src/x/mmap/mmap_linux.go +++ b/src/x/mmap/mmap_linux.go @@ -109,3 +109,15 @@ func Munmap(b []byte) error { return nil } + +// MadviseDontNeed frees mmapped memory. +// `MADV_DONTNEED` informs the kernel to free the mmapped pages right away instead of waiting for memory pressure. +// NB(bodu): DO NOT FREE anonymously mapped memory or else it will null all of the underlying bytes as the +// memory is not file backed. +func MadviseDontNeed(b []byte) error { + // Do nothing if there's no data. + if len(b) == 0 { + return nil + } + return syscall.Madvise(b, syscall.MADV_DONTNEED) +} diff --git a/src/x/mmap/mmap_other.go b/src/x/mmap/mmap_other.go index 1ce2f0a151..4bc803f3b5 100644 --- a/src/x/mmap/mmap_other.go +++ b/src/x/mmap/mmap_other.go @@ -25,6 +25,7 @@ package mmap import ( "fmt" "syscall" + "unsafe" ) // Fd mmaps a file @@ -81,3 +82,26 @@ func Munmap(b []byte) error { return nil } + +// MadviseDontNeed frees mmapped memory. +// `MADV_DONTNEED` informs the kernel to free the mmapped pages right away instead of waiting for memory pressure. +// NB(bodu): DO NOT FREE anonymously mapped memory or else it will null all of the underlying bytes as the +// memory is not file backed. +func MadviseDontNeed(b []byte) error { + // Do nothing if there's no data. + if len(b) == 0 { + return nil + } + return madvise(b, syscall.MADV_DONTNEED) +} + +// This is required because the unix package does not support the madvise system call. +// This works generically for other non linux platforms. +func madvise(b []byte, advice int) (err error) { + _, _, e1 := syscall.Syscall(syscall.SYS_MADVISE, uintptr(unsafe.Pointer(&b[0])), + uintptr(len(b)), uintptr(advice)) + if e1 != 0 { + err = e1 + } + return +} From 477c7bd1f78c1623b25a2a088c73978d6c7a197a Mon Sep 17 00:00:00 2001 From: Matt Schallert Date: Mon, 18 Nov 2019 08:25:49 -0800 Subject: [PATCH 02/11] [aggregator] lower default entry TTL; set in coord (#2047) Change the default aggregator TTL, as 24h is too high for most users and causes unnecessary memory usage. Also expose the config setting from the coordinator downsampler so that users using coordinators for aggregation may override. --- src/aggregator/aggregator/options.go | 2 +- src/cmd/services/m3coordinator/downsample/options.go | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/aggregator/aggregator/options.go b/src/aggregator/aggregator/options.go index 2bef24cb12..c511fb0bce 100644 --- a/src/aggregator/aggregator/options.go +++ b/src/aggregator/aggregator/options.go @@ -42,7 +42,7 @@ var ( defaultCounterPrefix = []byte("counts.") defaultTimerPrefix = []byte("timers.") defaultGaugePrefix = []byte("gauges.") - defaultEntryTTL = 24 * time.Hour + defaultEntryTTL = time.Hour defaultEntryCheckInterval = time.Hour defaultEntryCheckBatchPercent = 0.01 defaultMaxTimerBatchSizePerWrite = 0 diff --git a/src/cmd/services/m3coordinator/downsample/options.go b/src/cmd/services/m3coordinator/downsample/options.go index 934c0290ea..f6af82d0ec 100644 --- a/src/cmd/services/m3coordinator/downsample/options.go +++ b/src/cmd/services/m3coordinator/downsample/options.go @@ -189,6 +189,9 @@ type Configuration struct { // BufferPastLimits specifies the buffer past limits. BufferPastLimits []BufferPastLimitConfiguration `yaml:"bufferPastLimits"` + + // EntryTTL determines how long an entry remains alive before it may be expired due to inactivity. + EntryTTL time.Duration `yaml:"entryTTL"` } // RemoteAggregatorConfiguration specifies a remote aggregator @@ -423,6 +426,10 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { ) }) + if cfg.EntryTTL != 0 { + aggregatorOpts = aggregatorOpts.SetEntryTTL(cfg.EntryTTL) + } + aggregatorInstance := aggregator.NewAggregator(aggregatorOpts) if err := aggregatorInstance.Open(); err != nil { return agg{}, err From a64a80f37b4b997ceebfb5135db8cf3bbb449b49 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 23 Nov 2019 09:32:58 -0800 Subject: [PATCH 03/11] [x] Add ability to override the on error method --- src/x/instrument/config.go | 28 +++++++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/src/x/instrument/config.go b/src/x/instrument/config.go index 01f738db47..b50fb4ec55 100644 --- a/src/x/instrument/config.go +++ b/src/x/instrument/config.go @@ -31,6 +31,7 @@ import ( "github.com/uber-go/tally/m3" "github.com/uber-go/tally/multi" "github.com/uber-go/tally/prometheus" + "go.uber.org/zap" ) var ( @@ -73,7 +74,8 @@ type MetricsConfiguration struct { // NewRootScope creates a new tally.Scope based on a tally.CachedStatsReporter // based on the the the config. func (mc *MetricsConfiguration) NewRootScope() (tally.Scope, io.Closer, error) { - scope, closer, _, err := mc.NewRootScopeAndReporters() + opts := NewRootScopeAndReportersOptions{} + scope, closer, _, err := mc.NewRootScopeAndReporters(opts) return scope, closer, err } @@ -95,14 +97,31 @@ type MetricsConfigurationPrometheusReporter struct { Registry *prom.Registry } +// NewRootScopeAndReportersOptions is a set of options +type NewRootScopeAndReportersOptions struct { + OnError func(e error) +} + // NewRootScopeAndReporters creates a new tally.Scope based on a tally.CachedStatsReporter // based on the the the config along with the reporters used. -func (mc *MetricsConfiguration) NewRootScopeAndReporters() ( +func (mc *MetricsConfiguration) NewRootScopeAndReporters( + opts NewRootScopeAndReportersOptions, +) ( tally.Scope, io.Closer, MetricsConfigurationReporters, error, ) { + // Set a default on error method for sane handling when registering metrics + // results in an error with the Prometheus reporter. + onError := func(e error) { + logger := NewOptions().Logger() + logger.Error("register metrics error", zap.Error(e)) + } + if opts.OnError != nil { + onError = opts.OnError + } + var result MetricsConfigurationReporters if mc.M3Reporter != nil { r, err := mc.M3Reporter.NewReporter() @@ -134,7 +153,10 @@ func (mc *MetricsConfiguration) NewRootScopeAndReporters() ( })); err != nil { return nil, nil, MetricsConfigurationReporters{}, fmt.Errorf("could not create process collector: %v", err) } - opts := prometheus.ConfigurationOptions{Registry: registry} + opts := prometheus.ConfigurationOptions{ + Registry: registry, + OnError: onError, + } r, err := mc.PrometheusReporter.NewReporter(opts) if err != nil { return nil, nil, MetricsConfigurationReporters{}, err From 638ecfbfdf05564628e54d6df65bdccbceabba43 Mon Sep 17 00:00:00 2001 From: Matt Schallert Date: Mon, 25 Nov 2019 20:37:01 -0500 Subject: [PATCH 04/11] [ci] update ci-scripts for pre-release tagging (#2048) --- .ci | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.ci b/.ci index be9c6e9414..4b6ec85740 160000 --- a/.ci +++ b/.ci @@ -1 +1 @@ -Subproject commit be9c6e941477e2eaa9e77090fc5f87a919b642a1 +Subproject commit 4b6ec85740248532d732d5bebc1dedb01a57a2d8 From 1f40a47004dfaed2bf5cf2b8aab77c6a2fb6e750 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 26 Nov 2019 13:52:42 -0500 Subject: [PATCH 05/11] [dbnode] Single pass bootstrap and raw bootstrap memory/performance enhancements (#1989) --- .fossa.yml | 4 - Makefile | 6 +- scripts/auto-gen-helpers.sh | 6 +- src/cmd/tools/verify_commitlogs/README.MD | 25 - src/cmd/tools/verify_commitlogs/main/main.go | 285 --- src/dbnode/generated-source-files.mk | 64 +- src/dbnode/generated/mocks/generate.go | 1 + ...p_after_buffer_rotation_regression_test.go | 25 +- ...buffer_rotation_no_tick_regression_test.go | 26 +- src/dbnode/integration/bootstrap_helpers.go | 53 +- src/dbnode/namespace/utils.go | 45 + .../persist/fs/commitlog/commit_log_mock.go | 9 +- .../persist/fs/commitlog/commit_log_test.go | 52 +- src/dbnode/persist/fs/commitlog/iterator.go | 54 +- .../fs/commitlog/read_write_prop_test.go | 29 +- src/dbnode/persist/fs/commitlog/reader.go | 215 ++- src/dbnode/persist/fs/commitlog/types.go | 56 +- src/dbnode/persist/fs/seek_manager.go | 61 +- src/dbnode/persist/fs/seek_manager_test.go | 15 +- src/dbnode/sharding/types.go | 20 +- src/dbnode/storage/block/types.go | 3 +- src/dbnode/storage/bootstrap.go | 162 +- .../storage/bootstrap/bootstrap_mock.go | 164 +- .../storage/bootstrap/bootstrapper/base.go | 332 ++-- .../bootstrap/bootstrapper/base_data_step.go | 136 -- .../bootstrap/bootstrapper/base_index_step.go | 141 -- .../bootstrap/bootstrapper/base_test.go | 559 ++---- .../bootstrapper/commitlog/options.go | 42 +- .../bootstrapper/commitlog/source.go | 1538 ++++++----------- .../commitlog/source_data_proto_test.go | 8 +- .../commitlog/source_data_test.go | 520 ++---- .../commitlog/source_index_test.go | 289 ++-- .../commitlog/source_prop_test.go | 39 +- .../bootstrap/bootstrapper/commitlog/types.go | 26 +- .../bootstrap/bootstrapper/fs/source.go | 331 ++-- .../bootstrapper/fs/source_data_test.go | 378 ++-- .../bootstrapper/fs/source_index_test.go | 62 +- .../storage/bootstrap/bootstrapper/noop.go | 76 +- .../bootstrap/bootstrapper/noop_test.go | 41 +- .../bootstrap/bootstrapper/peers/source.go | 168 +- .../bootstrapper/peers/source_data_test.go | 195 ++- .../bootstrapper/peers/source_index_test.go | 78 +- .../bootstrapper/peers/source_test.go | 10 +- .../bootstrapper/uninitialized/source.go | 72 +- .../bootstrapper/uninitialized/source_test.go | 28 +- .../bootstrap/namespace_results_map_gen.go | 299 ++++ .../namespace_results_new_map_gen.go | 95 + ..._time_map_gen.go => namespaces_map_gen.go} | 110 +- ...w_map_gen.go => namespaces_new_map_gen.go} | 22 +- src/dbnode/storage/bootstrap/noop.go | 13 +- src/dbnode/storage/bootstrap/process.go | 268 ++- .../storage/bootstrap/result/result_data.go | 29 +- .../bootstrap/result/result_data_test.go | 197 +-- .../storage/bootstrap/result/result_index.go | 10 + .../storage/bootstrap/result/shard_ranges.go | 12 +- src/dbnode/storage/bootstrap/result/types.go | 9 +- src/dbnode/storage/bootstrap/types.go | 173 +- src/dbnode/storage/bootstrap/util.go | 732 ++++++++ src/dbnode/storage/bootstrap_test.go | 58 +- src/dbnode/storage/database.go | 12 +- src/dbnode/storage/database_test.go | 4 +- src/dbnode/storage/fs_merge_with_mem.go | 6 + src/dbnode/storage/mediator.go | 2 + src/dbnode/storage/namespace.go | 160 +- .../namespace_bootstrap_data_accumulator.go | 104 ++ ...mespace_bootstrap_data_accumulator_test.go | 159 ++ src/dbnode/storage/namespace_test.go | 60 +- src/dbnode/storage/repair.go | 2 +- src/dbnode/storage/repair_test.go | 11 +- src/dbnode/storage/series/buffer.go | 15 +- .../storage/series/buffer_proto_test.go | 9 +- src/dbnode/storage/series/buffer_test.go | 164 +- src/dbnode/storage/series/lookup/entry.go | 18 + .../storage/series/lookup/lookup_mock.go | 66 + src/dbnode/storage/series/series.go | 173 +- src/dbnode/storage/series/series_all_test.go | 71 +- src/dbnode/storage/series/series_mock.go | 53 +- .../storage/series/series_parallel_test.go | 8 +- src/dbnode/storage/series/series_test.go | 238 ++- src/dbnode/storage/series/types.go | 54 +- src/dbnode/storage/series/util.go | 94 + .../series_wired_list_interaction_test.go | 22 +- src/dbnode/storage/shard.go | 239 ++- src/dbnode/storage/shard_insert_queue.go | 2 +- src/dbnode/storage/shard_race_prop_test.go | 5 +- src/dbnode/storage/shard_test.go | 121 +- src/dbnode/storage/storage_mock.go | 100 +- src/dbnode/storage/types.go | 90 +- src/x/generated-source-files.mk | 14 +- src/x/ident/bytes_id.go | 53 + src/x/test/reporter.go | 7 + 91 files changed, 5709 insertions(+), 4903 deletions(-) delete mode 100644 src/cmd/tools/verify_commitlogs/README.MD delete mode 100644 src/cmd/tools/verify_commitlogs/main/main.go create mode 100644 src/dbnode/namespace/utils.go delete mode 100644 src/dbnode/storage/bootstrap/bootstrapper/base_data_step.go delete mode 100644 src/dbnode/storage/bootstrap/bootstrapper/base_index_step.go create mode 100644 src/dbnode/storage/bootstrap/namespace_results_map_gen.go create mode 100644 src/dbnode/storage/bootstrap/namespace_results_new_map_gen.go rename src/dbnode/storage/bootstrap/{bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go => namespaces_map_gen.go} (73%) rename src/dbnode/storage/bootstrap/{bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go => namespaces_new_map_gen.go} (86%) create mode 100644 src/dbnode/storage/bootstrap/util.go create mode 100644 src/dbnode/storage/namespace_bootstrap_data_accumulator.go create mode 100644 src/dbnode/storage/namespace_bootstrap_data_accumulator_test.go create mode 100644 src/dbnode/storage/series/lookup/lookup_mock.go create mode 100644 src/dbnode/storage/series/util.go diff --git a/.fossa.yml b/.fossa.yml index a1176e32df..f9e3d2f988 100755 --- a/.fossa.yml +++ b/.fossa.yml @@ -86,10 +86,6 @@ analyze: type: go target: github.com/m3db/m3/src/cmd/tools/read_index_ids/main path: src/cmd/tools/read_index_ids/main - - name: github.com/m3db/m3/src/cmd/tools/verify_commitlogs/main - type: go - target: github.com/m3db/m3/src/cmd/tools/verify_commitlogs/main - path: src/cmd/tools/verify_commitlogs/main - name: github.com/m3db/m3/src/cmd/tools/verify_index_files/main type: go target: github.com/m3db/m3/src/cmd/tools/verify_index_files/main diff --git a/Makefile b/Makefile index b5cdb515ee..440e7caf89 100644 --- a/Makefile +++ b/Makefile @@ -33,6 +33,7 @@ thrift_output_dir := generated/thrift/rpc thrift_rules_dir := generated/thrift vendor_prefix := vendor cache_policy ?= recently_read +genny_target ?= genny-all BUILD := $(abspath ./bin) VENDOR := $(m3_package_path)/$(vendor_prefix) @@ -88,7 +89,6 @@ TOOLS := \ read_index_files \ clone_fileset \ dtest \ - verify_commitlogs \ verify_index_files \ carbon_load \ docs_test \ @@ -333,7 +333,7 @@ asset-gen-$(SUBDIR): install-tools genny-gen-$(SUBDIR): install-tools @echo "--- Generating genny files $(SUBDIR)" @[ ! -f $(SELF_DIR)/src/$(SUBDIR)/generated-source-files.mk ] || \ - PATH=$(combined_bin_paths):$(PATH) make -f $(SELF_DIR)/src/$(SUBDIR)/generated-source-files.mk genny-all + PATH=$(combined_bin_paths):$(PATH) make -f $(SELF_DIR)/src/$(SUBDIR)/generated-source-files.mk $(genny_target) @PATH=$(combined_bin_paths):$(PATH) bash -c "source ./scripts/auto-gen-helpers.sh && gen_cleanup_dir '*_gen.go' $(SELF_DIR)/src/$(SUBDIR)/ && gen_cleanup_dir '*_gen_test.go' $(SELF_DIR)/src/$(SUBDIR)/" .PHONY: license-gen-$(SUBDIR) @@ -474,7 +474,7 @@ metalint: install-gometalinter install-linter-badtime install-linter-importorder # Tests that all currently generated types match their contents if they were regenerated .PHONY: test-all-gen test-all-gen: all-gen - @test "$(shell git diff --exit-code --shortstat 2>/dev/null)" = "" || (git diff --text --exit-code && echo "Check git status, there are dirty files" && exit 1) + @test "$(shell git --no-pager diff --exit-code --shortstat 2>/dev/null)" = "" || (git --no-pager diff --text --exit-code && echo "Check git status, there are dirty files" && exit 1) @test "$(shell git status --exit-code --porcelain 2>/dev/null | grep "^??")" = "" || (git status --exit-code --porcelain && echo "Check git status, there are untracked files" && exit 1) # Runs a fossa license report diff --git a/scripts/auto-gen-helpers.sh b/scripts/auto-gen-helpers.sh index 7f86ea5972..0c3acdcb2e 100755 --- a/scripts/auto-gen-helpers.sh +++ b/scripts/auto-gen-helpers.sh @@ -27,9 +27,9 @@ revert_copyright_only_change() { # generated file will not contain the copyright notice and thus it will # add in the copyright (with the new year). local FILE=$0 - numDiffLines=$(git diff --text -U0 $FILE | # Get file text diffs with no context. - grep -E -v '^\+\+\+|^---' | # Exclude file descriptors. - grep -E '^-|^\+' | # Get only line diffs. + numDiffLines=$(git --no-pager diff --text -U0 $FILE | # Get file text diffs with no context. + grep -E -v '^\+\+\+|^---' | # Exclude file descriptors. + grep -E '^-|^\+' | # Get only line diffs. grep -Evc '^-// Copyright \(c\)|^\+// Copyright \(c\)') # Exclude copyrights and get the number of lines remaining. if [ $numDiffLines = 0 ]; then git checkout -- "$FILE" 2> /dev/null # Remove changes, since the only change was the copyright year. diff --git a/src/cmd/tools/verify_commitlogs/README.MD b/src/cmd/tools/verify_commitlogs/README.MD deleted file mode 100644 index ab797357c4..0000000000 --- a/src/cmd/tools/verify_commitlogs/README.MD +++ /dev/null @@ -1,25 +0,0 @@ -# verify_commitlogs - -`verify_commitlogs` is a utility to verify a set of commit logs to ensure they are valid. It's also useful for testing / benchmarking the commitlog bootstrapper. Note that it requires the commitlogs to be present in a folder called "commitlogs" inside of the directory provided as the -path-prefix argument. - -# Usage - -```bash -$ git clone git@github.com:m3db/m3.git -$ make verify_commitlogs -$ ./bin/verify_commitlogs -h -``` - -# Example usage -```bash -./verify_commitlogs \ - -path-prefix /var/lib/m3db \ - -namespace metrics \ - -block-size 2h \ - -start-unix-timestamp 1507667028 \ - -end-unix-timestamp 1507677000 \ - -debug-listen-address :8080 \ -``` - -Some of the flags in the example above are optional, and there are other flags -that are not shown. Run `./verify_commitlogs -h` to see all possible flags. \ No newline at end of file diff --git a/src/cmd/tools/verify_commitlogs/main/main.go b/src/cmd/tools/verify_commitlogs/main/main.go deleted file mode 100644 index b8099b8132..0000000000 --- a/src/cmd/tools/verify_commitlogs/main/main.go +++ /dev/null @@ -1,285 +0,0 @@ -// Copyright (c) 2017 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 main - -import ( - "flag" - "io" - "log" - "net/http" - _ "net/http/pprof" - "os" - "strconv" - "strings" - "time" - - "github.com/m3db/m3/src/cmd/tools" - "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/encoding/m3tsz" - "github.com/m3db/m3/src/dbnode/persist/fs" - "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/retention" - "github.com/m3db/m3/src/dbnode/storage/block" - "github.com/m3db/m3/src/dbnode/storage/bootstrap" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" - commitlogsrc "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/ts" - "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" - "github.com/m3db/m3/src/x/pool" - xtime "github.com/m3db/m3/src/x/time" - - "go.uber.org/zap" -) - -var flagParser = flag.NewFlagSet("Verify Commitlogs", flag.ExitOnError) - -var ( - pathPrefixArg = flagParser.String("path-prefix", "/var/lib/m3db", "Path prefix - must contain a folder called 'commitlogs'") - namespaceArg = flagParser.String("namespace", "metrics", "Namespace") - blockSizeArg = flagParser.Duration("block-size", 10*time.Minute, "Block size") - flushSizeArg = flagParser.Int("flush-size", 524288, "Flush size of commit log") - bootstrapRetentionArg = flagParser.Duration("retention", 48*time.Hour, "Retention") - shardsCountArg = flagParser.Int("shards-count", 8192, "Shards count - set number too bootstrap all shards in range") - shardsArg = flagParser.String("shards", "", "Shards - set comma separated list of shards") - debugListenAddressArg = flagParser.String("debug-listen-address", "", "Debug listen address - if set will expose pprof, i.e. ':8080'") - startUnixTimestampArg = flagParser.Int64("start-unix-timestamp", 0, "Start unix timestamp (Seconds) - If set will bootstrap all data after this timestamp up to end-unix-timestamp, defaults to reading from the beginning of the first commitlog") - // 1<<63-62135596801 is the largest possible time.Time that can be represented - // without causing overflow when passed to functions in the time package - endUnixTimestampArg = flagParser.Int64("end-unix-timestamp", 1<<63-62135596801, "End unix timestamp (Seconds) - If set will bootstrap all data from start-unix-timestamp up to this timestamp, defaults to reading up to the end of the last commitlog") - readConcurrency = flagParser.Int("read-concurrency", 4, "Commitlog read concurrency") - encodingConcurrency = flagParser.Int("encoding-concurrency", 4, "Encoding concurrency") - mergeShardsConcurrency = flagParser.Int("merge-shards-concurrency", 4, "Merge shards concurrency") -) - -func main() { - flagParser.Parse(os.Args[1:]) - - var ( - pathPrefix = *pathPrefixArg - namespaceStr = *namespaceArg - blockSize = *blockSizeArg - flushSize = *flushSizeArg - bootstrapRetention = *bootstrapRetentionArg - shardsCount = *shardsCountArg - shards = *shardsArg - debugListenAddress = *debugListenAddressArg - startUnixTimestamp = *startUnixTimestampArg - endUnixTimestamp = *endUnixTimestampArg - ) - - rawLogger, err := zap.NewDevelopment() - if err != nil { - log.Fatalf("unable to create logger: %+v", err) - } - log := rawLogger.Sugar() - - if debugListenAddress != "" { - go func() { - log.Infof("starting debug listen server at '%s'\n", debugListenAddress) - err := http.ListenAndServe(debugListenAddress, http.DefaultServeMux) - if err != nil { - log.Fatalf("could not start debug listen server at '%s': %v", debugListenAddress, err) - } - }() - } - - shardTimeRanges := result.ShardTimeRanges{} - - start := time.Unix(startUnixTimestamp, 0) - end := time.Unix(endUnixTimestamp, 0) - - // Ony used for logging - var shardsAll []uint32 - // Handle comma-delimited shard list 1,3,5, etc - if strings.TrimSpace(shards) != "" { - for _, shard := range strings.Split(shards, ",") { - shard = strings.TrimSpace(shard) - if shard == "" { - log.Fatalf("Invalid shard list: '%s'", shards) - } - value, err := strconv.Atoi(shard) - if err != nil { - log.Fatalf("could not parse shard '%s': %v", shard, err) - } - rng := xtime.Range{Start: start, End: end} - shardTimeRanges[uint32(value)] = xtime.NewRanges(rng) - shardsAll = append(shardsAll, uint32(value)) - } - // Or just handled up to N (shard-count) shards - } else if shardsCount > 0 { - for i := uint32(0); i < uint32(shardsCount); i++ { - rng := xtime.Range{Start: start, End: end} - shardTimeRanges[i] = xtime.NewRanges(rng) - shardsAll = append(shardsAll, i) - } - } else { - log.Info("Either the shards or shards-count argument need to be valid") - flag.Usage() - os.Exit(1) - } - - log.With( - zap.String("pathPrefix", pathPrefix), - zap.String("namespace", namespaceStr), - zap.Uint32s("shards", shardsAll), - ).Infof("configured") - - instrumentOpts := instrument.NewOptions(). - SetLogger(rawLogger) - - retentionOpts := retention.NewOptions(). - SetBlockSize(blockSize). - SetRetentionPeriod(bootstrapRetention). - SetBufferPast(1 * time.Minute). - SetBufferFuture(1 * time.Minute) - - blockOpts := block.NewOptions() - - encoderPoolOpts := pool. - NewObjectPoolOptions(). - SetSize(25165824). - SetRefillLowWatermark(0.001). - SetRefillHighWatermark(0.002) - encoderPool := encoding.NewEncoderPool(encoderPoolOpts) - - iteratorPoolOpts := pool.NewObjectPoolOptions(). - SetSize(2048). - SetRefillLowWatermark(0.01). - SetRefillHighWatermark(0.02) - iteratorPool := encoding.NewReaderIteratorPool(iteratorPoolOpts) - multiIteratorPool := encoding.NewMultiReaderIteratorPool(iteratorPoolOpts) - - segmentReaderPoolOpts := pool.NewObjectPoolOptions(). - SetSize(16384). - SetRefillLowWatermark(0.01). - SetRefillHighWatermark(0.02) - segmentReaderPool := xio.NewSegmentReaderPool(segmentReaderPoolOpts) - - bytesPool := tools.NewCheckedBytesPool() - - encodingOpts := encoding.NewOptions(). - SetEncoderPool(encoderPool). - SetReaderIteratorPool(iteratorPool). - SetBytesPool(bytesPool). - SetSegmentReaderPool(segmentReaderPool) - - encoderPool.Init(func() encoding.Encoder { - return m3tsz.NewEncoder(time.Time{}, nil, true, encodingOpts) - }) - - iteratorPool.Init(func(r io.Reader, _ namespace.SchemaDescr) encoding.ReaderIterator { - return m3tsz.NewReaderIterator(r, true, encodingOpts) - }) - - multiIteratorPool.Init(func(r io.Reader, descr namespace.SchemaDescr) encoding.ReaderIterator { - iter := iteratorPool.Get() - iter.Reset(r, descr) - return iter - }) - bytesPool.Init() - - segmentReaderPool.Init() - - blockPoolOpts := pool.NewObjectPoolOptions(). - SetSize(4194304). - SetRefillLowWatermark(0.001). - SetRefillHighWatermark(0.002) - blockPool := block.NewDatabaseBlockPool(blockPoolOpts) - blockPool.Init(func() block.DatabaseBlock { - return block.NewDatabaseBlock(time.Time{}, 0, ts.Segment{}, blockOpts, namespace.Context{}) - }) - - blockOpts = blockOpts. - SetEncoderPool(encoderPool). - SetReaderIteratorPool(iteratorPool). - SetMultiReaderIteratorPool(multiIteratorPool). - SetDatabaseBlockPool(blockPool). - SetSegmentReaderPool(segmentReaderPool) - - resultOpts := result.NewOptions(). - SetInstrumentOptions(instrumentOpts). - SetDatabaseBlockOptions(blockOpts) - - fsOpts := fs.NewOptions(). - SetInstrumentOptions(instrumentOpts). - SetFilePathPrefix(pathPrefix) - - commitLogOpts := commitlog.NewOptions(). - SetInstrumentOptions(instrumentOpts). - SetFilesystemOptions(fsOpts). - SetFlushSize(flushSize). - SetBlockSize(blockSize). - SetReadConcurrency(*readConcurrency). - SetBytesPool(bytesPool) - - opts := commitlogsrc.NewOptions(). - SetResultOptions(resultOpts). - SetCommitLogOptions(commitLogOpts). - SetEncodingConcurrency(*encodingConcurrency). - SetMergeShardsConcurrency(*mergeShardsConcurrency) - - log.Infof("bootstrapping") - - // Don't bootstrap anything else - next := bootstrapper.NewNoOpAllBootstrapperProvider() - inspection, err := fs.InspectFilesystem(fsOpts) - if err != nil { - log.Fatal(err.Error()) - } - provider, err := commitlogsrc.NewCommitLogBootstrapperProvider(opts, inspection, next) - if err != nil { - log.Fatal(err.Error()) - } - - source, err := provider.Provide() - if err != nil { - log.Fatal(err.Error()) - } - - nsID := ident.StringID(namespaceStr) - runOpts := bootstrap.NewRunOptions(). - // Dont save intermediate results - SetPersistConfig(bootstrap.PersistConfig{Enabled: false}) - nsMetadata, err := namespace.NewMetadata(nsID, namespace.NewOptions().SetRetentionOptions(retentionOpts)) - if err != nil { - log.Fatal(err.Error()) - } - result, err := source.BootstrapData(nsMetadata, shardTimeRanges, runOpts) - if err != nil { - log.Fatalf("failed to bootstrap: %v", err) - } - - log.With( - zap.Any("shardResults", len(result.ShardResults())), - zap.Any("unfulfilled", len(result.Unfulfilled())), - ).Info("bootstrapped") - - for shard, result := range result.ShardResults() { - log.With( - zap.Any("shard", shard), - zap.Any("series", result.AllSeries().Len()), - ).Info("shard result") - } -} diff --git a/src/dbnode/generated-source-files.mk b/src/dbnode/generated-source-files.mk index dde7785251..fc1c7d29be 100644 --- a/src/dbnode/generated-source-files.mk +++ b/src/dbnode/generated-source-files.mk @@ -13,17 +13,18 @@ genny-all: genny-map-all genny-arraypool-all genny-leakcheckpool-all genny-list- # Map generation rule for all generated maps .PHONY: genny-map-all -genny-map-all: \ - genny-map-client-received-blocks \ - genny-map-storage-block-retriever \ - genny-map-storage-bootstrap-result \ - genny-map-storage \ - genny-map-storage-namespace-metadata \ - genny-map-storage-repair \ - genny-map-storage-index-results \ - genny-map-storage-index-aggregate-values \ - genny-map-storage-index-aggregation-results \ - genny-map-storage-bootstrap-bootstrapper-commitlog +genny-map-all: \ + genny-map-client-received-blocks \ + genny-map-storage-block-retriever \ + genny-map-storage-bootstrap-namespaces \ + genny-map-storage-bootstrap-namespace-results \ + genny-map-storage-bootstrap-result \ + genny-map-storage \ + genny-map-storage-namespace-metadata \ + genny-map-storage-repair \ + genny-map-storage-index-results \ + genny-map-storage-index-aggregate-values \ + genny-map-storage-index-aggregation-results \ # Map generation rule for client/receivedBlocksMap .PHONY: genny-map-client-received-blocks @@ -51,6 +52,34 @@ genny-map-storage-block-retriever: mv -f $(m3db_package_path)/src/dbnode/storage/block/map_gen.go $(m3db_package_path)/src/dbnode/storage/block/retriever_map_gen.go mv -f $(m3db_package_path)/src/dbnode/storage/block/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/block/retriever_new_map_gen.go +# Map generation rule for storage/bootstrap/NamespacesMap +.PHONY: genny-map-storage-bootstrap-namespaces +genny-map-storage-bootstrap-namespaces: + cd $(m3x_package_path) && make idhashmap-gen \ + pkg=bootstrap \ + value_type=Namespace \ + target_package=$(m3db_package)/src/dbnode/storage/bootstrap \ + rename_type_prefix=Namespaces \ + rename_constructor=NewNamespacesMap \ + rename_constructor_options=NamespacesMapOptions + # Rename both generated map and constructor files + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/namespaces_map_gen.go + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/namespaces_new_map_gen.go + +# Map generation rule for storage/bootstrap/NamespaceResultsMap +.PHONY: genny-map-storage-bootstrap-namespace-results +genny-map-storage-bootstrap-namespace-results: + cd $(m3x_package_path) && make idhashmap-gen \ + pkg=bootstrap \ + value_type=NamespaceResult \ + target_package=$(m3db_package)/src/dbnode/storage/bootstrap \ + rename_type_prefix=NamespaceResults \ + rename_constructor=NewNamespaceResultsMap \ + rename_constructor_options=NamespaceResultsMapOptions + # Rename both generated map and constructor files + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/namespace_results_map_gen.go + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/namespace_results_new_map_gen.go + # Map generation rule for storage/bootstrap/result/Map .PHONY: genny-map-storage-bootstrap-result genny-map-storage-bootstrap-result: @@ -102,19 +131,6 @@ genny-map-storage-repair: value_type=ReplicaSeriesBlocksMetadata \ target_package=$(m3db_package)/src/dbnode/storage/repair -# Map generation rule for storage/bootstrap/bootstrapper/commitlog -.PHONY: genny-map-storage-bootstrap-bootstrapper-commitlog -genny-map-storage-bootstrap-bootstrapper-commitlog: - cd $(m3x_package_path) && make idhashmap-gen \ - pkg=commitlog \ - value_type=metadataAndEncodersByTime \ - target_package=$(m3db_package)/src/dbnode/storage/bootstrap/bootstrapper/commitlog \ - rename_constructor=newMetadataAndEncodersByTimeMap \ - rename_constructor_options=newMetadataAndEncodersByTimeMapOptions - # Rename both generated map and constructor files - mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go - mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go - # Map generation rule for persist/fs .PHONY: genny-map-persist-fs genny-map-persist-fs: diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index c36595f5c3..7c8d6ac8da 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -24,6 +24,7 @@ //go:generate sh -c "mockgen -package=xio $PACKAGE/src/dbnode/x/xio SegmentReader,SegmentReaderPool | genclean -pkg $PACKAGE/src/dbnode/x/xio -out $GOPATH/src/$PACKAGE/src/dbnode/x/xio/io_mock.go" //go:generate sh -c "mockgen -package=digest -destination=$GOPATH/src/$PACKAGE/src/dbnode/digest/digest_mock.go $PACKAGE/src/dbnode/digest ReaderWithDigest" //go:generate sh -c "mockgen -package=series $PACKAGE/src/dbnode/storage/series DatabaseSeries,QueryableBlockRetriever | genclean -pkg $PACKAGE/src/dbnode/storage/series -out $GOPATH/src/$PACKAGE/src/dbnode/storage/series/series_mock.go" +//go:generate sh -c "mockgen -package=lookup $PACKAGE/src/dbnode/storage/series/lookup OnReleaseReadWriteRef | genclean -pkg $PACKAGE/src/dbnode/storage/series/lookup -out $GOPATH/src/$PACKAGE/src/dbnode/storage/series/lookup/lookup_mock.go" //go:generate sh -c "mockgen -package=config $PACKAGE/src/cmd/services/m3dbnode/config BootstrapConfigurationValidator | genclean -pkg $PACKAGE/src/cmd/services/m3dbnode/config -out $GOPATH/src/$PACKAGE/src/cmd/services/m3dbnode/config/config_mock.go" // mockgen rules for generating mocks for unexported interfaces (file mode) diff --git a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go index 60c598a1f9..17a38e70ee 100644 --- a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go +++ b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go @@ -27,12 +27,12 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -117,22 +117,23 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { // Setup the test bootstrapper to only return success when a signal is sent. signalCh := make(chan struct{}) - bootstrapper, err := commitlogBootstrapperProvider.Provide() + bs, err := commitlogBootstrapperProvider.Provide() require.NoError(t, err) test := newTestBootstrapperSource(testBootstrapperSourceOptions{ - readData: func( - _ namespace.Metadata, - shardTimeRanges result.ShardTimeRanges, - _ bootstrap.RunOptions, - ) (result.DataBootstrapResult, error) { + read: func( + namespaces bootstrap.Namespaces, + ) (bootstrap.NamespaceResults, error) { <-signalCh - result := result.NewDataBootstrapResult() // Mark all as unfulfilled so the commitlog bootstrapper will be called after - result.SetUnfulfilled(shardTimeRanges) - return result, nil + noopNone := bootstrapper.NewNoOpNoneBootstrapperProvider() + bs, err := noopNone.Provide() + if err != nil { + return bootstrap.NamespaceResults{}, err + } + return bs.Bootstrap(namespaces) }, - }, bootstrapOpts, bootstrapper) + }, bootstrapOpts, bs) processOpts := bootstrap.NewProcessOptions(). SetTopologyMapProvider(setup). diff --git a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go index f4d3252761..80ef12e351 100644 --- a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go +++ b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go @@ -27,12 +27,12 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -131,21 +131,23 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { // Setup the test bootstrapper to only return success when a signal is sent. signalCh := make(chan struct{}) - bootstrapper, err := commitlogBootstrapperProvider.Provide() + bs, err := commitlogBootstrapperProvider.Provide() require.NoError(t, err) + test := newTestBootstrapperSource(testBootstrapperSourceOptions{ - readData: func( - _ namespace.Metadata, - shardTimeRanges result.ShardTimeRanges, - _ bootstrap.RunOptions, - ) (result.DataBootstrapResult, error) { + read: func( + namespaces bootstrap.Namespaces, + ) (bootstrap.NamespaceResults, error) { <-signalCh - result := result.NewDataBootstrapResult() // Mark all as unfulfilled so the commitlog bootstrapper will be called after - result.SetUnfulfilled(shardTimeRanges) - return result, nil + noopNone := bootstrapper.NewNoOpNoneBootstrapperProvider() + bs, err := noopNone.Provide() + if err != nil { + return bootstrap.NamespaceResults{}, err + } + return bs.Bootstrap(namespaces) }, - }, bootstrapOpts, bootstrapper) + }, bootstrapOpts, bs) processOpts := bootstrap.NewProcessOptions(). SetTopologyMapProvider(setup). diff --git a/src/dbnode/integration/bootstrap_helpers.go b/src/dbnode/integration/bootstrap_helpers.go index 0737254303..f11a4a3ac2 100644 --- a/src/dbnode/integration/bootstrap_helpers.go +++ b/src/dbnode/integration/bootstrap_helpers.go @@ -25,6 +25,7 @@ package integration import ( "testing" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/runtime" @@ -32,7 +33,6 @@ import ( "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/stretchr/testify/require" ) @@ -43,11 +43,6 @@ func newTestBootstrapperSource( next bootstrap.Bootstrapper, ) bootstrap.BootstrapperProvider { src := testBootstrapperSource{} - if opts.can != nil { - src.can = opts.can - } else { - src.can = func(bootstrap.Strategy) bool { return true } - } if opts.availableData != nil { src.availableData = opts.availableData @@ -57,14 +52,6 @@ func newTestBootstrapperSource( } } - if opts.readData != nil { - src.readData = opts.readData - } else { - src.readData = func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.DataBootstrapResult, error) { - return result.NewDataBootstrapResult(), nil - } - } - if opts.availableIndex != nil { src.availableIndex = opts.availableIndex } else { @@ -73,11 +60,11 @@ func newTestBootstrapperSource( } } - if opts.readIndex != nil { - src.readIndex = opts.readIndex + if opts.read != nil { + src.read = opts.read } else { - src.readIndex = func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.IndexBootstrapResult, error) { - return result.NewIndexBootstrapResult(), nil + src.read = func(namespaces bootstrap.Namespaces) (bootstrap.NamespaceResults, error) { + return bootstrap.NewNamespaceResults(namespaces), nil } } @@ -111,25 +98,17 @@ type testBootstrapper struct { } type testBootstrapperSourceOptions struct { - can func(bootstrap.Strategy) bool availableData func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.ShardTimeRanges, error) - readData func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.DataBootstrapResult, error) availableIndex func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.ShardTimeRanges, error) - readIndex func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.IndexBootstrapResult, error) + read func(namespaces bootstrap.Namespaces) (bootstrap.NamespaceResults, error) } var _ bootstrap.Source = &testBootstrapperSource{} type testBootstrapperSource struct { - can func(bootstrap.Strategy) bool availableData func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.ShardTimeRanges, error) - readData func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.DataBootstrapResult, error) availableIndex func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.ShardTimeRanges, error) - readIndex func(namespace.Metadata, result.ShardTimeRanges, bootstrap.RunOptions) (result.IndexBootstrapResult, error) -} - -func (t testBootstrapperSource) Can(strategy bootstrap.Strategy) bool { - return t.can(strategy) + read func(namespaces bootstrap.Namespaces) (bootstrap.NamespaceResults, error) } func (t testBootstrapperSource) AvailableData( @@ -140,14 +119,6 @@ func (t testBootstrapperSource) AvailableData( return t.availableData(ns, shardsTimeRanges, runOpts) } -func (t testBootstrapperSource) ReadData( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - return t.readData(ns, shardsTimeRanges, opts) -} - func (t testBootstrapperSource) AvailableIndex( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -156,12 +127,10 @@ func (t testBootstrapperSource) AvailableIndex( return t.availableIndex(ns, shardsTimeRanges, runOpts) } -func (t testBootstrapperSource) ReadIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - return t.readIndex(ns, shardsTimeRanges, opts) +func (t testBootstrapperSource) Read( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + return t.read(namespaces) } func (t testBootstrapperSource) String() string { diff --git a/src/dbnode/namespace/utils.go b/src/dbnode/namespace/utils.go new file mode 100644 index 0000000000..d31698f95e --- /dev/null +++ b/src/dbnode/namespace/utils.go @@ -0,0 +1,45 @@ +// Copyright (c) 2019 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 namespace + +import ( + "fmt" + + "github.com/m3db/m3/src/x/ident" +) + +// MustBuildMetadatas builds a list of metadatas for testing with given +// index enabled option and ids. +func MustBuildMetadatas(indexEnabled bool, ids ...string) []Metadata { + idxOpts := NewIndexOptions().SetEnabled(indexEnabled) + opts := NewOptions().SetIndexOptions(idxOpts) + mds := make([]Metadata, 0, len(ids)) + for _, id := range ids { + md, err := NewMetadata(ident.StringID(id), opts) + if err != nil { + panic(fmt.Sprintf("error during MustBuildMetadatas: %v", err)) + } + + mds = append(mds, md) + } + + return mds +} diff --git a/src/dbnode/persist/fs/commitlog/commit_log_mock.go b/src/dbnode/persist/fs/commitlog/commit_log_mock.go index 2a82e332ac..4eb0e1eb0a 100644 --- a/src/dbnode/persist/fs/commitlog/commit_log_mock.go +++ b/src/dbnode/persist/fs/commitlog/commit_log_mock.go @@ -202,14 +202,11 @@ func (mr *MockIteratorMockRecorder) Next() *gomock.Call { } // Current mocks base method -func (m *MockIterator) Current() (ts.Series, ts.Datapoint, time0.Unit, ts.Annotation) { +func (m *MockIterator) Current() LogEntry { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Current") - ret0, _ := ret[0].(ts.Series) - ret1, _ := ret[1].(ts.Datapoint) - ret2, _ := ret[2].(time0.Unit) - ret3, _ := ret[3].(ts.Annotation) - return ret0, ret1, ret2, ret3 + ret0, _ := ret[0].(LogEntry) + return ret0 } // Current indicates an expected call of Current diff --git a/src/dbnode/persist/fs/commitlog/commit_log_test.go b/src/dbnode/persist/fs/commitlog/commit_log_test.go index e3d3894d24..f82d496f06 100644 --- a/src/dbnode/persist/fs/commitlog/commit_log_test.go +++ b/src/dbnode/persist/fs/commitlog/commit_log_test.go @@ -46,22 +46,6 @@ import ( "github.com/uber-go/tally" ) -// readAllSeriesPredicateTest is the same as ReadAllSeriesPredicate except -// it asserts that the ID and the namespace are not nil. -func readAllSeriesPredicateTest() SeriesFilterPredicate { - return func(id ident.ID, namespace ident.ID) bool { - if id == nil { - panic(fmt.Sprintf("series ID passed to series predicate is nil")) - } - - if namespace == nil { - panic(fmt.Sprintf("namespace ID passed to series predicate is nil")) - } - - return true - } -} - type mockTime struct { sync.Mutex t time.Time @@ -317,9 +301,8 @@ type seriesTestWritesAndReadPosition struct { func assertCommitLogWritesByIterating(t *testing.T, l *commitLog, writes []testWrite) { iterOpts := IteratorOpts{ - CommitLogOptions: l.opts, - FileFilterPredicate: ReadAllPredicate(), - SeriesFilterPredicate: readAllSeriesPredicateTest(), + CommitLogOptions: l.opts, + FileFilterPredicate: ReadAllPredicate(), } iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) @@ -340,15 +323,16 @@ func assertCommitLogWritesByIterating(t *testing.T, l *commitLog, writes []testW } for iter.Next() { - series, datapoint, unit, annotation := iter.Current() + entry := iter.Current() - seriesWrites := writesBySeries[series.ID.String()] + id := entry.Series.ID.String() + seriesWrites := writesBySeries[id] write := seriesWrites.writes[seriesWrites.readPosition] - write.assert(t, series, datapoint, unit, annotation) + write.assert(t, entry.Series, entry.Datapoint, entry.Unit, entry.Annotation) seriesWrites.readPosition++ - writesBySeries[series.ID.String()] = seriesWrites + writesBySeries[id] = seriesWrites } require.NoError(t, iter.Err()) @@ -444,9 +428,8 @@ func TestReadCommitLogMissingMetadata(t *testing.T) { // Make sure we don't panic / deadlock iterOpts := IteratorOpts{ - CommitLogOptions: opts, - FileFilterPredicate: ReadAllPredicate(), - SeriesFilterPredicate: readAllSeriesPredicateTest(), + CommitLogOptions: opts, + FileFilterPredicate: ReadAllPredicate(), } iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) @@ -494,7 +477,7 @@ func TestCommitLogReaderIsNotReusable(t *testing.T) { require.Equal(t, 2, len(files)) // Assert commitlog cannot be opened more than once - reader := newCommitLogReader(opts, readAllSeriesPredicateTest()) + reader := newCommitLogReader(commitLogReaderOptions{commitLogOptions: opts}) _, err = reader.Open(files[0]) require.NoError(t, err) reader.Close() @@ -550,9 +533,8 @@ func TestCommitLogIteratorUsesPredicateFilterForNonCorruptFiles(t *testing.T) { // Assert that the commitlog iterator honors the predicate and only uses // 2 of the 3 files. iterOpts := IteratorOpts{ - CommitLogOptions: opts, - FileFilterPredicate: commitLogPredicate, - SeriesFilterPredicate: readAllSeriesPredicateTest(), + CommitLogOptions: opts, + FileFilterPredicate: commitLogPredicate, } iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) @@ -595,9 +577,8 @@ func TestCommitLogIteratorUsesPredicateFilterForCorruptFiles(t *testing.T) { // Assert that the corrupt file is returned from the iterator. iterOpts := IteratorOpts{ - CommitLogOptions: opts, - FileFilterPredicate: ReadAllPredicate(), - SeriesFilterPredicate: readAllSeriesPredicateTest(), + CommitLogOptions: opts, + FileFilterPredicate: ReadAllPredicate(), } iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) @@ -612,9 +593,8 @@ func TestCommitLogIteratorUsesPredicateFilterForCorruptFiles(t *testing.T) { } iterOpts = IteratorOpts{ - CommitLogOptions: opts, - FileFilterPredicate: ignoreCorruptPredicate, - SeriesFilterPredicate: readAllSeriesPredicateTest(), + CommitLogOptions: opts, + FileFilterPredicate: ignoreCorruptPredicate, } iter, corruptFiles, err = NewIterator(iterOpts) require.NoError(t, err) diff --git a/src/dbnode/persist/fs/commitlog/iterator.go b/src/dbnode/persist/fs/commitlog/iterator.go index f0723fa455..0211b076c3 100644 --- a/src/dbnode/persist/fs/commitlog/iterator.go +++ b/src/dbnode/persist/fs/commitlog/iterator.go @@ -25,8 +25,6 @@ import ( "io" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/ts" - xtime "github.com/m3db/m3/src/x/time" "github.com/uber-go/tally" "go.uber.org/zap" @@ -41,24 +39,17 @@ type iteratorMetrics struct { } type iterator struct { - opts Options - scope tally.Scope - metrics iteratorMetrics - log *zap.Logger - files []persist.CommitLogFile - reader commitLogReader - read iteratorRead - err error - seriesPred SeriesFilterPredicate - setRead bool - closed bool -} - -type iteratorRead struct { - series ts.Series - datapoint ts.Datapoint - unit xtime.Unit - annotation []byte + iterOpts IteratorOpts + opts Options + scope tally.Scope + metrics iteratorMetrics + log *zap.Logger + files []persist.CommitLogFile + reader commitLogReader + read LogEntry + err error + setRead bool + closed bool } // ReadAllPredicate can be passed as the ReadCommitLogPredicate for callers @@ -82,14 +73,14 @@ func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []ErrorWith scope := iops.MetricsScope() return &iterator{ - opts: opts, - scope: scope, + iterOpts: iterOpts, + opts: opts, + scope: scope, metrics: iteratorMetrics{ readsErrors: scope.Counter("reads.errors"), }, - log: iops.Logger(), - files: filteredFiles, - seriesPred: iterOpts.SeriesFilterPredicate, + log: iops.Logger(), + files: filteredFiles, }, filteredCorruptFiles, nil } @@ -103,7 +94,7 @@ func (i *iterator) Next() bool { } } var err error - i.read.series, i.read.datapoint, i.read.unit, i.read.annotation, err = i.reader.Read() + i.read, err = i.reader.Read() if err == io.EOF { closeErr := i.closeAndResetReader() if closeErr != nil { @@ -127,12 +118,12 @@ func (i *iterator) Next() bool { return true } -func (i *iterator) Current() (ts.Series, ts.Datapoint, xtime.Unit, ts.Annotation) { +func (i *iterator) Current() LogEntry { read := i.read if i.hasError() || i.closed || !i.setRead { - read = iteratorRead{} + read = LogEntry{} } - return read.series, read.datapoint, read.unit, read.annotation + return read } func (i *iterator) Err() error { @@ -166,7 +157,10 @@ func (i *iterator) nextReader() bool { file := i.files[0] i.files = i.files[1:] - reader := newCommitLogReader(i.opts, i.seriesPred) + reader := newCommitLogReader(commitLogReaderOptions{ + commitLogOptions: i.opts, + returnMetadataAsRef: i.iterOpts.ReturnMetadataAsRef, + }) index, err := reader.Open(file.FilePath) if err != nil { i.err = err diff --git a/src/dbnode/persist/fs/commitlog/read_write_prop_test.go b/src/dbnode/persist/fs/commitlog/read_write_prop_test.go index 91ba2d8eb6..ada6737cb1 100644 --- a/src/dbnode/persist/fs/commitlog/read_write_prop_test.go +++ b/src/dbnode/persist/fs/commitlog/read_write_prop_test.go @@ -87,9 +87,8 @@ func TestCommitLogReadWrite(t *testing.T) { i := 0 iterOpts := IteratorOpts{ - CommitLogOptions: opts, - FileFilterPredicate: ReadAllPredicate(), - SeriesFilterPredicate: ReadAllSeriesPredicate(), + CommitLogOptions: opts, + FileFilterPredicate: ReadAllPredicate(), } iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) @@ -110,9 +109,13 @@ func TestCommitLogReadWrite(t *testing.T) { } for ; iter.Next(); i++ { - series, datapoint, _, _ := iter.Current() - require.NoError(t, iter.Err()) + logEntry := iter.Current() + var ( + series = logEntry.Series + datapoint = logEntry.Datapoint + ) + require.NoError(t, iter.Err()) seriesWrites := writesBySeries[series.ID.String()] write := seriesWrites.writes[seriesWrites.readPosition] @@ -484,9 +487,8 @@ func newInitState( func (s *clState) writesArePresent(writes ...generatedWrite) error { writesOnDisk := make(map[string]map[xtime.UnixNano]generatedWrite) iterOpts := IteratorOpts{ - CommitLogOptions: s.opts, - FileFilterPredicate: ReadAllPredicate(), - SeriesFilterPredicate: ReadAllSeriesPredicate(), + CommitLogOptions: s.opts, + FileFilterPredicate: ReadAllPredicate(), } // Based on the corruption type this could return some corrupt files // or it could not, so we don't check it. @@ -502,8 +504,15 @@ func (s *clState) writesArePresent(writes ...generatedWrite) error { count := 0 for iter.Next() { - series, datapoint, unit, annotation := iter.Current() - idString := series.ID.String() + logEntry := iter.Current() + var ( + series = logEntry.Series + datapoint = logEntry.Datapoint + unit = logEntry.Unit + annotation = logEntry.Annotation + idString = series.ID.String() + ) + seriesMap, ok := writesOnDisk[idString] if !ok { seriesMap = make(map[xtime.UnixNano]generatedWrite) diff --git a/src/dbnode/persist/fs/commitlog/reader.go b/src/dbnode/persist/fs/commitlog/reader.go index c6ef49e24e..ac5ea3fed2 100644 --- a/src/dbnode/persist/fs/commitlog/reader.go +++ b/src/dbnode/persist/fs/commitlog/reader.go @@ -36,9 +36,13 @@ import ( "github.com/m3db/m3/src/x/pool" "github.com/m3db/m3/src/x/serialize" xtime "github.com/m3db/m3/src/x/time" + + "github.com/uber-go/atomic" ) var ( + commitLogFileReadCounter = atomic.NewUint64(0) + // var instead of const so we can modify them in tests. defaultDecodeEntryBufSize = 1024 decoderInBufChanSize = 1000 @@ -54,32 +58,19 @@ var ( errCommitLogReaderMissingMetadata = errors.New("commit log reader encountered a datapoint without corresponding metadata") ) -// ReadAllSeriesPredicate can be passed as the seriesPredicate for callers -// that want a convenient way to read all series in the commitlogs -func ReadAllSeriesPredicate() SeriesFilterPredicate { - return func(id ident.ID, namespace ident.ID) bool { return true } -} - -type seriesMetadata struct { - ts.Series - passedPredicate bool -} - type commitLogReader interface { // Open opens the commit log for reading Open(filePath string) (int64, error) // Read returns the next id and data pair or error, will return io.EOF at end of volume - Read() (ts.Series, ts.Datapoint, xtime.Unit, ts.Annotation, error) + Read() (LogEntry, error) // Close the reader Close() error } type reader struct { - opts Options - - seriesPredicate SeriesFilterPredicate + opts commitLogReaderOptions logEntryBytes []byte tagDecoder serialize.TagDecoder @@ -89,25 +80,38 @@ type reader struct { infoDecoder *msgpack.Decoder infoDecoderStream msgpack.ByteDecoderStream hasBeenOpened bool + fileReadID uint64 + + metadataLookup map[uint64]ts.Series + namespacesRead []namespaceRead + seriesIDReused *ident.ReuseableBytesID +} - metadataLookup map[uint64]seriesMetadata - namespacesRead []ident.ID +type namespaceRead struct { + namespaceIDBytes []byte + namespaceIDRef ident.ID } -func newCommitLogReader(opts Options, seriesPredicate SeriesFilterPredicate) commitLogReader { +type commitLogReaderOptions struct { + commitLogOptions Options + // returnMetadataAsRef indicates to not allocate metadata results. + returnMetadataAsRef bool +} + +func newCommitLogReader(opts commitLogReaderOptions) commitLogReader { tagDecoderCheckedBytes := checked.NewBytes(nil, nil) tagDecoderCheckedBytes.IncRef() return &reader{ opts: opts, - seriesPredicate: seriesPredicate, - logEntryBytes: make([]byte, 0, opts.FlushSize()), - metadataLookup: make(map[uint64]seriesMetadata), - tagDecoder: opts.FilesystemOptions().TagDecoderPool().Get(), + logEntryBytes: make([]byte, 0, opts.commitLogOptions.FlushSize()), + metadataLookup: make(map[uint64]ts.Series), + tagDecoder: opts.commitLogOptions.FilesystemOptions().TagDecoderPool().Get(), tagDecoderCheckedBytes: tagDecoderCheckedBytes, - checkedBytesPool: opts.BytesPool(), - chunkReader: newChunkReader(opts.FlushSize()), - infoDecoder: msgpack.NewDecoder(opts.FilesystemOptions().DecodingOptions()), + checkedBytesPool: opts.commitLogOptions.BytesPool(), + chunkReader: newChunkReader(opts.commitLogOptions.FlushSize()), + infoDecoder: msgpack.NewDecoder(opts.commitLogOptions.FilesystemOptions().DecodingOptions()), infoDecoderStream: msgpack.NewByteDecoderStream(nil), + seriesIDReused: ident.NewReuseableBytesID(), } } @@ -129,6 +133,9 @@ func (r *reader) Open(filePath string) (int64, error) { r.Close() return 0, err } + + r.fileReadID = commitLogFileReadCounter.Inc() + index := info.Index return index, nil } @@ -144,49 +151,41 @@ func (r *reader) readInfo() (schema.LogInfo, error) { } // Read reads the next log entry in order. -func (r *reader) Read() ( - series ts.Series, - datapoint ts.Datapoint, - unit xtime.Unit, - annotation ts.Annotation, - err error, -) { - var ( - entry schema.LogEntry - metadata seriesMetadata - ) - for !metadata.passedPredicate { - err = r.readLogEntry() - if err != nil { - return ts.Series{}, ts.Datapoint{}, xtime.Unit(0), ts.Annotation(nil), err - } - - entry, err = msgpack.DecodeLogEntryFast(r.logEntryBytes) - if err != nil { - return ts.Series{}, ts.Datapoint{}, xtime.Unit(0), ts.Annotation(nil), err - } - - metadata, err = r.seriesMetadataForEntry(entry) - if err != nil { - return ts.Series{}, ts.Datapoint{}, xtime.Unit(0), ts.Annotation(nil), err - } +func (r *reader) Read() (LogEntry, error) { + err := r.readLogEntry() + if err != nil { + return LogEntry{}, err } - series = metadata.Series + entry, err := msgpack.DecodeLogEntryFast(r.logEntryBytes) + if err != nil { + return LogEntry{}, err + } - datapoint = ts.Datapoint{ - Timestamp: time.Unix(0, entry.Timestamp), - Value: entry.Value, + metadata, err := r.seriesMetadataForEntry(entry) + if err != nil { + return LogEntry{}, err } - unit = xtime.Unit(entry.Unit) + result := LogEntry{ + Series: metadata, + Datapoint: ts.Datapoint{ + Timestamp: time.Unix(0, entry.Timestamp), + Value: entry.Value, + }, + Unit: xtime.Unit(entry.Unit), + Metadata: LogEntryMetadata{ + FileReadID: r.fileReadID, + SeriesUniqueIndex: entry.Index, + }, + } if len(entry.Annotation) > 0 { // Copy annotation to prevent reference to pooled byte slice - annotation = append(ts.Annotation(nil), ts.Annotation(entry.Annotation)...) + result.Annotation = append(ts.Annotation(nil), ts.Annotation(entry.Annotation)...) } - return series, datapoint, unit, annotation, nil + return result, nil } func (r *reader) readLogEntry() error { @@ -207,9 +206,64 @@ func (r *reader) readLogEntry() error { return nil } +func (r *reader) namespaceIDReused(id []byte) ident.ID { + var namespaceID ident.ID + for _, ns := range r.namespacesRead { + if bytes.Equal(ns.namespaceIDBytes, id) { + namespaceID = ns.namespaceIDRef + break + } + } + if namespaceID == nil { + // Take a copy and keep around to reuse later. + namespaceBytes := append(make([]byte, 0, len(id)), id...) + namespaceID = ident.BytesID(namespaceBytes) + r.namespacesRead = append(r.namespacesRead, namespaceRead{ + namespaceIDBytes: namespaceBytes, + namespaceIDRef: namespaceID, + }) + } + return namespaceID +} + func (r *reader) seriesMetadataForEntry( entry schema.LogEntry, -) (seriesMetadata, error) { +) (ts.Series, error) { + if r.opts.returnMetadataAsRef { + // NB(r): This is a fast path for callers where nothing + // is cached locally in terms of metadata lookup and the + // caller is returned just references to all the bytes in + // the backing commit log file the first and only time + // we encounter the series metadata, and then the refs are + // invalid on the next call to metadata. + if len(entry.Metadata) == 0 { + // Valid, nothing to return here and caller will already + // have processed metadata for this entry (based on the + // FileReadID and the SeriesUniqueIndex returned). + return ts.Series{}, nil + } + + decoded, err := msgpack.DecodeLogMetadataFast(entry.Metadata) + if err != nil { + return ts.Series{}, err + } + + // Reset the series ID being returned. + r.seriesIDReused.Reset(decoded.ID) + // Find or allocate the namespace ID. + namespaceID := r.namespaceIDReused(decoded.Namespace) + metadata := ts.Series{ + UniqueIndex: entry.Index, + ID: r.seriesIDReused, + Namespace: namespaceID, + Shard: decoded.Shard, + EncodedTags: ts.EncodedTags(decoded.EncodedTags), + } + return metadata, nil + } + + // We only check for previously returned metadata + // if we're allocating results and can hold onto them. metadata, ok := r.metadataLookup[entry.Index] if ok { // If the metadata already exists, we can skip this step. @@ -218,12 +272,12 @@ func (r *reader) seriesMetadataForEntry( if len(entry.Metadata) == 0 { // Expected metadata but not encoded. - return seriesMetadata{}, errCommitLogReaderMissingMetadata + return ts.Series{}, errCommitLogReaderMissingMetadata } decoded, err := msgpack.DecodeLogMetadataFast(entry.Metadata) if err != nil { - return seriesMetadata{}, err + return ts.Series{}, err } id := r.checkedBytesPool.Get(len(decoded.ID)) @@ -231,21 +285,10 @@ func (r *reader) seriesMetadataForEntry( id.AppendAll(decoded.ID) // Find or allocate the namespace ID. - var namespaceID ident.ID - for _, ns := range r.namespacesRead { - if bytes.Equal(ns.Bytes(), decoded.Namespace) { - namespaceID = ns - break - } - } - if namespaceID == nil { - // Take a copy and keep around to reuse later. - namespaceID = ident.BytesID(append([]byte(nil), decoded.Namespace...)) - r.namespacesRead = append(r.namespacesRead, namespaceID) - } + namespaceID := r.namespaceIDReused(decoded.Namespace) var ( - idPool = r.opts.IdentifierPool() + idPool = r.opts.commitLogOptions.IdentifierPool() tags ident.Tags tagBytesLen = len(decoded.EncodedTags) ) @@ -260,20 +303,17 @@ func (r *reader) seriesMetadataForEntry( } err = r.tagDecoder.Err() if err != nil { - return seriesMetadata{}, err + return ts.Series{}, err } } seriesID := idPool.BinaryID(id) - metadata = seriesMetadata{ - Series: ts.Series{ - UniqueIndex: entry.Index, - ID: seriesID, - Namespace: namespaceID, - Shard: decoded.Shard, - Tags: tags, - }, - passedPredicate: r.seriesPredicate(seriesID, namespaceID), + metadata = ts.Series{ + UniqueIndex: entry.Index, + ID: seriesID, + Namespace: namespaceID, + Shard: decoded.Shard, + Tags: tags, } r.metadataLookup[entry.Index] = metadata @@ -284,7 +324,12 @@ func (r *reader) seriesMetadataForEntry( } func (r *reader) Close() error { - return r.chunkReader.fd.Close() + err := r.chunkReader.fd.Close() + // NB(r): Reset to free resources, but explicitly do + // not support reopening for now. + *r = reader{} + r.hasBeenOpened = true + return err } func resizeBufferOrGrowIfNeeded(buf []byte, length int) []byte { diff --git a/src/dbnode/persist/fs/commitlog/types.go b/src/dbnode/persist/fs/commitlog/types.go index 0532965d42..cd13dae10b 100644 --- a/src/dbnode/persist/fs/commitlog/types.go +++ b/src/dbnode/persist/fs/commitlog/types.go @@ -84,26 +84,56 @@ type CommitLog interface { QueueLength() int64 } -// Iterator provides an iterator for commit logs +// LogEntry is a commit log entry being read. +type LogEntry struct { + Series ts.Series + Datapoint ts.Datapoint + Unit xtime.Unit + Annotation ts.Annotation + Metadata LogEntryMetadata +} + +// LogEntryMetadata is a set of metadata about a commit log entry being read. +type LogEntryMetadata struct { + // FileReadID is a unique index for the current commit log + // file that is being read (only unique per-process). + FileReadID uint64 + // SeriesUniqueIndex is the series unique index relative to the + // current commit log file being read. + SeriesUniqueIndex uint64 +} + +// Iterator provides an iterator for commit logs. type Iterator interface { - // Next returns whether the iterator has the next value + // Next returns whether the iterator has the next value. Next() bool - // Current returns the current commit log entry - Current() (ts.Series, ts.Datapoint, xtime.Unit, ts.Annotation) + // Current returns the current commit log entry. + Current() LogEntry - // Err returns an error if an error occurred + // Err returns an error if an error occurred. Err() error - // Close the iterator + // Close the iterator. Close() } -// IteratorOpts is a struct that contains coptions for the Iterator +// IteratorOpts is a struct that contains coptions for the Iterator. type IteratorOpts struct { - CommitLogOptions Options - FileFilterPredicate FileFilterPredicate - SeriesFilterPredicate SeriesFilterPredicate + CommitLogOptions Options + FileFilterPredicate FileFilterPredicate + // ReturnMetadataAsRef will return all series metadata such as ID, + // tags and namespace as a reference instead of returning pooled + // or allocated byte/string/ID references. + // Useful if caller does not hold onto the result between calls to + // the next read log entry and wants to avoid allocations and pool + // contention. + // Note: Series metadata will only be set on the result of a log + // entry read if the series is read for the first time for the + // combined tuple of FileReadID and SeriesUniqueIndex returned by + // the LogEntryMetadata. EncodedTags will also be returned + // instead of Tags on the series metadata. + ReturnMetadataAsRef bool } // Options represents the options for the commit log. @@ -199,9 +229,3 @@ type FileFilterInfo struct { // FileFilterPredicate is a predicate that allows the caller to determine // which commitlogs the iterator should read from. type FileFilterPredicate func(f FileFilterInfo) bool - -// SeriesFilterPredicate is a predicate that determines whether datapoints for a given series -// should be returned from the Commit log reader. The predicate is pushed down to the -// reader level to prevent having to run the same function for every datapoint for a -// given series. -type SeriesFilterPredicate func(id ident.ID, namespace ident.ID) bool diff --git a/src/dbnode/persist/fs/seek_manager.go b/src/dbnode/persist/fs/seek_manager.go index 00aeed8e4e..e2497c5ed4 100644 --- a/src/dbnode/persist/fs/seek_manager.go +++ b/src/dbnode/persist/fs/seek_manager.go @@ -32,6 +32,7 @@ import ( xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/pool" + xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" "go.uber.org/zap" @@ -40,6 +41,7 @@ import ( const ( seekManagerCloseInterval = time.Second reusableSeekerResourcesPoolSize = 10 + concurrentCacheShardIndices = 16 ) var ( @@ -89,6 +91,8 @@ type seekerManager struct { status seekerManagerStatus isUpdatingLease bool + cacheShardIndicesWorkers xsync.WorkerPool + // seekersByShardIdx provides access to all seekers, first partitioned by // shard and then by block start. seekersByShardIdx []*seekersByTime @@ -99,6 +103,7 @@ type seekerManager struct { newOpenSeekerFn newOpenSeekerFn sleepFn func(d time.Duration) openCloseLoopDoneCh chan struct{} + // Pool of seeker resources that can be used to open new seekers. reusableSeekerResourcesPool pool.ObjectPool } @@ -163,12 +168,18 @@ func NewSeekerManager( return NewReusableSeekerResources(opts) }) + // NB(r): Since this is mainly IO bound work, perfectly + // fine to do this in parallel. + cacheShardIndicesWorkers := xsync.NewWorkerPool(concurrentCacheShardIndices) + cacheShardIndicesWorkers.Init() + m := &seekerManager{ bytesPool: bytesPool, filePathPrefix: opts.FilePathPrefix(), opts: opts, blockRetrieverOpts: blockRetrieverOpts, fetchConcurrency: blockRetrieverOpts.FetchConcurrency(), + cacheShardIndicesWorkers: cacheShardIndicesWorkers, logger: opts.InstrumentOptions().Logger(), openCloseLoopDoneCh: make(chan struct{}), reusableSeekerResourcesPool: reusableSeekerResourcesPool, @@ -208,8 +219,11 @@ func (m *seekerManager) Open( } func (m *seekerManager) CacheShardIndices(shards []uint32) error { - multiErr := xerrors.NewMultiError() - + var ( + multiErr = xerrors.NewMultiError() + resultsLock sync.Mutex + wg sync.WaitGroup + ) for _, shard := range shards { byTime := m.seekersByTime(shard) @@ -218,34 +232,44 @@ func (m *seekerManager) CacheShardIndices(shards []uint32) error { byTime.accessed = true byTime.Unlock() - if err := m.openAnyUnopenSeekersFn(byTime); err != nil { - multiErr = multiErr.Add(err) - } + wg.Add(1) + m.cacheShardIndicesWorkers.Go(func() { + if err := m.openAnyUnopenSeekersFn(byTime); err != nil { + resultsLock.Lock() + multiErr = multiErr.Add(err) + resultsLock.Unlock() + } + wg.Done() + }) } + wg.Wait() return multiErr.FinalError() } func (m *seekerManager) Test(id ident.ID, shard uint32, start time.Time) (bool, error) { + startNano := xtime.ToUnixNano(start) byTime := m.seekersByTime(shard) // Try fast RLock() first. byTime.RLock() - startNano := xtime.ToUnixNano(start) - seekers, ok := byTime.seekers[startNano] - - // Seekers are open: good to test but still hold RLock while doing so - if ok && seekers.active.wg == nil { + if seekers, ok := byTime.seekers[startNano]; ok && seekers.active.wg == nil { + // Seekers are open: good to test but still hold RLock while doing so idExists := seekers.active.bloomFilter.Test(id.Bytes()) byTime.RUnlock() return idExists, nil - } else { - byTime.RUnlock() } + byTime.RUnlock() + byTime.Lock() defer byTime.Unlock() + // Check if raced with another call to this method + if seekers, ok := byTime.seekers[startNano]; ok && seekers.active.wg == nil { + return seekers.active.bloomFilter.Test(id.Bytes()), nil + } + seekersAndBloom, err := m.getOrOpenSeekersWithLock(startNano, byTime) if err != nil { return false, err @@ -762,6 +786,7 @@ func (m *seekerManager) seekersByTime(shard uint32) *seekersByTime { m.RUnlock() return byTime } + m.RUnlock() m.Lock() @@ -774,14 +799,10 @@ func (m *seekerManager) seekersByTime(shard uint32) *seekersByTime { } seekersByShardIdx := make([]*seekersByTime, shard+1) - - for i := range seekersByShardIdx { - if i < len(m.seekersByShardIdx) { - seekersByShardIdx[i] = m.seekersByShardIdx[i] - continue - } - seekersByShardIdx[i] = &seekersByTime{ - shard: uint32(i), + idx := copy(seekersByShardIdx, m.seekersByShardIdx) + for ; idx < len(seekersByShardIdx); idx++ { + seekersByShardIdx[idx] = &seekersByTime{ + shard: uint32(idx), seekers: make(map[xtime.UnixNano]rotatableSeekers), } } diff --git a/src/dbnode/persist/fs/seek_manager_test.go b/src/dbnode/persist/fs/seek_manager_test.go index 3459e1b90a..513235ddd9 100644 --- a/src/dbnode/persist/fs/seek_manager_test.go +++ b/src/dbnode/persist/fs/seek_manager_test.go @@ -44,18 +44,20 @@ func TestSeekerManagerCacheShardIndices(t *testing.T) { shards := []uint32{2, 5, 9, 478, 1023} m := NewSeekerManager(nil, testDefaultOpts, defaultTestBlockRetrieverOptions).(*seekerManager) - var byTimes []*seekersByTime + byTimes := make(map[uint32]*seekersByTime) + var mu sync.Mutex m.openAnyUnopenSeekersFn = func(byTime *seekersByTime) error { - byTimes = append(byTimes, byTime) + mu.Lock() + byTimes[byTime.shard] = byTime + mu.Unlock() return nil } require.NoError(t, m.CacheShardIndices(shards)) - // Assert captured byTime objects match expectations require.Equal(t, len(shards), len(byTimes)) - for idx, shard := range shards { - byTimes[idx].shard = shard + for _, shard := range shards { + byTimes[shard].shard = shard } // Assert seeksByShardIdx match expectations @@ -63,13 +65,14 @@ func TestSeekerManagerCacheShardIndices(t *testing.T) { for _, shard := range shards { shardSet[shard] = struct{}{} } + for shard, byTime := range m.seekersByShardIdx { _, exists := shardSet[uint32(shard)] if !exists { require.False(t, byTime.accessed) } else { require.True(t, byTime.accessed) - require.Equal(t, uint32(shard), byTime.shard) + require.Equal(t, int(shard), int(byTime.shard)) } } } diff --git a/src/dbnode/sharding/types.go b/src/dbnode/sharding/types.go index d587e56b04..d7babd5ae5 100644 --- a/src/dbnode/sharding/types.go +++ b/src/dbnode/sharding/types.go @@ -25,33 +25,33 @@ import ( "github.com/m3db/m3/src/x/ident" ) -// HashGen generates HashFn based on the length of shards +// HashGen generates HashFn based on the length of shards. type HashGen func(length int) HashFn -// HashFn is a sharding hash function +// HashFn is a sharding hash function. type HashFn func(id ident.ID) uint32 // ShardSet contains a sharding function and a set of shards, this interface -// allows for potentially out of order shard sets +// allows for potentially out of order shard sets. type ShardSet interface { - // All returns a slice to the shards in this set + // All returns a slice to the shards in this set. All() []shard.Shard - // AllIDs returns a slice to the shard IDs in this set + // AllIDs returns a slice to the shard IDs in this set. AllIDs() []uint32 - // Lookup will return a shard for a given identifier + // Lookup will return a shard for a given identifier. Lookup(id ident.ID) uint32 - // LookupStateByID returns the state of the shard with a given ID + // LookupStateByID returns the state of the shard with a given ID. LookupStateByID(shardID uint32) (shard.State, error) - // Min returns the smallest shard owned by this shard set + // Min returns the smallest shard owned by this shard set. Min() uint32 - // Max returns the largest shard owned by this shard set + // Max returns the largest shard owned by this shard set. Max() uint32 - // HashFn returns the sharding hash function + // HashFn returns the sharding hash function. HashFn() HashFn } diff --git a/src/dbnode/storage/block/types.go b/src/dbnode/storage/block/types.go index 35a4ab6f17..132b1fb20c 100644 --- a/src/dbnode/storage/block/types.go +++ b/src/dbnode/storage/block/types.go @@ -135,7 +135,6 @@ type NewDatabaseBlockFn func() DatabaseBlock // DatabaseBlock is the interface for a DatabaseBlock type DatabaseBlock interface { - // StartTime returns the start time of the block. StartTime() time.Time @@ -297,12 +296,14 @@ type DatabaseShardBlockRetriever interface { // DatabaseBlockRetrieverManager creates and holds block retrievers // for different namespaces. type DatabaseBlockRetrieverManager interface { + // Retriever provides the DatabaseBlockRetriever for the given namespace. Retriever(nsMetadata namespace.Metadata) (DatabaseBlockRetriever, error) } // DatabaseShardBlockRetrieverManager creates and holds shard block // retrievers binding shards to an existing retriever. type DatabaseShardBlockRetrieverManager interface { + // ShardRetriever provides the DatabaseShardBlockRetriever for the given shard. ShardRetriever(shard uint32) DatabaseShardBlockRetriever } diff --git a/src/dbnode/storage/bootstrap.go b/src/dbnode/storage/bootstrap.go index 395103fb3f..4aea4ad4f0 100644 --- a/src/dbnode/storage/bootstrap.go +++ b/src/dbnode/storage/bootstrap.go @@ -22,15 +22,18 @@ package storage import ( "errors" + "fmt" "sync" "time" "github.com/m3db/m3/src/dbnode/clock" "github.com/m3db/m3/src/dbnode/storage/bootstrap" xerrors "github.com/m3db/m3/src/x/errors" + "github.com/m3db/m3/src/x/instrument" "github.com/uber-go/tally" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) var ( @@ -59,6 +62,12 @@ var ( errBootstrapEnqueued = errors.New("database bootstrapping enqueued bootstrap") ) +const ( + bootstrapRetryInterval = 10 * time.Second +) + +type bootstrapFn func() error + type bootstrapManager struct { sync.RWMutex @@ -66,7 +75,9 @@ type bootstrapManager struct { mediator databaseMediator opts Options log *zap.Logger + bootstrapFn bootstrapFn nowFn clock.NowFn + sleepFn sleepFn processProvider bootstrap.ProcessProvider state BootstrapState hasPending bool @@ -80,15 +91,18 @@ func newBootstrapManager( opts Options, ) databaseBootstrapManager { scope := opts.InstrumentOptions().MetricsScope() - return &bootstrapManager{ + m := &bootstrapManager{ database: database, mediator: mediator, opts: opts, log: opts.InstrumentOptions().Logger(), nowFn: opts.ClockOptions().NowFn(), + sleepFn: time.Sleep, processProvider: opts.BootstrapProcessProvider(), status: scope.Gauge("bootstrapped"), } + m.bootstrapFn = m.bootstrap + return m } func (m *bootstrapManager) IsBootstrapped() bool { @@ -102,7 +116,7 @@ func (m *bootstrapManager) LastBootstrapCompletionTime() (time.Time, bool) { return m.lastBootstrapCompletionTime, !m.lastBootstrapCompletionTime.IsZero() } -func (m *bootstrapManager) Bootstrap() error { +func (m *bootstrapManager) Bootstrap() (BootstrapResult, error) { m.Lock() switch m.state { case Bootstrapping: @@ -114,7 +128,7 @@ func (m *bootstrapManager) Bootstrap() error { // reshard occurs and we need to bootstrap more shards. m.hasPending = true m.Unlock() - return errBootstrapEnqueued + return BootstrapResult{AlreadyBootstrapping: true}, errBootstrapEnqueued default: m.state = Bootstrapping } @@ -125,12 +139,13 @@ func (m *bootstrapManager) Bootstrap() error { m.mediator.DisableFileOps() defer m.mediator.EnableFileOps() - // Keep performing bootstraps until none pending - multiErr := xerrors.NewMultiError() - for { - err := m.bootstrap() - if err != nil { - multiErr = multiErr.Add(err) + // Keep performing bootstraps until none pending and no error returned. + var result BootstrapResult + for i := 0; true; i++ { + // NB(r): Decouple implementation of bootstrap so can override in tests. + bootstrapErr := m.bootstrapFn() + if bootstrapErr != nil { + result.ErrorsBootstrap = append(result.ErrorsBootstrap, bootstrapErr) } m.Lock() @@ -143,9 +158,24 @@ func (m *bootstrapManager) Bootstrap() error { } m.Unlock() - if !currPending { - break + if currPending { + // NB(r): Requires another bootstrap. + continue + } + + if bootstrapErr != nil { + // NB(r): Last bootstrap failed, since this could be due to transient + // failure we retry the bootstrap again. This is to avoid operators + // needing to manually intervene for cases where failures are transient. + m.log.Warn("retrying bootstrap after backoff", + zap.Duration("backoff", bootstrapRetryInterval), + zap.Int("numRetries", i+1)) + m.sleepFn(bootstrapRetryInterval) + continue } + + // No pending bootstraps and last finished successfully. + break } // NB(xichen): in order for bootstrapped data to be flushed to disk, a tick @@ -158,7 +188,7 @@ func (m *bootstrapManager) Bootstrap() error { // across the cluster. m.lastBootstrapCompletionTime = m.nowFn() - return multiErr.FinalError() + return result, nil } func (m *bootstrapManager) Report() { @@ -177,27 +207,109 @@ func (m *bootstrapManager) bootstrap() error { return err } - // NB(xichen): each bootstrapper should be responsible for choosing the most - // efficient way of bootstrapping database shards, be it sequential or parallel. - multiErr := xerrors.NewMultiError() - namespaces, err := m.database.GetOwnedNamespaces() if err != nil { return err } - startBootstrap := m.nowFn() + accmulators := make([]bootstrap.NamespaceDataAccumulator, 0, len(namespaces)) + defer func() { + // Close all accumulators at bootstrap completion, only error + // it returns is if already closed, so this is a code bug if ever + // an error returned. + for _, accumulator := range accmulators { + if err := accumulator.Close(); err != nil { + instrument.EmitAndLogInvariantViolation(m.opts.InstrumentOptions(), + func(l *zap.Logger) { + l.Error("could not close bootstrap data accumulator", + zap.Error(err)) + }) + } + } + }() + + targets := make([]bootstrap.ProcessNamespace, 0, len(namespaces)) + var uniqueShards map[uint32]struct{} + for _, namespace := range namespaces { + namespaceShards := namespace.GetOwnedShards() + bootstrapShards := make([]uint32, 0, len(namespaceShards)) + if uniqueShards == nil { + uniqueShards = make(map[uint32]struct{}, len(namespaceShards)) + } + + for _, shard := range namespaceShards { + if shard.IsBootstrapped() { + continue + } + + uniqueShards[shard.ID()] = struct{}{} + bootstrapShards = append(bootstrapShards, shard.ID()) + } + + accumulator := NewDatabaseNamespaceDataAccumulator(namespace) + accmulators = append(accmulators, accumulator) + + targets = append(targets, bootstrap.ProcessNamespace{ + Metadata: namespace.Metadata(), + Shards: bootstrapShards, + DataAccumulator: accumulator, + }) + } + + start := m.nowFn() + logFields := []zapcore.Field{ + zap.Int("numShards", len(uniqueShards)), + } + m.log.Info("bootstrap started", logFields...) + + // Run the bootstrap. + bootstrapResult, err := process.Run(start, targets) + + logFields = append(logFields, + zap.Duration("bootstrapDuration", m.nowFn().Sub(start))) + + if err != nil { + m.log.Error("bootstrap failed", + append(logFields, zap.Error(err))...) + return err + } + + m.log.Info("bootstrap succeeded, marking namespaces complete", logFields...) + // Use a multi-error here because we want to at least bootstrap + // as many of the namespaces as possible. + multiErr := xerrors.NewMultiError() for _, namespace := range namespaces { - startNamespaceBootstrap := m.nowFn() - if err := namespace.Bootstrap(startBootstrap, process); err != nil { + id := namespace.ID() + result, ok := bootstrapResult.Results.Get(id) + if !ok { + err := fmt.Errorf("missing namespace from bootstrap result: %v", + id.String()) + i := m.opts.InstrumentOptions() + instrument.EmitAndLogInvariantViolation(i, func(l *zap.Logger) { + l.Error("bootstrap failed", + append(logFields, zap.Error(err))...) + }) + return err + } + + if err := namespace.Bootstrap(result); err != nil { + m.log.Info("bootstrap error", append(logFields, + []zapcore.Field{ + zap.String("namespace", id.String()), + zap.Error(err), + }..., + )..., + ) multiErr = multiErr.Add(err) } - took := m.nowFn().Sub(startNamespaceBootstrap) - m.log.Info("bootstrap finished", - zap.String("namespace", namespace.ID().String()), - zap.Duration("duration", took), - ) } - return multiErr.FinalError() + if err := multiErr.FinalError(); err != nil { + m.log.Info("bootstrap namespaces failed", + append(logFields, zap.Error(err))...) + return err + } + + m.log.Info("bootstrap success", logFields...) + return nil } diff --git a/src/dbnode/storage/bootstrap/bootstrap_mock.go b/src/dbnode/storage/bootstrap/bootstrap_mock.go index 2e9982201b..b5a804b713 100644 --- a/src/dbnode/storage/bootstrap/bootstrap_mock.go +++ b/src/dbnode/storage/bootstrap/bootstrap_mock.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/topology" + "github.com/m3db/m3/src/x/ident" "github.com/golang/mock/gomock" ) @@ -123,18 +124,85 @@ func (m *MockProcess) EXPECT() *MockProcessMockRecorder { } // Run mocks base method -func (m *MockProcess) Run(start time.Time, ns namespace.Metadata, shards []uint32) (ProcessResult, error) { +func (m *MockProcess) Run(start time.Time, namespaces []ProcessNamespace) (NamespaceResults, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Run", start, ns, shards) - ret0, _ := ret[0].(ProcessResult) + ret := m.ctrl.Call(m, "Run", start, namespaces) + ret0, _ := ret[0].(NamespaceResults) ret1, _ := ret[1].(error) return ret0, ret1 } // Run indicates an expected call of Run -func (mr *MockProcessMockRecorder) Run(start, ns, shards interface{}) *gomock.Call { +func (mr *MockProcessMockRecorder) Run(start, namespaces interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Run", reflect.TypeOf((*MockProcess)(nil).Run), start, ns, shards) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Run", reflect.TypeOf((*MockProcess)(nil).Run), start, namespaces) +} + +// MockNamespaceDataAccumulator is a mock of NamespaceDataAccumulator interface +type MockNamespaceDataAccumulator struct { + ctrl *gomock.Controller + recorder *MockNamespaceDataAccumulatorMockRecorder +} + +// MockNamespaceDataAccumulatorMockRecorder is the mock recorder for MockNamespaceDataAccumulator +type MockNamespaceDataAccumulatorMockRecorder struct { + mock *MockNamespaceDataAccumulator +} + +// NewMockNamespaceDataAccumulator creates a new mock instance +func NewMockNamespaceDataAccumulator(ctrl *gomock.Controller) *MockNamespaceDataAccumulator { + mock := &MockNamespaceDataAccumulator{ctrl: ctrl} + mock.recorder = &MockNamespaceDataAccumulatorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockNamespaceDataAccumulator) EXPECT() *MockNamespaceDataAccumulatorMockRecorder { + return m.recorder +} + +// CheckoutSeriesWithoutLock mocks base method +func (m *MockNamespaceDataAccumulator) CheckoutSeriesWithoutLock(shardID uint32, id ident.ID, tags ident.TagIterator) (CheckoutSeriesResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CheckoutSeriesWithoutLock", shardID, id, tags) + ret0, _ := ret[0].(CheckoutSeriesResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CheckoutSeriesWithoutLock indicates an expected call of CheckoutSeriesWithoutLock +func (mr *MockNamespaceDataAccumulatorMockRecorder) CheckoutSeriesWithoutLock(shardID, id, tags interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CheckoutSeriesWithoutLock", reflect.TypeOf((*MockNamespaceDataAccumulator)(nil).CheckoutSeriesWithoutLock), shardID, id, tags) +} + +// CheckoutSeriesWithLock mocks base method +func (m *MockNamespaceDataAccumulator) CheckoutSeriesWithLock(shardID uint32, id ident.ID, tags ident.TagIterator) (CheckoutSeriesResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CheckoutSeriesWithLock", shardID, id, tags) + ret0, _ := ret[0].(CheckoutSeriesResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CheckoutSeriesWithLock indicates an expected call of CheckoutSeriesWithLock +func (mr *MockNamespaceDataAccumulatorMockRecorder) CheckoutSeriesWithLock(shardID, id, tags interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CheckoutSeriesWithLock", reflect.TypeOf((*MockNamespaceDataAccumulator)(nil).CheckoutSeriesWithLock), shardID, id, tags) +} + +// Close mocks base method +func (m *MockNamespaceDataAccumulator) 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 *MockNamespaceDataAccumulatorMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockNamespaceDataAccumulator)(nil).Close)) } // MockProcessOptions is a mock of ProcessOptions interface @@ -454,48 +522,19 @@ func (mr *MockBootstrapperMockRecorder) String() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "String", reflect.TypeOf((*MockBootstrapper)(nil).String)) } -// Can mocks base method -func (m *MockBootstrapper) Can(strategy Strategy) bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Can", strategy) - ret0, _ := ret[0].(bool) - return ret0 -} - -// Can indicates an expected call of Can -func (mr *MockBootstrapperMockRecorder) Can(strategy interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Can", reflect.TypeOf((*MockBootstrapper)(nil).Can), strategy) -} - -// BootstrapData mocks base method -func (m *MockBootstrapper) BootstrapData(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, opts RunOptions) (result.DataBootstrapResult, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "BootstrapData", ns, shardsTimeRanges, opts) - ret0, _ := ret[0].(result.DataBootstrapResult) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// BootstrapData indicates an expected call of BootstrapData -func (mr *MockBootstrapperMockRecorder) BootstrapData(ns, shardsTimeRanges, opts interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BootstrapData", reflect.TypeOf((*MockBootstrapper)(nil).BootstrapData), ns, shardsTimeRanges, opts) -} - -// BootstrapIndex mocks base method -func (m *MockBootstrapper) BootstrapIndex(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, opts RunOptions) (result.IndexBootstrapResult, error) { +// Bootstrap mocks base method +func (m *MockBootstrapper) Bootstrap(namespaces Namespaces) (NamespaceResults, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "BootstrapIndex", ns, shardsTimeRanges, opts) - ret0, _ := ret[0].(result.IndexBootstrapResult) + ret := m.ctrl.Call(m, "Bootstrap", namespaces) + ret0, _ := ret[0].(NamespaceResults) ret1, _ := ret[1].(error) return ret0, ret1 } -// BootstrapIndex indicates an expected call of BootstrapIndex -func (mr *MockBootstrapperMockRecorder) BootstrapIndex(ns, shardsTimeRanges, opts interface{}) *gomock.Call { +// Bootstrap indicates an expected call of Bootstrap +func (mr *MockBootstrapperMockRecorder) Bootstrap(namespaces interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BootstrapIndex", reflect.TypeOf((*MockBootstrapper)(nil).BootstrapIndex), ns, shardsTimeRanges, opts) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockBootstrapper)(nil).Bootstrap), namespaces) } // MockSource is a mock of Source interface @@ -521,20 +560,6 @@ func (m *MockSource) EXPECT() *MockSourceMockRecorder { return m.recorder } -// Can mocks base method -func (m *MockSource) Can(strategy Strategy) bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Can", strategy) - ret0, _ := ret[0].(bool) - return ret0 -} - -// Can indicates an expected call of Can -func (mr *MockSourceMockRecorder) Can(strategy interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Can", reflect.TypeOf((*MockSource)(nil).Can), strategy) -} - // AvailableData mocks base method func (m *MockSource) AvailableData(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts RunOptions) (result.ShardTimeRanges, error) { m.ctrl.T.Helper() @@ -550,21 +575,6 @@ func (mr *MockSourceMockRecorder) AvailableData(ns, shardsTimeRanges, runOpts in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AvailableData", reflect.TypeOf((*MockSource)(nil).AvailableData), ns, shardsTimeRanges, runOpts) } -// ReadData mocks base method -func (m *MockSource) ReadData(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts RunOptions) (result.DataBootstrapResult, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ReadData", ns, shardsTimeRanges, runOpts) - ret0, _ := ret[0].(result.DataBootstrapResult) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ReadData indicates an expected call of ReadData -func (mr *MockSourceMockRecorder) ReadData(ns, shardsTimeRanges, runOpts interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadData", reflect.TypeOf((*MockSource)(nil).ReadData), ns, shardsTimeRanges, runOpts) -} - // AvailableIndex mocks base method func (m *MockSource) AvailableIndex(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, opts RunOptions) (result.ShardTimeRanges, error) { m.ctrl.T.Helper() @@ -580,17 +590,17 @@ func (mr *MockSourceMockRecorder) AvailableIndex(ns, shardsTimeRanges, opts inte return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AvailableIndex", reflect.TypeOf((*MockSource)(nil).AvailableIndex), ns, shardsTimeRanges, opts) } -// ReadIndex mocks base method -func (m *MockSource) ReadIndex(ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, opts RunOptions) (result.IndexBootstrapResult, error) { +// Read mocks base method +func (m *MockSource) Read(namespaces Namespaces) (NamespaceResults, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ReadIndex", ns, shardsTimeRanges, opts) - ret0, _ := ret[0].(result.IndexBootstrapResult) + ret := m.ctrl.Call(m, "Read", namespaces) + ret0, _ := ret[0].(NamespaceResults) ret1, _ := ret[1].(error) return ret0, ret1 } -// ReadIndex indicates an expected call of ReadIndex -func (mr *MockSourceMockRecorder) ReadIndex(ns, shardsTimeRanges, opts interface{}) *gomock.Call { +// Read indicates an expected call of Read +func (mr *MockSourceMockRecorder) Read(namespaces interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadIndex", reflect.TypeOf((*MockSource)(nil).ReadIndex), ns, shardsTimeRanges, opts) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockSource)(nil).Read), namespaces) } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/base.go b/src/dbnode/storage/bootstrap/bootstrapper/base.go index 68ba5f15ab..29a6951443 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/base.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/base.go @@ -21,12 +21,10 @@ package bootstrapper import ( - "sync" + "fmt" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" - xerrors "github.com/m3db/m3/src/x/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -76,133 +74,253 @@ func (b baseBootstrapper) String() string { return baseBootstrapperName } -func (b baseBootstrapper) Can(strategy bootstrap.Strategy) bool { - return b.src.Can(strategy) -} - -func (b baseBootstrapper) BootstrapData( - namespace namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - if shardsTimeRanges.IsEmpty() { - return result.NewDataBootstrapResult(), nil - } - step := newBootstrapDataStep(namespace, b.src, b.next, opts) - err := b.runBootstrapStep(namespace, shardsTimeRanges, step) - if err != nil { - return nil, err +func (b baseBootstrapper) Bootstrap( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + logFields := []zapcore.Field{ + zap.String("bootstrapper", b.name), } - return step.result(), nil -} -func (b baseBootstrapper) BootstrapIndex( - namespace namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - if shardsTimeRanges.IsEmpty() { - return result.NewIndexBootstrapResult(), nil - } - step := newBootstrapIndexStep(namespace, b.src, b.next, opts) - err := b.runBootstrapStep(namespace, shardsTimeRanges, step) - if err != nil { - return nil, err + curr := bootstrap.Namespaces{ + Namespaces: bootstrap.NewNamespacesMap(bootstrap.NamespacesMapOptions{}), } - return step.result(), nil -} + for _, elem := range namespaces.Namespaces.Iter() { + id := elem.Key() -func (b baseBootstrapper) runBootstrapStep( - namespace namespace.Metadata, - totalRanges result.ShardTimeRanges, - step bootstrapStep, -) error { - prepareResult, err := step.prepare(totalRanges) - if err != nil { - return err - } + // Shallow copy the namespace, do not modify namespaces input to bootstrap call. + currNamespace := elem.Value() - var ( - wg sync.WaitGroup - currStatus, nextStatus bootstrapStepStatus - currErr, nextErr error - nextAttempted bool - ) - currRanges := prepareResult.currAvailable - nextRanges := totalRanges.Copy() - nextRanges.Subtract(currRanges) - if !nextRanges.IsEmpty() && - b.Can(bootstrap.BootstrapParallel) && - b.next.Can(bootstrap.BootstrapParallel) { - // If ranges can be bootstrapped now from the next source then begin attempt now - nextAttempted = true - wg.Add(1) - go func() { - defer wg.Done() - nextStatus, nextErr = step.runNextStep(nextRanges) - }() - } + b.logShardTimeRanges("bootstrap from source requested", + logFields, currNamespace) - min, max := currRanges.MinMax() - logFields := []zapcore.Field{ - zap.String("source", b.name), - zap.String("namespace", namespace.ID().String()), - zap.Time("from", min), - zap.Time("to", max), - zap.Duration("range", max.Sub(min)), - zap.Int("shards", len(currRanges)), + dataAvailable, err := b.src.AvailableData(currNamespace.Metadata, + currNamespace.DataRunOptions.ShardTimeRanges.Copy(), + currNamespace.DataRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + currNamespace.DataRunOptions.ShardTimeRanges = dataAvailable + + // Prepare index if required. + if currNamespace.Metadata.Options().IndexOptions().Enabled() { + indexAvailable, err := b.src.AvailableIndex(currNamespace.Metadata, + currNamespace.IndexRunOptions.ShardTimeRanges.Copy(), + currNamespace.IndexRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + currNamespace.IndexRunOptions.ShardTimeRanges = indexAvailable + } + + // Set the namespace options for the current bootstrapper source. + curr.Namespaces.Set(id, currNamespace) + + // Log the metadata about bootstrapping this namespace based on + // the availability returned. + b.logShardTimeRanges("bootstrap from source ready after availability query", + logFields, currNamespace) } - b.log.Info("bootstrapping from source starting", logFields...) nowFn := b.opts.ClockOptions().NowFn() begin := nowFn() - currStatus, currErr = step.runCurrStep(currRanges) + b.log.Info("bootstrap from source started", logFields...) + currResults, err := b.src.Read(curr) logFields = append(logFields, zap.Duration("took", nowFn().Sub(begin))) - if currErr != nil { - logFields = append(logFields, zap.Error(currErr)) - b.log.Info("bootstrapping from source completed with error", logFields...) - } else { - logFields = append(logFields, currStatus.logFields...) - b.log.Info("bootstrapping from source completed successfully", logFields...) + if err != nil { + errorLogFields := append(logFieldsCopy(logFields), zap.Error(err)) + b.log.Error("error bootstrapping from source", errorLogFields...) + return bootstrap.NamespaceResults{}, err } - wg.Wait() - if err := xerrors.FirstError(currErr, nextErr); err != nil { - return err + b.log.Info("bootstrap from source completed", logFields...) + // Determine the unfulfilled and the unattempted ranges to execute next. + next, err := b.logSuccessAndDetermineCurrResultsUnfulfilledAndNextBootstrapRanges(namespaces, + curr, currResults, logFields) + if err != nil { + return bootstrap.NamespaceResults{}, err } - fulfilledRanges := result.ShardTimeRanges{} - fulfilledRanges.AddRanges(currStatus.fulfilled) - fulfilledRanges.AddRanges(nextStatus.fulfilled) - unfulfilled := totalRanges.Copy() - unfulfilled.Subtract(fulfilledRanges) + // Unless next bootstrapper is required, this is the final results. + finalResults := currResults - step.mergeResults(unfulfilled) + // If there are some time ranges the current bootstrapper could not fulfill, + // that we can attempt then pass it along to the next bootstrapper. + if next.Namespaces.Len() > 0 { + nextResults, err := b.next.Bootstrap(next) + if err != nil { + return bootstrap.NamespaceResults{}, err + } - unattemptedNextRanges := currRanges.Copy() - unattemptedNextRanges.Subtract(currStatus.fulfilled) - if !nextAttempted { - // If we have never attempted the next time range then we want to also - // attempt the ranges we didn't even try to attempt - unattemptedNextRanges.AddRanges(nextRanges) + // Now merge the final results. + for _, elem := range nextResults.Results.Iter() { + id := elem.Key() + currNamespace := elem.Value() + + finalResult, ok := finalResults.Results.Get(id) + if !ok { + return bootstrap.NamespaceResults{}, + fmt.Errorf("expected result for namespace: %s", id.String()) + } + + // NB(r): Since we originally passed all unfulfilled ranges to the + // next bootstrapper, the final unfulfilled is simply what it could + // not fulfill. + finalResult.DataResult.SetUnfulfilled(currNamespace.DataResult.Unfulfilled().Copy()) + if currNamespace.Metadata.Options().IndexOptions().Enabled() { + finalResult.IndexResult.SetUnfulfilled(currNamespace.IndexResult.Unfulfilled().Copy()) + } + + // Map is by value, set the result altered struct. + finalResults.Results.Set(id, finalResult) + } } - // If there are some time ranges the current bootstrapper could not fulfill, - // that we can attempt then pass it along to the next bootstrapper. - // NB(r): We explicitly do not ask the next bootstrapper to retry ranges - // it could not fulfill as it's unlikely to be able to now. - if !unattemptedNextRanges.IsEmpty() { - nextStatus, nextErr = step.runNextStep(unattemptedNextRanges) - if nextErr != nil { - return nextErr + return finalResults, nil +} + +func (b baseBootstrapper) logSuccessAndDetermineCurrResultsUnfulfilledAndNextBootstrapRanges( + requested bootstrap.Namespaces, + curr bootstrap.Namespaces, + currResults bootstrap.NamespaceResults, + baseLogFields []zapcore.Field, +) (bootstrap.Namespaces, error) { + next := bootstrap.Namespaces{ + Namespaces: bootstrap.NewNamespacesMap(bootstrap.NamespacesMapOptions{}), + } + for _, elem := range requested.Namespaces.Iter() { + id := elem.Key() + requestedNamespace := elem.Value() + + currResult, ok := currResults.Results.Get(id) + if !ok { + return bootstrap.Namespaces{}, + fmt.Errorf("namespace result not returned by bootstrapper: %v", id.String()) + } + + currNamespace, ok := curr.Namespaces.Get(id) + if !ok { + return bootstrap.Namespaces{}, + fmt.Errorf("namespace prepared request not found: %v", id.String()) + } + + // Shallow copy the current namespace for the next namespace prepared request. + nextNamespace := currNamespace + + // Calculate bootstrap time ranges. + dataRequired := requestedNamespace.DataRunOptions.ShardTimeRanges.Copy() + dataCurrRequested := currNamespace.DataRunOptions.ShardTimeRanges.Copy() + dataCurrFulfilled := dataCurrRequested.Copy() + dataCurrFulfilled.Subtract(currResult.DataResult.Unfulfilled()) + + dataUnfulfilled := dataRequired.Copy() + dataUnfulfilled.Subtract(dataCurrFulfilled) + + // Modify the unfulfilled result. + currResult.DataResult.SetUnfulfilled(dataUnfulfilled.Copy()) + + // Set the next bootstrapper required ranges. + nextNamespace.DataRunOptions.ShardTimeRanges = dataUnfulfilled.Copy() + + var ( + indexCurrRequested = result.ShardTimeRanges{} + indexCurrFulfilled = result.ShardTimeRanges{} + indexUnfulfilled = result.ShardTimeRanges{} + ) + if currNamespace.Metadata.Options().IndexOptions().Enabled() { + // Calculate bootstrap time ranges. + indexRequired := requestedNamespace.IndexRunOptions.ShardTimeRanges.Copy() + indexCurrRequested = currNamespace.IndexRunOptions.ShardTimeRanges.Copy() + indexCurrFulfilled = indexCurrRequested.Copy() + indexCurrFulfilled.Subtract(currResult.IndexResult.Unfulfilled()) + + indexUnfulfilled = indexRequired.Copy() + indexUnfulfilled.Subtract(indexCurrFulfilled) + + // Modify the unfulfilled result. + currResult.IndexResult.SetUnfulfilled(indexUnfulfilled.Copy()) + } + + // Set the next bootstrapper required ranges. + // NB(r): Make sure to always set an empty requested range so IsEmpty + // does not cause nil ptr deref. + nextNamespace.IndexRunOptions.ShardTimeRanges = indexUnfulfilled.Copy() + + // Set the modified result. + currResults.Results.Set(id, currResult) + + // Set the next bootstrapper namespace run options if we need to bootstrap + // further time ranges. + if !nextNamespace.DataRunOptions.ShardTimeRanges.IsEmpty() || + !nextNamespace.IndexRunOptions.ShardTimeRanges.IsEmpty() { + next.Namespaces.Set(id, nextNamespace) + } + + // Log the result. + _, _, dataRangeRequested := dataCurrRequested.MinMaxRange() + _, _, dataRangeFulfilled := dataCurrFulfilled.MinMaxRange() + successLogFields := append(logFieldsCopy(baseLogFields), []zapcore.Field{ + zap.String("namespace", id.String()), + zap.Int("numShards", len(currNamespace.Shards)), + zap.Duration("dataRangeRequested", dataRangeRequested), + zap.Duration("dataRangeFulfilled", dataRangeFulfilled), + }...) + + if currNamespace.Metadata.Options().IndexOptions().Enabled() { + _, _, indexRangeRequested := indexCurrRequested.MinMaxRange() + _, _, indexRangeFulfilled := indexCurrFulfilled.MinMaxRange() + successLogFields = append(successLogFields, []zapcore.Field{ + zap.Duration("indexRangeRequested", indexRangeRequested), + zap.Duration("indexRangeFulfilled", indexRangeFulfilled), + zap.Int("numIndexBlocks", len(currResult.IndexResult.IndexResults())), + }...) } - unfulfilledFinal := unfulfilled.Copy() - unfulfilledFinal.Subtract(nextStatus.fulfilled) - step.mergeResults(unfulfilledFinal) + b.log.Info("bootstrapping from source completed successfully", + successLogFields...) } - return nil + return next, nil +} + +func (b baseBootstrapper) logShardTimeRanges( + msg string, + baseLogFields []zapcore.Field, + currNamespace bootstrap.Namespace, +) { + dataShardTimeRanges := currNamespace.DataRunOptions.ShardTimeRanges + dataMin, dataMax, dataRange := dataShardTimeRanges.MinMaxRange() + logFields := append(logFieldsCopy(baseLogFields), []zapcore.Field{ + zap.Stringer("namespace", currNamespace.Metadata.ID()), + zap.Int("numShards", len(currNamespace.Shards)), + zap.Duration("dataRange", dataRange), + }...) + if dataRange > 0 { + logFields = append(logFields, []zapcore.Field{ + zap.Time("dataFrom", dataMin), + zap.Time("dataTo", dataMax), + }...) + } + if currNamespace.Metadata.Options().IndexOptions().Enabled() { + indexShardTimeRanges := currNamespace.IndexRunOptions.ShardTimeRanges + indexMin, indexMax, indexRange := indexShardTimeRanges.MinMaxRange() + logFields = append(logFields, []zapcore.Field{ + zap.Duration("indexRange", indexRange), + }...) + if indexRange > 0 { + logFields = append(logFields, []zapcore.Field{ + zap.Time("indexFrom", indexMin), + zap.Time("indexTo", indexMax), + }...) + } + } + + b.log.Info(msg, logFields...) +} + +func logFieldsCopy(logFields []zapcore.Field) []zapcore.Field { + return append(make([]zapcore.Field, 0, 2*len(logFields)), logFields...) } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/base_data_step.go b/src/dbnode/storage/bootstrap/bootstrapper/base_data_step.go deleted file mode 100644 index 191e942ca5..0000000000 --- a/src/dbnode/storage/bootstrap/bootstrapper/base_data_step.go +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package bootstrapper - -import ( - "github.com/m3db/m3/src/dbnode/storage/bootstrap" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" - - "go.uber.org/zap" - "go.uber.org/zap/zapcore" -) - -type bootstrapDataStep interface { - bootstrapStep - result() result.DataBootstrapResult -} - -type bootstrapData struct { - namespace namespace.Metadata - curr bootstrap.Source - next bootstrap.Bootstrapper - opts bootstrap.RunOptions - currResult result.DataBootstrapResult - nextResult result.DataBootstrapResult - mergedResult result.DataBootstrapResult -} - -func newBootstrapDataStep( - namespace namespace.Metadata, - curr bootstrap.Source, - next bootstrap.Bootstrapper, - opts bootstrap.RunOptions, -) bootstrapDataStep { - return &bootstrapData{ - namespace: namespace, - curr: curr, - next: next, - opts: opts, - } -} - -func (s *bootstrapData) prepare( - totalRanges result.ShardTimeRanges, -) (bootstrapStepPreparedResult, error) { - currAvailable, err := s.curr.AvailableData(s.namespace, totalRanges, s.opts) - if err != nil { - return bootstrapStepPreparedResult{}, err - } - - return bootstrapStepPreparedResult{ - currAvailable: currAvailable, - }, nil -} - -func (s *bootstrapData) runCurrStep( - targetRanges result.ShardTimeRanges, -) (bootstrapStepStatus, error) { - var ( - requested = targetRanges.Copy() - fulfilled result.ShardTimeRanges - logFields []zapcore.Field - err error - ) - s.currResult, err = s.curr.ReadData(s.namespace, targetRanges, s.opts) - if result := s.currResult; result != nil { - fulfilled = requested - fulfilled.Subtract(result.Unfulfilled()) - - logFields = append(logFields, - zap.Int64("numSeries", result.ShardResults().NumSeries())) - } - return bootstrapStepStatus{ - fulfilled: fulfilled, - logFields: logFields, - }, err -} - -func (s *bootstrapData) runNextStep( - targetRanges result.ShardTimeRanges, -) (bootstrapStepStatus, error) { - var ( - requested = targetRanges.Copy() - fulfilled result.ShardTimeRanges - err error - ) - s.nextResult, err = s.next.BootstrapData(s.namespace, targetRanges, s.opts) - if result := s.nextResult; result != nil { - fulfilled = requested - fulfilled.Subtract(result.Unfulfilled()) - } - return bootstrapStepStatus{ - fulfilled: fulfilled, - }, err -} - -func (s *bootstrapData) mergeResults( - totalUnfulfilled result.ShardTimeRanges, -) { - if s.mergedResult == nil { - s.mergedResult = result.NewDataBootstrapResult() - } - if s.currResult != nil { - // Merge the curr results in - s.mergedResult.ShardResults().AddResults(s.currResult.ShardResults()) - s.currResult = nil - } - if s.nextResult != nil { - // Merge the next results in - s.mergedResult.ShardResults().AddResults(s.nextResult.ShardResults()) - s.nextResult = nil - } - s.mergedResult.SetUnfulfilled(totalUnfulfilled) -} - -func (s *bootstrapData) result() result.DataBootstrapResult { - return s.mergedResult -} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/base_index_step.go b/src/dbnode/storage/bootstrap/bootstrapper/base_index_step.go deleted file mode 100644 index 5a07a425b5..0000000000 --- a/src/dbnode/storage/bootstrap/bootstrapper/base_index_step.go +++ /dev/null @@ -1,141 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package bootstrapper - -import ( - "github.com/m3db/m3/src/dbnode/storage/bootstrap" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" - - "go.uber.org/zap" - "go.uber.org/zap/zapcore" -) - -type bootstrapIndexStep interface { - bootstrapStep - result() result.IndexBootstrapResult -} - -type bootstrapIndex struct { - namespace namespace.Metadata - curr bootstrap.Source - next bootstrap.Bootstrapper - opts bootstrap.RunOptions - currResult result.IndexBootstrapResult - nextResult result.IndexBootstrapResult - mergedResult result.IndexBootstrapResult -} - -func newBootstrapIndexStep( - namespace namespace.Metadata, - curr bootstrap.Source, - next bootstrap.Bootstrapper, - opts bootstrap.RunOptions, -) bootstrapIndexStep { - return &bootstrapIndex{ - namespace: namespace, - curr: curr, - next: next, - opts: opts, - } -} - -func (s *bootstrapIndex) prepare( - totalRanges result.ShardTimeRanges, -) (bootstrapStepPreparedResult, error) { - currAvailable, err := s.curr.AvailableIndex(s.namespace, totalRanges, s.opts) - if err != nil { - return bootstrapStepPreparedResult{}, err - } - - return bootstrapStepPreparedResult{ - currAvailable: currAvailable, - }, nil -} - -func (s *bootstrapIndex) runCurrStep( - targetRanges result.ShardTimeRanges, -) (bootstrapStepStatus, error) { - var ( - requested = targetRanges.Copy() - fulfilled result.ShardTimeRanges - logFields []zapcore.Field - err error - ) - s.currResult, err = s.curr.ReadIndex(s.namespace, targetRanges, s.opts) - if result := s.currResult; result != nil { - fulfilled = requested - fulfilled.Subtract(result.Unfulfilled()) - - blocks := 0 - segments := 0 - for _, block := range result.IndexResults() { - blocks++ - segments += len(block.Segments()) - } - logFields = append(logFields, zap.Int("numBlocks", blocks), zap.Int("numSegments", segments)) - } - return bootstrapStepStatus{ - fulfilled: fulfilled, - logFields: logFields, - }, err -} - -func (s *bootstrapIndex) runNextStep( - targetRanges result.ShardTimeRanges, -) (bootstrapStepStatus, error) { - var ( - requested = targetRanges.Copy() - fulfilled result.ShardTimeRanges - err error - ) - s.nextResult, err = s.next.BootstrapIndex(s.namespace, targetRanges, s.opts) - if result := s.nextResult; result != nil { - fulfilled = requested - fulfilled.Subtract(result.Unfulfilled()) - } - return bootstrapStepStatus{ - fulfilled: fulfilled, - }, err -} - -func (s *bootstrapIndex) mergeResults( - totalUnfulfilled result.ShardTimeRanges, -) { - if s.mergedResult == nil { - s.mergedResult = result.NewIndexBootstrapResult() - } - if s.currResult != nil { - // Merge the curr results in - s.mergedResult.IndexResults().AddResults(s.currResult.IndexResults()) - s.currResult = nil - } - if s.nextResult != nil { - // Merge the next results in - s.mergedResult.IndexResults().AddResults(s.nextResult.IndexResults()) - s.nextResult = nil - } - s.mergedResult.SetUnfulfilled(totalUnfulfilled) -} - -func (s *bootstrapIndex) result() result.IndexBootstrapResult { - return s.mergedResult -} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/base_test.go b/src/dbnode/storage/bootstrap/bootstrapper/base_test.go index 7cece86d78..71eb94f8e2 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/base_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/base_test.go @@ -21,17 +21,12 @@ package bootstrapper import ( - "fmt" "testing" "time" - "github.com/m3db/m3/src/dbnode/storage/block" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/ts" - "github.com/m3db/m3/src/m3ninx/index/segment" - "github.com/m3db/m3/src/m3ninx/index/segment/mem" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -48,24 +43,18 @@ var ( SetPersistConfig(bootstrap.PersistConfig{Enabled: false}) ) -func testNsMetadata(t *testing.T) namespace.Metadata { - md, err := namespace.NewMetadata(testNamespaceID, namespace.NewOptions()) +func testNsMetadata(t *testing.T, withIndex bool) namespace.Metadata { + opts := namespace.NewOptions() + opts = opts.SetIndexOptions(opts.IndexOptions().SetEnabled(withIndex)) + md, err := namespace.NewMetadata(testNamespaceID, opts) require.NoError(t, err) return md } -type testBlockEntry struct { - id string - tags []string - t time.Time -} - -type testShardResult struct { - result result.ShardResult - unfulfilled xtime.Ranges -} - -func testBaseBootstrapper(t *testing.T, ctrl *gomock.Controller) (*bootstrap.MockSource, *bootstrap.MockBootstrapper, baseBootstrapper) { +func testBaseBootstrapper( + t *testing.T, + ctrl *gomock.Controller, +) (*bootstrap.MockSource, *bootstrap.MockBootstrapper, baseBootstrapper) { source := bootstrap.NewMockSource(ctrl) opts := result.NewOptions() next := bootstrap.NewMockBootstrapper(ctrl) @@ -83,426 +72,244 @@ func testShardTimeRanges() result.ShardTimeRanges { return map[uint32]xtime.Ranges{testShard: testTargetRanges()} } -func shardResult(entries ...testBlockEntry) result.ShardResult { - opts := result.NewOptions() - res := result.NewShardResult(0, opts) - for _, entry := range entries { - block := opts.DatabaseBlockOptions().DatabaseBlockPool().Get() - - block.Reset(entry.t, time.Hour, ts.Segment{}, namespace.Context{}) - - if len(entry.tags)%2 != 0 { - panic(fmt.Sprintf("entry tags must be of even length: %v", entry.tags)) - } - tags := ident.NewTags() - for idx := 0; idx < len(entry.tags); idx += 2 { - tags.Append(ident.StringTag(entry.tags[idx], entry.tags[idx+1])) - } - res.AddBlock(ident.StringID(entry.id), tags, block) +func testResult( + ns namespace.Metadata, + withIndex bool, + shard uint32, + unfulfilledRange xtime.Ranges, +) bootstrap.NamespaceResults { + unfulfilled := result.ShardTimeRanges{ + shard: unfulfilledRange, } - return res -} -func testResult(results map[uint32]testShardResult) result.DataBootstrapResult { - result := result.NewDataBootstrapResult() - for shard, entry := range results { - result.Add(shard, entry.result, entry.unfulfilled) - } - return result -} + opts := bootstrap.NamespaceResultsMapOptions{} + results := bootstrap.NewNamespaceResultsMap(opts) + dataResult := result.NewDataBootstrapResult() + dataResult.SetUnfulfilled(unfulfilled.Copy()) -func validateBlock(t *testing.T, expectedBlock, actualBlock block.DatabaseBlock) { - if expectedBlock == nil { - require.Nil(t, actualBlock) - return + indexResult := result.NewIndexBootstrapResult() + if withIndex { + indexResult.SetUnfulfilled(unfulfilled.Copy()) } - require.Equal(t, expectedBlock.StartTime(), actualBlock.StartTime()) -} -func validateSeries(t *testing.T, expectedSeries, actualSeries block.DatabaseSeriesBlocks) { - if expectedSeries == nil { - require.Nil(t, actualSeries) - return - } - eb := expectedSeries.AllBlocks() - ab := actualSeries.AllBlocks() - require.Equal(t, len(eb), len(ab)) - for id, expectedBlock := range eb { - actualBlock, exists := ab[id] - require.True(t, exists) - validateBlock(t, expectedBlock, actualBlock) - } -} - -func validateResult(t *testing.T, expected, actual result.DataBootstrapResult) { - if expected == nil { - require.Nil(t, actual) - return - } + results.Set(ns.ID(), bootstrap.NamespaceResult{ + Metadata: ns, + Shards: []uint32{shard}, + DataResult: dataResult, + IndexResult: indexResult, + }) - expectedShardResults := expected.ShardResults() - actualShardResults := actual.ShardResults() - - require.Equal(t, len(expectedShardResults), len(actualShardResults)) - - for shard, result := range expected.ShardResults() { - _, ok := actualShardResults[shard] - require.True(t, ok) - es := result.AllSeries() - as := actualShardResults[shard].AllSeries() - require.Equal(t, es.Len(), as.Len()) - for _, entry := range es.Iter() { - id, expectedSeries := entry.Key(), entry.Value() - actualSeries, exists := as.Get(id) - require.True(t, exists) - validateSeries(t, expectedSeries.Blocks, actualSeries.Blocks) - } - } + return bootstrap.NamespaceResults{Results: results} +} - expectedUnfulfilled := expected.Unfulfilled() - actualUnfulfilled := actual.Unfulfilled() +func testEmptyResult( + ns namespace.Metadata, +) bootstrap.NamespaceResults { + opts := bootstrap.NamespaceResultsMapOptions{} + results := bootstrap.NewNamespaceResultsMap(opts) + results.Set(ns.ID(), bootstrap.NamespaceResult{ + Metadata: ns, + DataResult: result.NewDataBootstrapResult(), + IndexResult: result.NewIndexBootstrapResult(), + }) - require.Equal(t, len(expectedUnfulfilled), len(actualUnfulfilled)) + return bootstrap.NamespaceResults{Results: results} +} - for shard, ranges := range expectedUnfulfilled { - _, ok := actualUnfulfilled[shard] - require.True(t, ok) - validateRanges(t, ranges, actualUnfulfilled[shard]) - } +func TestBaseBootstrapperEmptyRange(t *testing.T) { + testBaseBootstrapperEmptyRange(t, false) } -func validateRanges(t *testing.T, expected, actual xtime.Ranges) { - require.Equal(t, expected.Len(), actual.Len()) - eit := expected.Iter() - ait := actual.Iter() - for eit.Next() { - require.True(t, ait.Next()) - require.Equal(t, eit.Value(), ait.Value()) - } - require.False(t, ait.Next()) +func TestBaseBootstrapperEmptyRangeWithIndex(t *testing.T) { + testBaseBootstrapperEmptyRange(t, true) } -func equalRanges(expected, actual xtime.Ranges) bool { - if expected.Len() != actual.Len() { - return false - } - eit := expected.Iter() - ait := actual.Iter() - read := 0 - mustRead := expected.Len() - for eit.Next() && ait.Next() { - if eit.Value() != ait.Value() { - return false - } +func testBaseBootstrapperEmptyRange(t *testing.T, withIndex bool) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + src, _, base := testBaseBootstrapper(t, ctrl) + testNs := testNsMetadata(t, withIndex) + + rngs := result.ShardTimeRanges{} + unfulfilled := xtime.NewRanges() + nsResults := testResult(testNs, withIndex, testShard, unfulfilled) + shardRangeMatcher := bootstrap.ShardTimeRangesMatcher{Ranges: rngs} + src.EXPECT().AvailableData(testNs, shardRangeMatcher, testDefaultRunOpts). + Return(rngs, nil) + if withIndex { + src.EXPECT().AvailableIndex(testNs, shardRangeMatcher, testDefaultRunOpts). + Return(rngs, nil) } - return read == mustRead -} -type shardTimeRangesMatcher struct { - expected map[uint32]xtime.Ranges -} + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, rngs, testNs) + defer tester.Finish() -func (m shardTimeRangesMatcher) Matches(x interface{}) bool { - actual, ok := x.(result.ShardTimeRanges) - if !ok { - return false - } + matcher := bootstrap.NamespaceMatcher{Namespaces: tester.Namespaces} + src.EXPECT().Read(matcher).DoAndReturn( + func(namespaces bootstrap.Namespaces) (bootstrap.NamespaceResults, error) { + return nsResults, nil + }) - for shard, ranges := range m.expected { - actualRanges, ok := actual[shard] - if !ok { - return false - } - if equalRanges(ranges, actualRanges) { - return false - } - } + // Test non-nil empty range + tester.TestBootstrapWith(base) + tester.TestUnfulfilledForNamespaceIsEmpty(testNs) + assert.Equal(t, nsResults, tester.Results) - return true + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } -func (m shardTimeRangesMatcher) String() string { - return "shardTimeRangesMatcher" +func TestBaseBootstrapperCurrentNoUnfulfilled(t *testing.T) { + testBaseBootstrapperCurrentNoUnfulfilled(t, false) } -func TestBaseBootstrapperEmptyRange(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - _, _, base := testBaseBootstrapper(t, ctrl) - testNs := testNsMetadata(t) - - // Test non-nil empty range - res, err := base.BootstrapData(testNs, map[uint32]xtime.Ranges{}, testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, 0, len(res.ShardResults())) - require.True(t, res.Unfulfilled().IsEmpty()) - - res, err = base.BootstrapData(testNs, nil, testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, 0, len(res.ShardResults())) - require.True(t, res.Unfulfilled().IsEmpty()) +func TestBaseBootstrapperCurrentNoUnfulfilledWithIndex(t *testing.T) { + testBaseBootstrapperCurrentNoUnfulfilled(t, true) } -func TestBaseBootstrapperCurrentNoUnfulfilled(t *testing.T) { +func testBaseBootstrapperCurrentNoUnfulfilled(t *testing.T, withIndex bool) { ctrl := gomock.NewController(t) defer ctrl.Finish() - source, _, base := testBaseBootstrapper(t, ctrl) - testNs := testNsMetadata(t) + src, _, base := testBaseBootstrapper(t, ctrl) + testNs := testNsMetadata(t, withIndex) - targetRanges := testShardTimeRanges() - result := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(testBlockEntry{"foo", nil, testTargetStart})}, - }) + unfulfilled := xtime.NewRanges() + nsResults := testResult(testNs, withIndex, testShard, unfulfilled) - source.EXPECT(). - AvailableData(testNs, targetRanges, testDefaultRunOpts). + targetRanges := testShardTimeRanges() + src.EXPECT().AvailableData(testNs, targetRanges, testDefaultRunOpts). Return(targetRanges, nil) - source.EXPECT(). - ReadData(testNs, targetRanges, testDefaultRunOpts). - Return(result, nil) + if withIndex { + src.EXPECT().AvailableIndex(testNs, targetRanges, testDefaultRunOpts). + Return(targetRanges, nil) + } - res, err := base.BootstrapData(testNs, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - validateResult(t, result, res) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, + testNs) + defer tester.Finish() + + matcher := bootstrap.NamespaceMatcher{Namespaces: tester.Namespaces} + src.EXPECT().Read(matcher).DoAndReturn( + func(namespaces bootstrap.Namespaces) (bootstrap.NamespaceResults, error) { + return nsResults, nil + }) + + tester.TestBootstrapWith(base) + assert.Equal(t, nsResults, tester.Results) + tester.TestUnfulfilledForNamespaceIsEmpty(testNs) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func TestBaseBootstrapperCurrentSomeUnfulfilled(t *testing.T) { + testBaseBootstrapperCurrentSomeUnfulfilled(t, false) +} + +func TestBaseBootstrapperCurrentSomeUnfulfilledWithIndex(t *testing.T) { + testBaseBootstrapperCurrentSomeUnfulfilled(t, true) +} + +func testBaseBootstrapperCurrentSomeUnfulfilled(t *testing.T, withIndex bool) { ctrl := gomock.NewController(t) defer ctrl.Finish() - source, next, base := testBaseBootstrapper(t, ctrl) - - testNs := testNsMetadata(t) - entries := []testBlockEntry{ - {"foo", []string{"foo", "foe"}, testTargetStart}, - {"foo", []string{"foo", "foe"}, testTargetStart.Add(time.Hour)}, - {"bar", []string{"bar", "baz"}, testTargetStart.Add(time.Hour)}, - } + src, next, base := testBaseBootstrapper(t, ctrl) + testNs := testNsMetadata(t, withIndex) targetRanges := testShardTimeRanges() currUnfulfilled := xtime.NewRanges(xtime.Range{ Start: testTargetStart.Add(time.Hour), End: testTargetStart.Add(time.Hour * 2), }) - currResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries[0]), unfulfilled: currUnfulfilled}, - }) - nextTargetRanges := map[uint32]xtime.Ranges{ - testShard: currUnfulfilled, - } - nextResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries[1:]...)}, - }) - source.EXPECT(). - AvailableData(testNs, targetRanges, testDefaultRunOpts). + src.EXPECT().AvailableData(testNs, targetRanges, testDefaultRunOpts). Return(targetRanges, nil) - source.EXPECT(). - ReadData(testNs, targetRanges, testDefaultRunOpts). - Return(currResult, nil) - next.EXPECT(). - BootstrapData(testNs, shardTimeRangesMatcher{nextTargetRanges}, - testDefaultRunOpts). - Return(nextResult, nil) - - expectedResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries...)}, - }) + if withIndex { + src.EXPECT().AvailableIndex(testNs, targetRanges, testDefaultRunOpts). + Return(targetRanges, nil) + } - res, err := base.BootstrapData(testNs, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - validateResult(t, expectedResult, res) + currResult := testResult(testNs, withIndex, testShard, currUnfulfilled) + nextResult := testResult(testNs, withIndex, testShard, xtime.NewRanges()) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, + testNs) + defer tester.Finish() + + matcher := bootstrap.NamespaceMatcher{Namespaces: tester.Namespaces} + src.EXPECT().Read(matcher).Return(currResult, nil) + next.EXPECT().Bootstrap(matcher).Return(nextResult, nil) + + tester.TestBootstrapWith(base) + tester.TestUnfulfilledForNamespaceIsEmpty(testNs) } -func testBasebootstrapperNext(t *testing.T, nextUnfulfilled result.ShardTimeRanges) { +func testBasebootstrapperNext( + t *testing.T, + nextUnfulfilled xtime.Ranges, + withIndex bool, +) { ctrl := gomock.NewController(t) defer ctrl.Finish() - source, next, base := testBaseBootstrapper(t, ctrl) - testNs := testNsMetadata(t) - - source.EXPECT().Can(bootstrap.BootstrapParallel).Return(true) - next.EXPECT().Can(bootstrap.BootstrapParallel).Return(true) - + src, next, base := testBaseBootstrapper(t, ctrl) + testNs := testNsMetadata(t, withIndex) targetRanges := testShardTimeRanges() - nextResult := testResult(map[uint32]testShardResult{ - testShard: { - result: shardResult(testBlockEntry{"foo", []string{"foo", "foe"}, testTargetStart}), - }, - }) - nextResult.SetUnfulfilled(nextUnfulfilled) - source.EXPECT(). + src.EXPECT(). AvailableData(testNs, targetRanges, testDefaultRunOpts). Return(nil, nil) - source.EXPECT(). - ReadData(testNs, shardTimeRangesMatcher{nil}, - testDefaultRunOpts). - Return(nil, nil) - next.EXPECT(). - BootstrapData(testNs, shardTimeRangesMatcher{targetRanges}, - testDefaultRunOpts). - Return(nextResult, nil) + if withIndex { + src.EXPECT(). + AvailableIndex(testNs, targetRanges, testDefaultRunOpts). + Return(nil, nil) + } - res, err := base.BootstrapData(testNs, targetRanges, - testDefaultRunOpts) - require.NoError(t, err) - validateResult(t, nextResult, res) -} + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, + testNs) + defer tester.Finish() -func TestBaseBootstrapperNextNoUnfulfilled(t *testing.T) { - nextUnfulfilled := testShardTimeRanges() - testBasebootstrapperNext(t, nextUnfulfilled) -} + emptyResult := testEmptyResult(testNs) + nextResult := testResult(testNs, withIndex, testShard, nextUnfulfilled) + matcher := bootstrap.NamespaceMatcher{Namespaces: tester.Namespaces} + src.EXPECT().Read(matcher).Return(emptyResult, nil) + next.EXPECT().Bootstrap(matcher).Return(nextResult, nil) -func TestBaseBootstrapperNextSomeUnfulfilled(t *testing.T) { - nextUnfulfilled := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(xtime.Range{ - Start: testTargetStart, - End: testTargetStart.Add(time.Hour), - }), - } - testBasebootstrapperNext(t, nextUnfulfilled) -} + tester.TestBootstrapWith(base) -func TestBaseBootstrapperBoth(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - source, next, base := testBaseBootstrapper(t, ctrl) - - testNs := testNsMetadata(t) - entries := []testBlockEntry{ - {"foo", []string{"foo", "foe"}, testTargetStart}, - {"foo", []string{"foo", "foe"}, testTargetStart.Add(time.Hour)}, - {"bar", []string{"bar", "bah"}, testTargetStart.Add(time.Hour)}, - {"baz", []string{"baz", "zab"}, testTargetStart}, - } + ex, ok := nextResult.Results.Get(testNs.ID()) + require.True(t, ok) - ranges := []xtime.Range{ - xtime.Range{Start: testTargetStart, End: testTargetStart.Add(time.Hour)}, - xtime.Range{Start: testTargetStart.Add(time.Hour), End: testTargetStart.Add(2 * time.Hour)}, - xtime.Range{Start: testTargetStart, End: testTargetStart.Add(30 * time.Minute)}, - xtime.Range{Start: testTargetStart.Add(90 * time.Minute), End: testTargetStart.Add(100 * time.Minute)}, - xtime.Range{Start: testTargetStart.Add(10 * time.Minute), End: testTargetStart.Add(20 * time.Minute)}, + expected := ex.DataResult.Unfulfilled() + expectedIdx := ex.IndexResult.Unfulfilled() + if !withIndex { + expectedIdx = result.ShardTimeRanges{} } - targetRanges := testShardTimeRanges() - availableRanges := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(ranges[0]), - } - remainingRanges := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(ranges[1]), - } + tester.TestUnfulfilledForNamespace(testNs, expected, expectedIdx) +} - currUnfulfilled := xtime.NewRanges(ranges[2]) - currResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries[0]), unfulfilled: currUnfulfilled}, - }) +func TestBaseBootstrapperNextNoUnfulfilled(t *testing.T) { + nextUnfulfilled := testTargetRanges() + testBasebootstrapperNext(t, nextUnfulfilled, false) +} - nextUnfulfilled := xtime.NewRanges(ranges[3]) - nextResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries[1:3]...), unfulfilled: nextUnfulfilled}, - }) +func TestBaseBootstrapperNextNoUnfulfilledWithIndex(t *testing.T) { + nextUnfulfilled := testTargetRanges() + testBasebootstrapperNext(t, nextUnfulfilled, true) +} - fallBackUnfulfilled := xtime.NewRanges(ranges[4]) - fallBackResult := testResult(map[uint32]testShardResult{ - testShard: {result: shardResult(entries[3]), unfulfilled: fallBackUnfulfilled}, +func TestBaseBootstrapperNextSomeUnfulfilled(t *testing.T) { + nextUnfulfilled := xtime.NewRanges(xtime.Range{ + Start: testTargetStart, + End: testTargetStart.Add(time.Hour), }) - source.EXPECT().Can(bootstrap.BootstrapParallel).Return(true) - source.EXPECT(). - AvailableData(testNs, targetRanges, testDefaultRunOpts). - Return(availableRanges, nil) - source.EXPECT(). - ReadData(testNs, shardTimeRangesMatcher{availableRanges}, - testDefaultRunOpts). - Return(currResult, nil) - next.EXPECT().Can(bootstrap.BootstrapParallel).Return(true) - next.EXPECT(). - BootstrapData(testNs, shardTimeRangesMatcher{remainingRanges}, - testDefaultRunOpts). - Return(nextResult, nil) - next.EXPECT(). - BootstrapData(testNs, shardTimeRangesMatcher{currResult.Unfulfilled()}, - testDefaultRunOpts). - Return(fallBackResult, nil) - - res, err := base.BootstrapData(testNs, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - - expectedResult := testResult(map[uint32]testShardResult{ - testShard: { - result: shardResult(entries...), - unfulfilled: xtime.NewRanges(ranges[3]).AddRange(ranges[4]), - }, - }) - validateResult(t, expectedResult, res) + testBasebootstrapperNext(t, nextUnfulfilled, false) } -func TestBaseBootstrapperIndexHalfCurrentHalfNext(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - source, next, base := testBaseBootstrapper(t, ctrl) - testNs := testNsMetadata(t) - - targetRanges := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(xtime.Range{ - Start: testTargetStart, - End: testTargetStart.Add(2 * time.Hour), - }), - } - firstHalf := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(xtime.Range{ - Start: testTargetStart, - End: testTargetStart.Add(1 * time.Hour), - }), - } - secondHalf := map[uint32]xtime.Ranges{ - testShard: xtime.NewRanges(xtime.Range{ - Start: testTargetStart.Add(1 * time.Hour), - End: testTargetStart.Add(2 * time.Hour), - }), - } - - segFirst, err := mem.NewSegment(0, mem.NewOptions()) - require.NoError(t, err) - - segSecond, err := mem.NewSegment(0, mem.NewOptions()) - require.NoError(t, err) - - currResult := result.NewIndexBootstrapResult() - currResult.Add(result.NewIndexBlock(testTargetStart, - []segment.Segment{segFirst}, firstHalf), nil) - nextResult := result.NewIndexBootstrapResult() - nextResult.Add(result.NewIndexBlock(testTargetStart.Add(1*time.Hour), - []segment.Segment{segSecond}, secondHalf), nil) - - source.EXPECT().Can(bootstrap.BootstrapParallel).Return(false) - source.EXPECT(). - AvailableIndex(testNs, shardTimeRangesMatcher{targetRanges}, testDefaultRunOpts). - Return(firstHalf, nil) - source.EXPECT(). - ReadIndex(testNs, shardTimeRangesMatcher{firstHalf}, testDefaultRunOpts). - Return(currResult, nil) - - next.EXPECT(). - BootstrapIndex(testNs, shardTimeRangesMatcher{secondHalf}, testDefaultRunOpts). - Return(nextResult, nil) - - res, err := base.BootstrapIndex(testNs, targetRanges, testDefaultRunOpts) - require.NoError(t, err) +func TestBaseBootstrapperNextSomeUnfulfilledWithIndex(t *testing.T) { + nextUnfulfilled := xtime.NewRanges(xtime.Range{ + Start: testTargetStart, + End: testTargetStart.Add(time.Hour), + }) - assert.True(t, res.Unfulfilled().IsEmpty()) - assert.Equal(t, 2, len(res.IndexResults())) - - first, ok := res.IndexResults()[xtime.ToUnixNano(testTargetStart)] - assert.True(t, ok) - assert.True(t, first.BlockStart().Equal(testTargetStart)) - require.Equal(t, 1, len(first.Segments())) - assert.True(t, segFirst == first.Segments()[0]) - assert.Equal(t, firstHalf, map[uint32]xtime.Ranges(first.Fulfilled())) - - second, ok := res.IndexResults()[xtime.ToUnixNano(testTargetStart.Add(time.Hour))] - assert.True(t, ok) - assert.True(t, second.BlockStart().Equal(testTargetStart.Add(time.Hour))) - require.Equal(t, 1, len(second.Segments())) - assert.True(t, segSecond == second.Segments()[0]) - assert.Equal(t, secondHalf, map[uint32]xtime.Ranges(second.Fulfilled())) + testBasebootstrapperNext(t, nextUnfulfilled, true) } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go index d1869dc926..39f098ea46 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go @@ -22,6 +22,8 @@ package commitlog import ( "errors" + "math" + goruntime "runtime" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/runtime" @@ -33,22 +35,20 @@ const ( // value for whether to return unfulfilled when encountering corrupt // commit log files. DefaultReturnUnfulfilledForCorruptCommitLogFiles = true - - defaultEncodingConcurrency = 4 - defaultMergeShardConcurrency = 4 ) var ( - errEncodingConcurrencyPositive = errors.New("encoding concurrency must be positive") - errMergeShardConcurrencyPositive = errors.New("merge shard concurrency must be positive") + errAccumulateConcurrencyPositive = errors.New("accumulate concurrency must be positive") errRuntimeOptsMgrNotSet = errors.New("runtime options manager is not set") + + // defaultAccumulateConcurrency determines how fast to accumulate results. + defaultAccumulateConcurrency = int(math.Max(float64(goruntime.NumCPU())*0.75, 1)) ) type options struct { resultOpts result.Options commitLogOpts commitlog.Options - encodingConcurrency int - mergeShardConcurrency int + accumulateConcurrency int runtimeOptsMgr runtime.OptionsManager returnUnfulfilledForCorruptCommitLogFiles bool } @@ -58,18 +58,14 @@ func NewOptions() Options { return &options{ resultOpts: result.NewOptions(), commitLogOpts: commitlog.NewOptions(), - encodingConcurrency: defaultEncodingConcurrency, - mergeShardConcurrency: defaultMergeShardConcurrency, + accumulateConcurrency: defaultAccumulateConcurrency, returnUnfulfilledForCorruptCommitLogFiles: DefaultReturnUnfulfilledForCorruptCommitLogFiles, } } func (o *options) Validate() error { - if o.encodingConcurrency <= 0 { - return errEncodingConcurrencyPositive - } - if o.mergeShardConcurrency <= 0 { - return errMergeShardConcurrencyPositive + if o.accumulateConcurrency <= 0 { + return errAccumulateConcurrencyPositive } if o.runtimeOptsMgr == nil { return errRuntimeOptsMgrNotSet @@ -97,24 +93,14 @@ func (o *options) CommitLogOptions() commitlog.Options { return o.commitLogOpts } -func (o *options) SetEncodingConcurrency(value int) Options { - opts := *o - opts.encodingConcurrency = value - return &opts -} - -func (o *options) EncodingConcurrency() int { - return o.encodingConcurrency -} - -func (o *options) SetMergeShardsConcurrency(value int) Options { +func (o *options) SetAccumulateConcurrency(value int) Options { opts := *o - opts.mergeShardConcurrency = value + opts.accumulateConcurrency = value return &opts } -func (o *options) MergeShardsConcurrency() int { - return o.mergeShardConcurrency +func (o *options) AccumulateConcurrency() int { + return o.accumulateConcurrency } func (o *options) SetRuntimeOptionsManager(value runtime.OptionsManager) Options { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 3367863234..358644ea70 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -21,6 +21,7 @@ package commitlog import ( + "bytes" "errors" "fmt" "io" @@ -28,35 +29,28 @@ import ( "time" "github.com/m3db/m3/src/cluster/shard" - "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" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/storage/index/convert" + "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/topology" "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/instrument" "github.com/m3db/m3/src/x/pool" - xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" "github.com/uber-go/tally" "go.uber.org/zap" ) -var ( - errIndexingNotEnableForNamespace = errors.New("indexing not enabled for namespace") -) - const ( - encoderChanBufSize = 1000 + workerChannelSize = 256 ) type newIteratorFn func(opts commitlog.IteratorOpts) ( @@ -65,8 +59,9 @@ type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint3 type newReaderFn func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) type commitLogSource struct { - opts Options - log *zap.Logger + opts Options + log *zap.Logger + nowFn func() time.Time // Filesystem inspection capture before node was started. inspection fs.Inspection @@ -75,15 +70,53 @@ type commitLogSource struct { snapshotFilesFn snapshotFilesFn newReaderFn newReaderFn - metrics commitLogSourceDataAndIndexMetrics + metrics commitLogSourceMetrics +} + +type bootstrapNamespace struct { + namespaceID []byte + bootstrapping bool + dataAndIndexShardRanges result.ShardTimeRanges + namespace namespace.Metadata + namespaceContext namespace.Context + dataBlockSize time.Duration + accumulator bootstrap.NamespaceDataAccumulator +} + +type seriesMap map[seriesMapKey]*seriesMapEntry + +type seriesMapKey struct { + fileReadID uint64 + uniqueIndex uint64 +} + +type seriesMapEntry struct { + namespace *bootstrapNamespace + series bootstrap.CheckoutSeriesResult +} + +// accumulateArg contains all the information a worker go-routine needs to +// accumulate a write for encoding into the database. +type accumulateArg struct { + namespace *bootstrapNamespace + series bootstrap.CheckoutSeriesResult + shard uint32 + dp ts.Datapoint + unit xtime.Unit + annotation ts.Annotation } -type encoder struct { - lastWriteAt time.Time - enc encoding.Encoder +type accumulateWorker struct { + inputCh chan accumulateArg + datapointsSkippedNotInRange int + datapointsRead int + numErrors int } -func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source { +func newCommitLogSource( + opts Options, + inspection fs.Inspection, +) bootstrap.Source { scope := opts. ResultOptions(). InstrumentOptions(). @@ -97,6 +130,7 @@ func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source InstrumentOptions(). Logger(). With(zap.String("bootstrapper", "commitlog")), + nowFn: opts.ResultOptions().ClockOptions().NowFn(), inspection: inspection, @@ -104,19 +138,19 @@ func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source snapshotFilesFn: fs.SnapshotFiles, newReaderFn: fs.NewReader, - metrics: newCommitLogSourceDataAndIndexMetrics(scope), + metrics: newCommitLogSourceMetrics(scope), } } -func (s *commitLogSource) Can(strategy bootstrap.Strategy) bool { - switch strategy { - case bootstrap.BootstrapSequential: - return true - } - return false +func (s *commitLogSource) AvailableData( + ns namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + runOpts bootstrap.RunOptions, +) (result.ShardTimeRanges, error) { + return s.availability(ns, shardsTimeRanges, runOpts) } -func (s *commitLogSource) AvailableData( +func (s *commitLogSource) AvailableIndex( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts bootstrap.RunOptions, @@ -124,142 +158,337 @@ func (s *commitLogSource) AvailableData( return s.availability(ns, shardsTimeRanges, runOpts) } -// ReadData will read all commitlog files on disk, as well as as the latest snapshot for +type readNamespaceResult struct { + namespace bootstrap.Namespace + dataAndIndexShardRanges result.ShardTimeRanges +} + +// Read will read all commitlog files on disk, as well as as the latest snapshot for // each shard/block combination (if it exists) and merge them. // TODO(rartoul): Make this take the SnapshotMetadata files into account to reduce the // number of commitlogs / snapshots that we need to read. -func (s *commitLogSource) ReadData( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - if shardsTimeRanges.IsEmpty() { - return result.NewDataBootstrapResult(), nil +func (s *commitLogSource) Read( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + timeRangesEmpty := true + for _, elem := range namespaces.Namespaces.Iter() { + namespace := elem.Value() + dataRangesNotEmpty := !namespace.DataRunOptions.ShardTimeRanges.IsEmpty() + + indexEnabled := namespace.Metadata.Options().IndexOptions().Enabled() + indexRangesNotEmpty := indexEnabled && !namespace.IndexRunOptions.ShardTimeRanges.IsEmpty() + if dataRangesNotEmpty || indexRangesNotEmpty { + timeRangesEmpty = false + break + } + } + if timeRangesEmpty { + // Return empty result with no unfulfilled ranges. + return bootstrap.NewNamespaceResults(namespaces), nil } var ( - // Emit bootstrapping gauge for duration of ReadData - doneReadingData = s.metrics.data.emitBootstrapping() - encounteredCorruptData = false - fsOpts = s.opts.CommitLogOptions().FilesystemOptions() - filePathPrefix = fsOpts.FilePathPrefix() + // Emit bootstrapping gauge for duration of ReadData. + doneReadingData = s.metrics.emitBootstrapping() + encounteredCorruptData = false + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + filePathPrefix = fsOpts.FilePathPrefix() + namespaceIter = namespaces.Namespaces.Iter() + namespaceResults = make(map[string]*readNamespaceResult, len(namespaceIter)) + setInitialTopologyState bool + initialTopologyState *topology.StateSnapshot ) defer doneReadingData() - // Determine which snapshot files are available. - snapshotFilesByShard, err := s.snapshotFilesByShard( - ns.ID(), filePathPrefix, shardsTimeRanges) - if err != nil { - return nil, err + startSnapshotsRead := s.nowFn() + s.log.Info("read snapshots start") + for _, elem := range namespaceIter { + ns := elem.Value() + accumulator := ns.DataAccumulator + + // NB(r): Combine all shard time ranges across data and index + // so we can do in one go. + shardTimeRanges := result.ShardTimeRanges{} + shardTimeRanges.AddRanges(ns.DataRunOptions.ShardTimeRanges) + if ns.Metadata.Options().IndexOptions().Enabled() { + shardTimeRanges.AddRanges(ns.IndexRunOptions.ShardTimeRanges) + } + + namespaceResults[ns.Metadata.ID().String()] = &readNamespaceResult{ + namespace: ns, + dataAndIndexShardRanges: shardTimeRanges, + } + + // Make the initial topology state available. + if !setInitialTopologyState { + setInitialTopologyState = true + initialTopologyState = ns.DataRunOptions.RunOptions.InitialTopologyState() + } + + // Determine which snapshot files are available. + snapshotFilesByShard, err := s.snapshotFilesByShard( + ns.Metadata.ID(), filePathPrefix, shardTimeRanges) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + mostRecentCompleteSnapshotByBlockShard, err := s.mostRecentSnapshotByBlockShard( + ns.Metadata, shardTimeRanges, snapshotFilesByShard) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + // Start by reading any available snapshot files. + blockSize := ns.Metadata.Options().RetentionOptions().BlockSize() + for shard, tr := range shardTimeRanges { + err := s.bootstrapShardSnapshots( + ns.Metadata, accumulator, shard, tr, blockSize, + mostRecentCompleteSnapshotByBlockShard) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + } } + s.log.Info("read snapshots done", + zap.Duration("took", s.nowFn().Sub(startSnapshotsRead))) + + // Setup the series accumulator pipeline. var ( - bOpts = s.opts.ResultOptions() - blOpts = bOpts.DatabaseBlockOptions() - blockSize = ns.Options().RetentionOptions().BlockSize() + numWorkers = s.opts.AccumulateConcurrency() + workers = make([]*accumulateWorker, 0, numWorkers) ) - - readCommitLogPred, mostRecentCompleteSnapshotByBlockShard, err := s.newReadCommitlogPredAndMostRecentSnapshotByBlockShard( - ns, shardsTimeRanges, snapshotFilesByShard) - if err != nil { - return nil, err + for i := 0; i < numWorkers; i++ { + worker := &accumulateWorker{ + inputCh: make(chan accumulateArg, workerChannelSize), + } + workers = append(workers, worker) + } + closedWorkerChannels := false + closeWorkerChannels := func() { + if closedWorkerChannels { + return + } + closedWorkerChannels = true + for _, worker := range workers { + close(worker.inputCh) + } } + // NB(r): Ensure that channels always get closed. + defer closeWorkerChannels() // Setup the commit log iterator. var ( - nsID = ns.ID() - seriesSkipped int - datapointsSkipped int - datapointsRead int - - // TODO(rartoul): When we implement caching data across namespaces, this will need - // to be commitlog.ReadAllSeriesPredicate() if CacheSeriesMetadata() is enabled - // because we'll need to read data for all namespaces, not just the one we're currently - // bootstrapping. - readSeriesPredicate = func(id ident.ID, namespace ident.ID) bool { - shouldReadSeries := nsID.Equal(namespace) - if !shouldReadSeries { - seriesSkipped++ - } - return shouldReadSeries - } - iterOpts = commitlog.IteratorOpts{ - CommitLogOptions: s.opts.CommitLogOptions(), - FileFilterPredicate: readCommitLogPred, - SeriesFilterPredicate: readSeriesPredicate, + CommitLogOptions: s.opts.CommitLogOptions(), + FileFilterPredicate: s.readCommitLogFilePredicate, + // NB(r): ReturnMetadataAsRef used to all series metadata as + // references instead of pulling from pool and allocating, + // which means need to not hold onto any references returned + // from a call to the commit log read log entry call. + ReturnMetadataAsRef: true, } + datapointsSkippedNotBootstrappingNamespace = 0 + startCommitLogsRead = s.nowFn() ) - + s.log.Info("read commit logs start") defer func() { - s.log.Info("ReadData finished", - zap.Int("seriesSkipped", seriesSkipped), - zap.Int("datapointsSkipped", datapointsSkipped), - zap.Int("datapointsRead", datapointsRead)) + datapointsSkippedNotInRange := 0 + datapointsRead := 0 + for _, worker := range workers { + datapointsSkippedNotInRange += worker.datapointsSkippedNotInRange + datapointsRead += worker.datapointsRead + } + s.log.Info("read finished", + zap.Stringer("took", s.nowFn().Sub(startCommitLogsRead)), + zap.Int("datapointsRead", datapointsRead), + zap.Int("datapointsSkippedNotInRange", datapointsSkippedNotInRange), + zap.Int("datapointsSkippedNotBootstrappingNamespace", datapointsSkippedNotBootstrappingNamespace)) }() iter, corruptFiles, err := s.newIteratorFn(iterOpts) if err != nil { - return nil, fmt.Errorf("unable to create commit log iterator: %v", err) + err = fmt.Errorf("unable to create commit log iterator: %v", err) + return bootstrap.NamespaceResults{}, err } if len(corruptFiles) > 0 { - s.logAndEmitCorruptFiles(corruptFiles, true) + s.logAndEmitCorruptFiles(corruptFiles) encounteredCorruptData = true } defer iter.Close() - // Setup the M3TSZ encoding pipeline - var ( - // +1 so we can use the shard number as an index throughout without constantly - // remembering to subtract 1 to convert to zero-based indexing - numShards = s.findHighestShard(shardsTimeRanges) + 1 - numConc = s.opts.EncodingConcurrency() - encoderPool = blOpts.EncoderPool() - workerErrs = make([]int, numConc) - shardDataByShard = s.newShardDataByShard(shardsTimeRanges, numShards) - ) - - encoderChans := make([]chan encoderArg, numConc) - for i := 0; i < numConc; i++ { - encoderChans[i] = make(chan encoderArg, encoderChanBufSize) - } - - // Spin up numConc background go-routines to handle M3TSZ encoding. This must + // Spin up numWorkers background go-routines to handle accumulation. This must // happen before we start reading to prevent infinitely blocking writes to - // the encoderChans. - wg := &sync.WaitGroup{} - for workerNum, encoderChan := range encoderChans { + // the worker channels. + var wg sync.WaitGroup + for _, worker := range workers { + worker := worker wg.Add(1) - go s.startM3TSZEncodingWorker( - ns, runOpts, workerNum, encoderChan, shardDataByShard, encoderPool, workerErrs, blOpts, wg) + go func() { + s.startAccumulateWorker(worker) + wg.Done() + }() } - // Read / M3TSZ encode all the datapoints in the commit log that we need to read. + var ( + // NB(r): Use pointer type for the namespaces so we don't have to + // memcopy the large namespace context struct to the work channel and + // can pass by pointer. + // For the commit log series map we use by value since it grows + // large in size and we want to avoid allocating a struct per series + // read and just have a by value struct stored in the map (also makes + // reusing memory set aside on a per series level between commit + // log files much easier to do). + commitLogNamespaces []*bootstrapNamespace + commitLogSeries = make(map[seriesMapKey]seriesMapEntry) + workerEnqueue = 0 + tagDecoder = s.opts.CommitLogOptions().FilesystemOptions().TagDecoderPool().Get() + tagDecoderCheckedBytes = checked.NewBytes(nil, nil) + ) + tagDecoderCheckedBytes.IncRef() + + // Read and accumulate all the log entries in the commit log that we need + // to read. + var lastFileReadID uint64 for iter.Next() { - series, dp, unit, annotation := iter.Current() - if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) { - datapointsSkipped++ + entry := iter.Current() + + currFileReadID := entry.Metadata.FileReadID + if currFileReadID != lastFileReadID { + // NB(r): If switched between files, we can reuse slice and + // map which is useful so map doesn't grow infinitely. + for k := range commitLogSeries { + delete(commitLogSeries, k) + } + lastFileReadID = currFileReadID + } + + // First lookup series, if not found we are guaranteed to have + // the series metadata returned by the commit log reader. + seriesKey := seriesMapKey{ + fileReadID: entry.Metadata.FileReadID, + uniqueIndex: entry.Metadata.SeriesUniqueIndex, + } + + seriesEntry, ok := commitLogSeries[seriesKey] + if !ok { + // Resolve the namespace. + var ( + nsID = entry.Series.Namespace + nsIDBytes = nsID.Bytes() + ns *bootstrapNamespace + ) + for _, elem := range commitLogNamespaces { + if bytes.Equal(elem.namespaceID, nsIDBytes) { + ns = elem + break + } + } + if ns == nil { + // NB(r): Need to create an entry into our namespaces, this will happen + // at most once per commit log file read and unique namespace. + nsResult, ok := namespaceResults[nsID.String()] + // Take a copy so that not taking ref to reused bytes from the commit log. + nsIDCopy := append([]byte(nil), nsIDBytes...) + if !ok { + // Not bootstrapping this namespace. + ns = &bootstrapNamespace{ + namespaceID: nsIDCopy, + bootstrapping: false, + } + } else { + // Bootstrapping this namespace. + nsMetadata := nsResult.namespace.Metadata + ns = &bootstrapNamespace{ + namespaceID: nsIDCopy, + bootstrapping: true, + dataAndIndexShardRanges: nsResult.dataAndIndexShardRanges, + namespace: nsMetadata, + namespaceContext: namespace.NewContextFrom(nsMetadata), + dataBlockSize: nsMetadata.Options().RetentionOptions().BlockSize(), + accumulator: nsResult.namespace.DataAccumulator, + } + } + // Append for quick re-lookup with other series. + commitLogNamespaces = append(commitLogNamespaces, ns) + } + if !ns.bootstrapping { + // NB(r): Just set the series map entry to the memoized + // fact that we are not bootstrapping this namespace. + seriesEntry = seriesMapEntry{ + namespace: ns, + } + } else { + // Resolve the series in the accumulator. + accumulator := ns.accumulator + + // NB(r): Make sure that only series.EncodedTags are used and not + // series.Tags (we explicitly ask for references to be returned and to + // avoid decoding the tags if we don't have to). + if decodedTags := len(entry.Series.Tags.Values()); decodedTags > 0 { + msg := "commit log reader expects encoded tags" + instrumentOpts := s.opts.ResultOptions().InstrumentOptions() + instrument.EmitAndLogInvariantViolation(instrumentOpts, func(l *zap.Logger) { + l.Error(msg, + zap.Int("decodedTags", decodedTags), + zap.Int("encodedTags", len(entry.Series.EncodedTags))) + }) + err := instrument.InvariantErrorf(fmt.Sprintf("%s: decoded=%d", msg, decodedTags)) + return bootstrap.NamespaceResults{}, err + } + + var tagIter ident.TagIterator + if len(entry.Series.EncodedTags) > 0 { + tagDecoderCheckedBytes.Reset(entry.Series.EncodedTags) + tagDecoder.Reset(tagDecoderCheckedBytes) + tagIter = tagDecoder + } else { + // NB(r): Always expect a tag iterator in checkout series. + tagIter = ident.EmptyTagIterator + } + + // Check out the series for writing, no need for concurrency + // as commit log bootstrapper does not perform parallel + // checking out of series. + series, err := accumulator.CheckoutSeriesWithoutLock( + entry.Series.Shard, + entry.Series.ID, + tagIter, + ) + + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + seriesEntry = seriesMapEntry{ + namespace: ns, + series: series, + } + } + + commitLogSeries[seriesKey] = seriesEntry + } + + // If not bootstrapping this namespace then skip this result. + if !seriesEntry.namespace.bootstrapping { + datapointsSkippedNotBootstrappingNamespace++ continue } - datapointsRead++ - - // Distribute work such that each encoder goroutine is responsible for - // approximately numShards / numConc shards. This also means that all - // datapoints for a given shard/series will be processed in a serialized - // manner. - // We choose to distribute work by shard instead of series.UniqueIndex - // because it means that all accesses to the shardDataByShard slice don't need - // to be synchronized because each index belongs to a single shard so it - // will only be accessed serially from a single worker routine. - encoderChans[series.Shard%uint32(numConc)] <- encoderArg{ - series: series, - dp: dp, - unit: unit, - annotation: annotation, - blockStart: dp.Timestamp.Truncate(blockSize), + // Distribute work. + // NB(r): In future we could batch a few points together before sending + // to a channel to alleviate lock contention/stress on the channels. + workerEnqueue++ + worker := workers[workerEnqueue%numWorkers] + worker.inputCh <- accumulateArg{ + namespace: seriesEntry.namespace, + series: seriesEntry.series, + shard: seriesEntry.series.Shard, + dp: entry.Datapoint, + unit: entry.Unit, + annotation: entry.Annotation, } } @@ -269,45 +498,49 @@ func (s *commitLogSource) ReadData( // opportunity to repair the data instead of failing the bootstrap // altogether. s.log.Error("error in commitlog iterator", zap.Error(iterErr)) - s.metrics.data.corruptCommitlogFile.Inc(1) + s.metrics.corruptCommitlogFile.Inc(1) encounteredCorruptData = true } - for _, encoderChan := range encoderChans { - close(encoderChan) - } + // Close the worker channels since we've enqueued all required data. + closeWorkerChannels() // Block until all required data from the commit log has been read and - // encoded by the worker goroutines + // accumulated by the worker goroutines. wg.Wait() - s.logEncodingOutcome(workerErrs, iter) - - // Merge all the different encoders from the commit log that we created with - // the data that is available in the snapshot files. - s.log.Info("starting merge...") - mergeStart := time.Now() - bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots( - ns, - shardsTimeRanges, - snapshotFilesByShard, - mostRecentCompleteSnapshotByBlockShard, - int(numShards), - blockSize, - shardDataByShard, - ) - if err != nil { - return nil, err - } - s.log.Info("done merging...", zap.Duration("took", time.Since(mergeStart))) + // Log the outcome and calculate if required to return unfulfilled. + s.logAccumulateOutcome(workers, iter) shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled( - encounteredCorruptData, ns, shardsTimeRanges, runOpts) + workers, encounteredCorruptData, initialTopologyState) if err != nil { - return nil, err + return bootstrap.NamespaceResults{}, err } - if shouldReturnUnfulfilled { - bootstrapResult.SetUnfulfilled(shardsTimeRanges) + bootstrapResult := bootstrap.NamespaceResults{ + Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}), + } + for _, ns := range namespaceResults { + id := ns.namespace.Metadata.ID() + dataResult := result.NewDataBootstrapResult() + if shouldReturnUnfulfilled { + shardTimeRanges := ns.namespace.DataRunOptions.ShardTimeRanges + dataResult = shardTimeRanges.ToUnfulfilledDataResult() + } + var indexResult result.IndexBootstrapResult + if ns.namespace.Metadata.Options().IndexOptions().Enabled() { + indexResult = result.NewIndexBootstrapResult() + if shouldReturnUnfulfilled { + shardTimeRanges := ns.namespace.IndexRunOptions.ShardTimeRanges + indexResult = shardTimeRanges.ToUnfulfilledIndexResult() + } + } + bootstrapResult.Results.Set(id, bootstrap.NamespaceResult{ + Metadata: ns.namespace.Metadata, + Shards: ns.namespace.Shards, + DataResult: dataResult, + IndexResult: indexResult, + }) } return bootstrapResult, nil @@ -330,21 +563,6 @@ func (s *commitLogSource) snapshotFilesByShard( return snapshotFilesByShard, nil } -func (s *commitLogSource) newShardDataByShard( - shardsTimeRanges result.ShardTimeRanges, - numShards uint32, -) []shardData { - shardDataByShard := make([]shardData, numShards) - for shard := range shardsTimeRanges { - shardDataByShard[shard] = shardData{ - series: NewMap(MapOptions{}), - ranges: shardsTimeRanges[shard], - } - } - - return shardDataByShard -} - // mostRecentCompleteSnapshotByBlockShard returns a // map[xtime.UnixNano]map[uint32]fs.FileSetFile with the contract that // for each shard/block combination in shardsTimeRanges, an entry will @@ -434,22 +652,14 @@ func (s *commitLogSource) mostRecentCompleteSnapshotByBlockShard( func (s *commitLogSource) bootstrapShardSnapshots( ns namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, shard uint32, - metadataOnly bool, shardTimeRanges xtime.Ranges, blockSize time.Duration, - snapshotFiles fs.FileSetFilesSlice, mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile, -) (result.ShardResult, error) { - var ( - shardResult result.ShardResult - allSeriesSoFar *result.Map - rangeIter = shardTimeRanges.Iter() - err error - ) - +) error { + rangeIter := shardTimeRanges.Iter() for rangeIter.Next() { - var ( currRange = rangeIter.Value() currRangeDuration = currRange.End.Sub(currRange.Start) @@ -457,17 +667,12 @@ func (s *commitLogSource) bootstrapShardSnapshots( ) if !isMultipleOfBlockSize { - return nil, fmt.Errorf( + return fmt.Errorf( "received bootstrap range that is not multiple of blockSize, blockSize: %d, start: %s, end: %s", blockSize, currRange.End.String(), currRange.Start.String(), ) } - // Reset this after we bootstrap each block to make sure its up to date. - if shardResult != nil { - allSeriesSoFar = shardResult.AllSeries() - } - for blockStart := currRange.Start.Truncate(blockSize); blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) { snapshotsForBlock := mostRecentCompleteSnapshotByBlockShard[xtime.ToUnixNano(blockStart)] mostRecentCompleteSnapshotForShardBlock := snapshotsForBlock[shard] @@ -485,46 +690,38 @@ func (s *commitLogSource) bootstrapShardSnapshots( continue } - shardResult, err = s.bootstrapShardBlockSnapshot( - ns, shard, blockStart, metadataOnly, shardResult, allSeriesSoFar, blockSize, - snapshotFiles, mostRecentCompleteSnapshotForShardBlock) - if err != nil { - return shardResult, err + if err := s.bootstrapShardBlockSnapshot( + ns, accumulator, shard, blockStart, blockSize, + mostRecentCompleteSnapshotForShardBlock); err != nil { + return err } } } - if shardResult == nil { - shardResult = result.NewShardResult(0, s.opts.ResultOptions()) - } - return shardResult, nil + return nil } func (s *commitLogSource) bootstrapShardBlockSnapshot( ns namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, shard uint32, blockStart time.Time, - metadataOnly bool, - shardResult result.ShardResult, - allSeriesSoFar *result.Map, blockSize time.Duration, - snapshotFiles fs.FileSetFilesSlice, mostRecentCompleteSnapshot fs.FileSetFile, -) (result.ShardResult, error) { +) error { var ( bOpts = s.opts.ResultOptions() blOpts = bOpts.DatabaseBlockOptions() blocksPool = blOpts.DatabaseBlockPool() bytesPool = blOpts.BytesPool() fsOpts = s.opts.CommitLogOptions().FilesystemOptions() - idPool = s.opts.CommitLogOptions().IdentifierPool() nsCtx = namespace.NewContextFrom(ns) ) - // Bootstrap the snapshot file + // Bootstrap the snapshot file. reader, err := s.newReaderFn(bytesPool, fsOpts) if err != nil { - return shardResult, err + return err } err = reader.Open(fs.DataReaderOpenOptions{ @@ -537,7 +734,7 @@ func (s *commitLogSource) bootstrapShardBlockSnapshot( FileSetType: persist.FileSetSnapshotType, }) if err != nil { - return shardResult, err + return err } defer func() { err := reader.Close() @@ -554,94 +751,54 @@ func (s *commitLogSource) bootstrapShardBlockSnapshot( zap.Uint32("shard", shard), zap.Time("blockStart", blockStart), zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex)) + for { - var ( - id ident.ID - tagsIter ident.TagIterator - data checked.Bytes - expectedChecksum uint32 - ) - if metadataOnly { - id, tagsIter, _, _, err = reader.ReadMetadata() - } else { - id, tagsIter, data, expectedChecksum, err = reader.Read() - } + id, tags, data, expectedChecksum, err := reader.Read() if err != nil && err != io.EOF { - return shardResult, err + return err } - if err == io.EOF { break } dbBlock := blocksPool.Get() - dbBlock.Reset(blockStart, blockSize, ts.NewSegment(data, nil, ts.FinalizeHead), nsCtx) + dbBlock.Reset(blockStart, blockSize, + ts.NewSegment(data, nil, ts.FinalizeHead), nsCtx) - if !metadataOnly { - // Resetting the block will trigger a checksum calculation, so use that instead - // of calculating it twice. - checksum, err := dbBlock.Checksum() - if err != nil { - return shardResult, err - } - - if checksum != expectedChecksum { - return shardResult, fmt.Errorf("checksum for series: %s was %d but expected %d", id, checksum, expectedChecksum) - } + // Resetting the block will trigger a checksum calculation, so use that instead + // of calculating it twice. + checksum, err := dbBlock.Checksum() + if err != nil { + return err } - - var ( - tags ident.Tags - shouldDecodeTags = true - ) - if allSeriesSoFar != nil { - if existing, ok := allSeriesSoFar.Get(id); ok { - // If we've already bootstrapped this series for a different block, we don't need - // another copy of the IDs and tags. - id.Finalize() - id = existing.ID - tags = existing.Tags - shouldDecodeTags = false - } + if checksum != expectedChecksum { + return fmt.Errorf("checksum for series: %s was %d but expected %d", id, checksum, expectedChecksum) } - if shouldDecodeTags { - // Only spend cycles decoding the tags if we've never seen them before. - if tagsIter.Remaining() > 0 { - tags, err = convert.TagsFromTagsIter(id, tagsIter, idPool) - if err != nil { - return shardResult, fmt.Errorf("unable to decode tags: %v", err) - } - } + // NB(r): No parallelization required to checkout the series. + ref, err := accumulator.CheckoutSeriesWithoutLock(shard, id, tags) + if err != nil { + return err } - // Always close even if we didn't use it. - tagsIter.Close() - - // Mark the ID and Tags as no finalize to enable no-copy optimization later - // in the bootstrap process (when they're being loaded into the shard). Also, - // technically we'll be calling NoFinalize() repeatedly on the same IDs for - // different blocks since we're reusing them, but thats ok as it an idempotent - // operation and there is no concurrency here. - id.NoFinalize() - tags.NoFinalize() - - if shardResult == nil { - // Delay initialization so we can estimate size. - shardResult = result.NewShardResult(reader.Entries(), s.opts.ResultOptions()) + // Load into series. + if err := ref.Series.LoadBlock(dbBlock, series.WarmWrite); err != nil { + return err } - shardResult.AddBlock(id, tags, dbBlock) + + // Always finalize both ID and tags after loading block. + id.Finalize() + tags.Close() } - return shardResult, nil + return nil } -func (s *commitLogSource) newReadCommitlogPredAndMostRecentSnapshotByBlockShard( +func (s *commitLogSource) mostRecentSnapshotByBlockShard( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, snapshotFilesByShard map[uint32]fs.FileSetFilesSlice, ) ( - commitlog.FileFilterPredicate, map[xtime.UnixNano]map[uint32]fs.FileSetFile, error, ) { @@ -654,7 +811,7 @@ func (s *commitLogSource) newReadCommitlogPredAndMostRecentSnapshotByBlockShard( if mostRecent.CachedSnapshotTime.IsZero() { // Should never happen. - return nil, nil, instrument.InvariantErrorf( + return nil, instrument.InvariantErrorf( "shard: %d and block: %s had zero value for most recent snapshot time", shard, block.ToTime().String()) } @@ -666,131 +823,95 @@ func (s *commitLogSource) newReadCommitlogPredAndMostRecentSnapshotByBlockShard( } } - // TODO(rartoul): Refactor this to take the SnapshotMetadata files into account to reduce - // the number of commitlog files that need to be read. - return func(f commitlog.FileFilterInfo) bool { - // Read all the commitlog files that were available on disk before the node started - // accepting writes. - commitlogFilesPresentBeforeStart := s.inspection.CommitLogFilesSet() - if f.IsCorrupt { - // Corrupt files that existed on disk before the node started should be included so - // that the commitlog bootstrapper can detect them and determine if it will return - // unfulfilled or ignore them. - // - // Corrupt files that did not exist on disk before the node started should always be - // ignored since they have no impact on the bootstrapping process and likely only - // appear corrupt because they were just created recently by the current node as - // its alreadying accepting writes at this point. - _, ok := commitlogFilesPresentBeforeStart[f.Err.Path()] - return ok - } - // Only attempt to read commitlog files that were present on disk before the node started. - // If a commitlog file was not present when the node started then it was created once the - // node began accepting writes and the data is already in memory. - _, ok := commitlogFilesPresentBeforeStart[f.File.FilePath] + return mostRecentCompleteSnapshotByBlockShard, nil +} + +// TODO(rartoul): Refactor this to take the SnapshotMetadata files into account to reduce +// the number of commitlog files that need to be read. +func (s *commitLogSource) readCommitLogFilePredicate(f commitlog.FileFilterInfo) bool { + // Read all the commitlog files that were available on disk before the node started + // accepting writes. + commitlogFilesPresentBeforeStart := s.inspection.CommitLogFilesSet() + if f.IsCorrupt { + // Corrupt files that existed on disk before the node started should be included so + // that the commitlog bootstrapper can detect them and determine if it will return + // unfulfilled or ignore them. + // + // Corrupt files that did not exist on disk before the node started should always be + // ignored since they have no impact on the bootstrapping process and likely only + // appear corrupt because they were just created recently by the current node as + // its alreadying accepting writes at this point. + _, ok := commitlogFilesPresentBeforeStart[f.Err.Path()] return ok - }, mostRecentCompleteSnapshotByBlockShard, nil + } + // Only attempt to read commitlog files that were present on disk before the node started. + // If a commitlog file was not present when the node started then it was created once the + // node began accepting writes and the data is already in memory. + _, ok := commitlogFilesPresentBeforeStart[f.File.FilePath] + return ok } -func (s *commitLogSource) startM3TSZEncodingWorker( - ns namespace.Metadata, - runOpts bootstrap.RunOptions, - workerNum int, - ec <-chan encoderArg, - unmerged []shardData, - encoderPool encoding.EncoderPool, - workerErrs []int, - blopts block.Options, - wg *sync.WaitGroup, +func (s *commitLogSource) startAccumulateWorker( + worker *accumulateWorker, ) { - nsCtx := namespace.NewContextFrom(ns) - for arg := range ec { + ctx := context.NewContext() + for input := range worker.inputCh { var ( - series = arg.series - dp = arg.dp - unit = arg.unit - annotation = arg.annotation - blockStart = arg.blockStart + namespace = input.namespace + entry = input.series + shard = input.shard + dp = input.dp + unit = input.unit + annotation = input.annotation ) - var ( - unmergedShard = unmerged[series.Shard].series - unmergedSeries, ok = unmergedShard.Get(series.ID) - ) - if !ok { - unmergedSeries = metadataAndEncodersByTime{ - id: series.ID, - tags: series.Tags, - encoders: make(map[xtime.UnixNano][]encoder)} - // Have to use unsafe because we don't want to copy the IDs we put - // into this map because its lifecycle is much shorter than that of - // the IDs we're putting into it so copying would waste too much - // memory unnecessarily, and we don't want to finalize the IDs for the - // same reason. - unmergedShard.SetUnsafe( - series.ID, unmergedSeries, - SetUnsafeOptions{NoCopyKey: true, NoFinalizeKey: true}) + if !s.shouldAccumulateForTime(namespace, shard, dp.Timestamp) { + worker.datapointsSkippedNotInRange++ + continue } - var ( - err error - blockStartNano = xtime.ToUnixNano(blockStart) - unmergedBlock = unmergedSeries.encoders[blockStartNano] - wroteExisting = false - ) - for i := range unmergedBlock { - // TODO(r): Write unit test to ensure that different values that arrive - // later in the commit log will upsert the previous value when bootstrapping - // Tracking with issue: https://github.com/m3db/m3/issues/898 - if unmergedBlock[i].lastWriteAt.Before(dp.Timestamp) { - unmergedBlock[i].lastWriteAt = dp.Timestamp - err = unmergedBlock[i].enc.Encode(dp, unit, annotation) - wroteExisting = true - break - } - } - if !wroteExisting { - enc := encoderPool.Get() - enc.Reset(blockStart, blopts.DatabaseBlockAllocSize(), nsCtx.Schema) - - err = enc.Encode(dp, unit, annotation) - if err == nil { - unmergedBlock = append(unmergedBlock, encoder{ - lastWriteAt: dp.Timestamp, - enc: enc, - }) - unmergedSeries.encoders[blockStartNano] = unmergedBlock - } - } + worker.datapointsRead++ + _, err := entry.Series.Write(ctx, dp.Timestamp, dp.Value, + unit, annotation, series.WriteOptions{ + SchemaDesc: namespace.namespaceContext.Schema, + // NB(r): Make sure this is the series we originally + // checked out for writing too (which should be guaranteed + // by the fact during shard tick we do not expire any + // series unless they are bootstrapped). + MatchUniqueIndex: true, + MatchUniqueIndexValue: entry.UniqueIndex, + BootstrapWrite: true, + }) if err != nil { - workerErrs[workerNum]++ + // NB(r): Only log first error per worker since this could be very + // noisy if it actually fails for "most" writes. + if worker.numErrors == 0 { + s.log.Error("failed to write commit log entry", zap.Error(err)) + } else { + // Always write a debug log, most of these will go nowhere if debug + // logging not enabled however. + s.log.Debug("failed to write commit log entry", zap.Error(err)) + } + worker.numErrors++ } } - wg.Done() } -func (s *commitLogSource) shouldEncodeForData( - unmerged []shardData, - dataBlockSize time.Duration, - series ts.Series, +func (s *commitLogSource) shouldAccumulateForTime( + ns *bootstrapNamespace, + shard uint32, timestamp time.Time, ) bool { - // Check if the shard number is higher the amount of space we pre-allocated. - // If it is, then it's not one of the shards we're trying to bootstrap - if series.Shard > uint32(len(unmerged)-1) { - return false - } - - // Check if the shard is one of the shards we're trying to bootstrap - ranges := unmerged[series.Shard].ranges - if ranges.IsEmpty() { - // Did not allocate map for this shard so not expecting data for it + // Check if the shard is one of the shards we're trying to bootstrap. + ranges, ok := ns.dataAndIndexShardRanges[shard] + if !ok || ranges.IsEmpty() { + // Not expecting data for this shard. return false } - // Check if the block corresponds to the time-range that we're trying to bootstrap - blockStart := timestamp.Truncate(dataBlockSize) - blockEnd := blockStart.Add(dataBlockSize) + // Check if the value corresponds to the time-range that we're trying to bootstrap. + blockStart := timestamp.Truncate(ns.dataBlockSize) + blockEnd := blockStart.Add(ns.dataBlockSize) blockRange := xtime.Range{ Start: blockStart, End: blockEnd, @@ -799,504 +920,22 @@ func (s *commitLogSource) shouldEncodeForData( return ranges.Overlaps(blockRange) } -func (s *commitLogSource) shouldIncludeInIndex( - shard uint32, - ts time.Time, - highestShard uint32, - indexBlockSize time.Duration, - bootstrapRangesByShard []xtime.Ranges, -) bool { - if shard > highestShard { - // Not trying to bootstrap this shard - return false - } - - rangesToBootstrap := bootstrapRangesByShard[shard] - if rangesToBootstrap.IsEmpty() { - // No ShardTimeRanges were provided for this shard, so we're not - // bootstrapping it. - return false - } - - // Check if the timestamp corresponds to one of the index blocks we're - // trying to bootstrap. - indexBlockStart := ts.Truncate(indexBlockSize) - indexBlockEnd := indexBlockStart.Add(indexBlockSize) - indexBlockRange := xtime.Range{ - Start: indexBlockStart, - End: indexBlockEnd, - } - - return rangesToBootstrap.Overlaps(indexBlockRange) -} - -func (s *commitLogSource) mergeAllShardsCommitLogEncodersAndSnapshots( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - snapshotFiles map[uint32]fs.FileSetFilesSlice, - mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile, - numShards int, - blockSize time.Duration, - unmerged []shardData, -) (result.DataBootstrapResult, error) { - var ( - shardErrs = make([]int, numShards) - shardEmptyErrs = make([]int, numShards) - bootstrapResult = result.NewDataBootstrapResult() - // Controls how many shards can be merged in parallel - workerPool = xsync.NewWorkerPool(s.opts.MergeShardsConcurrency()) - bootstrapResultLock sync.Mutex - wg sync.WaitGroup - ) - workerPool.Init() - - for shard, unmergedShard := range unmerged { - if unmergedShard.series == nil { - // Not bootstrapping this shard - continue - } - - nsCtx := namespace.NewContextFrom(ns) - - snapshotData, err := s.bootstrapShardSnapshots( - ns, - uint32(shard), - false, - shardsTimeRanges[uint32(shard)], - blockSize, - snapshotFiles[uint32(shard)], - mostRecentCompleteSnapshotByBlockShard, - ) - if err != nil { - bootstrapResultLock.Lock() - // Mark the shard time ranges as unfulfilled so a subsequent bootstrapper - // has the chance to fulfill it. - bootstrapResult.Add( - uint32(shard), - result.NewShardResult(0, s.opts.ResultOptions()), - shardsTimeRanges[uint32(shard)], - ) - bootstrapResultLock.Unlock() - return nil, err - } - - // Merge snapshot and commit log data - wg.Add(1) - shard, unmergedShard := shard, unmergedShard - mergeShardFunc := func() { - var shardResult result.ShardResult - shardResult, shardEmptyErrs[shard], shardErrs[shard] = s.mergeShardCommitLogEncodersAndSnapshots(nsCtx, - shard, snapshotData, unmergedShard, blockSize) - - if shardResult != nil && shardResult.NumSeries() > 0 { - // Prevent race conditions while updating bootstrapResult from multiple go-routines - bootstrapResultLock.Lock() - if shardEmptyErrs[shard] != 0 || shardErrs[shard] != 0 { - // If there were any errors, keep the data but mark the shard time ranges as - // unfulfilled so a subsequent bootstrapper has the chance to fulfill it. - bootstrapResult.Add(uint32(shard), shardResult, shardsTimeRanges[uint32(shard)]) - } else { - bootstrapResult.Add(uint32(shard), shardResult, xtime.Ranges{}) - } - bootstrapResultLock.Unlock() - } - wg.Done() - } - workerPool.Go(mergeShardFunc) - } - - // Wait for all merge goroutines to complete - wg.Wait() - s.logMergeShardsOutcome(shardErrs, shardEmptyErrs) - return bootstrapResult, nil -} - -func (s *commitLogSource) mergeShardCommitLogEncodersAndSnapshots( - nsCtx namespace.Context, - shard int, - snapshotData result.ShardResult, - unmergedShard shardData, - blockSize time.Duration, -) (result.ShardResult, int, int) { - var ( - bOpts = s.opts.ResultOptions() - blOpts = bOpts.DatabaseBlockOptions() - blocksPool = blOpts.DatabaseBlockPool() - multiReaderIteratorPool = blOpts.MultiReaderIteratorPool() - segmentReaderPool = blOpts.SegmentReaderPool() - encoderPool = blOpts.EncoderPool() - ) - - numSeries := 0 - if unmergedShard.series != nil { - numSeries = unmergedShard.series.Len() - } - - var ( - shardResult = result.NewShardResult(numSeries, s.opts.ResultOptions()) - numShardEmptyErrs int - numErrs int - ) - - allSnapshotSeries := snapshotData.AllSeries() - - if unmergedShard.series != nil { - for _, unmergedBlocks := range unmergedShard.series.Iter() { - val := unmergedBlocks.Value() - snapshotSeriesData, _ := allSnapshotSeries.Get(val.id) - seriesBlocks, numSeriesEmptyErrs, numSeriesErrs := s.mergeSeries( - nsCtx, - snapshotSeriesData, - val, - blocksPool, - multiReaderIteratorPool, - segmentReaderPool, - encoderPool, - blockSize, - blOpts, - ) - - if seriesBlocks != nil && seriesBlocks.Len() > 0 { - shardResult.AddSeries(val.id, val.tags, seriesBlocks) - } - - numShardEmptyErrs += numSeriesEmptyErrs - numErrs += numSeriesErrs - } - } - - allShardResultSeries := shardResult.AllSeries() - for _, val := range allSnapshotSeries.Iter() { - id := val.Key() - blocks := val.Value() - - if allShardResultSeries.Contains(id) { - // Already merged so we know the ID and tags from the snapshot - // won't be used and can be closed. We can't close the blocks - // though because we may have loaded some of the blocks into - // the shard result and we don't want to close them. - id.Finalize() - blocks.Tags.Finalize() - continue - } - - shardResult.AddSeries(id, blocks.Tags, blocks.Blocks) - } - return shardResult, numShardEmptyErrs, numErrs -} - -func (s *commitLogSource) mergeSeries( - nsCtx namespace.Context, - snapshotData result.DatabaseSeriesBlocks, - unmergedCommitlogBlocks metadataAndEncodersByTime, - blocksPool block.DatabaseBlockPool, - multiReaderIteratorPool encoding.MultiReaderIteratorPool, - segmentReaderPool xio.SegmentReaderPool, - encoderPool encoding.EncoderPool, - blockSize time.Duration, - blopts block.Options, -) (block.DatabaseSeriesBlocks, int, int) { - var seriesBlocks block.DatabaseSeriesBlocks - var numEmptyErrs int - var numErrs int - - for startNano, encoders := range unmergedCommitlogBlocks.encoders { - var ( - start = startNano.ToTime() - snapshotBlock block.DatabaseBlock - hasSnapshotBlock bool - ) - - if snapshotData.Blocks != nil { - snapshotBlock, hasSnapshotBlock = snapshotData.Blocks.BlockAt(start) - } - - if !hasSnapshotBlock { - // Make sure snapshotBlock is nil if it does not exist. - snapshotBlock = nil - } - - // Closes encoders and snapshotBlock by calling Discard() on each. - readers, err := newIOReadersFromEncodersAndBlock( - segmentReaderPool, encoders, snapshotBlock) - if err != nil { - numErrs++ - continue - } - - iter := multiReaderIteratorPool.Get() - iter.Reset(readers, time.Time{}, 0, nsCtx.Schema) - - enc := encoderPool.Get() - enc.Reset(start, blopts.DatabaseBlockAllocSize(), nsCtx.Schema) - for iter.Next() { - dp, unit, annotation := iter.Current() - encodeErr := enc.Encode(dp, unit, annotation) - if encodeErr != nil { - err = encodeErr - numErrs++ - break - } - } - - if iterErr := iter.Err(); iterErr != nil { - if err == nil { - err = iter.Err() - } - numErrs++ - } - - // Automatically returns iter to the pool - iter.Close() - readers.close() - if hasSnapshotBlock { - // Block is already closed, but we need to remove from the Blocks - // to prevent a double free when we call Blocks.Close() later. - snapshotData.Blocks.RemoveBlockAt(start) - } - - if err != nil { - continue - } - - pooledBlock := blocksPool.Get() - pooledBlock.Reset(start, blockSize, enc.Discard(), nsCtx) - if seriesBlocks == nil { - seriesBlocks = block.NewDatabaseSeriesBlocks(len(unmergedCommitlogBlocks.encoders)) - } - seriesBlocks.AddBlock(pooledBlock) - } - - if snapshotData.Blocks != nil { - allSnapshotBlocks := snapshotData.Blocks.AllBlocks() - for startNano, snapshotBlock := range snapshotData.Blocks.AllBlocks() { - if seriesBlocks == nil { - seriesBlocks = block.NewDatabaseSeriesBlocks(len(allSnapshotBlocks)) - } - _, ok := seriesBlocks.BlockAt(startNano.ToTime()) - if ok { - // Should never happen because we would have called - // Blocks.RemoveBlockAt() above. - iOpts := s.opts.CommitLogOptions().InstrumentOptions() - instrument.EmitAndLogInvariantViolation(iOpts, func(l *zap.Logger) { - l.Error("tried to merge block that should have been removed", - zap.Time("blockStart", startNano.ToTime())) - }) - continue - } - - seriesBlocks.AddBlock(snapshotBlock) - } - } - return seriesBlocks, numEmptyErrs, numErrs -} - -func (s *commitLogSource) findHighestShard(shardsTimeRanges result.ShardTimeRanges) uint32 { - var max uint32 - for shard := range shardsTimeRanges { - if shard > max { - max = shard - } - } - return max -} - -func (s *commitLogSource) logEncodingOutcome(workerErrs []int, iter commitlog.Iterator) { - errSum := 0 - for _, numErrs := range workerErrs { - errSum += numErrs +func (s *commitLogSource) logAccumulateOutcome( + workers []*accumulateWorker, + iter commitlog.Iterator, +) { + errs := 0 + for _, worker := range workers { + errs += worker.numErrors } - if errSum > 0 { - s.log.Error("error bootstrapping from commit log", zap.Int("block encode errors", errSum)) + if errs > 0 { + s.log.Error("error bootstrapping from commit log", zap.Int("accmulateErrors", errs)) } if err := iter.Err(); err != nil { s.log.Error("error reading commit log", zap.Error(err)) } } -func (s *commitLogSource) logMergeShardsOutcome(shardErrs []int, shardEmptyErrs []int) { - errSum := 0 - for _, numErrs := range shardErrs { - errSum += numErrs - } - if errSum > 0 { - s.log.Error("error bootstrapping from commit log", zap.Int("merge out of order errors", errSum)) - } - - emptyErrSum := 0 - for _, numEmptyErr := range shardEmptyErrs { - emptyErrSum += numEmptyErr - } - if emptyErrSum > 0 { - s.log.Error("error bootstrapping from commit log", zap.Int("empty unmerged blocks errors", emptyErrSum)) - } -} - -func (s *commitLogSource) AvailableIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.ShardTimeRanges, error) { - return s.availability(ns, shardsTimeRanges, runOpts) -} - -func (s *commitLogSource) ReadIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - if !ns.Options().IndexOptions().Enabled() { - return result.NewIndexBootstrapResult(), errIndexingNotEnableForNamespace - } - - if shardsTimeRanges.IsEmpty() { - return result.NewIndexBootstrapResult(), nil - } - - var ( - // Emit bootstrapping gauge for duration of ReadIndex - doneReadingIndex = s.metrics.index.emitBootstrapping() - encounteredCorruptData = false - fsOpts = s.opts.CommitLogOptions().FilesystemOptions() - filePathPrefix = fsOpts.FilePathPrefix() - ) - defer doneReadingIndex() - - // Determine which snapshot files are available. - snapshotFilesByShard, err := s.snapshotFilesByShard( - ns.ID(), filePathPrefix, shardsTimeRanges) - if err != nil { - return nil, err - } - - var ( - highestShard = s.findHighestShard(shardsTimeRanges) - // +1 so we can use the shard number as an index throughout without constantly - // remembering to subtract 1 to convert to zero-based indexing. - numShards = highestShard + 1 - // Convert the map to a slice for faster lookups - bootstrapRangesByShard = make([]xtime.Ranges, numShards) - ) - - for shard, ranges := range shardsTimeRanges { - bootstrapRangesByShard[shard] = ranges - } - - var ( - indexResult = result.NewIndexBootstrapResult() - indexResults = indexResult.IndexResults() - indexOptions = ns.Options().IndexOptions() - indexBlockSize = indexOptions.BlockSize() - resultOptions = s.opts.ResultOptions() - blockSize = ns.Options().RetentionOptions().BlockSize() - ) - - // Determine which commit log files we need to read based on which snapshot - // snapshot files are available. - readCommitLogPredicate, mostRecentCompleteSnapshotByBlockShard, err := s.newReadCommitlogPredAndMostRecentSnapshotByBlockShard( - ns, shardsTimeRanges, snapshotFilesByShard) - if err != nil { - return nil, err - } - - var ( - readSeriesPredicate = newReadSeriesPredicate(ns) - iterOpts = commitlog.IteratorOpts{ - CommitLogOptions: s.opts.CommitLogOptions(), - FileFilterPredicate: readCommitLogPredicate, - SeriesFilterPredicate: readSeriesPredicate, - } - ) - - // Start by reading any available snapshot files. - for shard, tr := range shardsTimeRanges { - shardResult, err := s.bootstrapShardSnapshots( - ns, shard, true, tr, blockSize, snapshotFilesByShard[shard], - mostRecentCompleteSnapshotByBlockShard) - if err != nil { - return nil, err - } - - // Bootstrap any series we got from the snapshot files into the index. - for _, val := range shardResult.AllSeries().Iter() { - id := val.Key() - val := val.Value() - for block := range val.Blocks.AllBlocks() { - s.maybeAddToIndex( - id, val.Tags, shard, highestShard, block.ToTime(), bootstrapRangesByShard, - indexResults, indexOptions, indexBlockSize, resultOptions) - } - } - } - - // Next, read all of the data from the commit log files that wasn't covered - // by the snapshot files. - iter, corruptFiles, err := s.newIteratorFn(iterOpts) - if err != nil { - return nil, fmt.Errorf("unable to create commit log iterator: %v", err) - } - if len(corruptFiles) > 0 { - s.logAndEmitCorruptFiles(corruptFiles, false) - encounteredCorruptData = true - } - - defer iter.Close() - - for iter.Next() { - series, dp, _, _ := iter.Current() - - s.maybeAddToIndex( - series.ID, series.Tags, series.Shard, highestShard, dp.Timestamp, bootstrapRangesByShard, - indexResults, indexOptions, indexBlockSize, resultOptions) - } - - if iterErr := iter.Err(); iterErr != nil { - // Log the error and mark that we encountered corrupt data, but don't - // return the error because we want to give the peers bootstrapper the - // opportunity to repair the data instead of failing the bootstrap - // altogether. - s.log.Error("error in commitlog iterator", zap.Error(iterErr)) - encounteredCorruptData = true - s.metrics.index.corruptCommitlogFile.Inc(1) - } - - // If all successful then we mark each index block as fulfilled - for _, block := range indexResult.IndexResults() { - blockRange := xtime.Range{ - Start: block.BlockStart(), - End: block.BlockStart().Add(indexOptions.BlockSize()), - } - fulfilled := result.ShardTimeRanges{} - for shard, timeRanges := range shardsTimeRanges { - iter := timeRanges.Iter() - for iter.Next() { - curr := iter.Value() - intersection, intersects := curr.Intersect(blockRange) - if intersects { - fulfilled[shard] = fulfilled[shard].AddRange(intersection) - } - } - } - // Now mark as much of the block that we fulfilled - err := indexResult.IndexResults().MarkFulfilled(blockRange.Start, - fulfilled, indexOptions) - if err != nil { - return nil, err - } - } - - shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled( - encounteredCorruptData, ns, shardsTimeRanges, opts) - if err != nil { - return nil, err - } - - if shouldReturnUnfulfilled { - indexResult.SetUnfulfilled(shardsTimeRanges) - } - return indexResult, nil -} - // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, we want to mark the entire range // as unfulfilled so the peers bootstrapper can attempt a repair, but keep @@ -1304,11 +943,18 @@ func (s *commitLogSource) ReadIndex( // bootstrapper is unable to satisfy the bootstrap because all peers are // down or if the commitlog contained data that the peers do not have. func (s commitLogSource) shouldReturnUnfulfilled( + workers []*accumulateWorker, encounteredCorruptData bool, - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, + initialTopologyState *topology.StateSnapshot, ) (bool, error) { + errs := 0 + for _, worker := range workers { + errs += worker.numErrors + } + if errs > 0 { + return true, fmt.Errorf("return unfulfilled: %d accumulate errors", errs) + } + if !s.opts.ReturnUnfulfilledForCorruptCommitLogFiles() { s.log.Info("returning not-unfulfilled: ReturnUnfulfilledForCorruptCommitLogFiles is false") return false, nil @@ -1319,65 +965,18 @@ func (s commitLogSource) shouldReturnUnfulfilled( return false, nil } - areShardsReplicated := s.areShardsReplicated( - ns, shardsTimeRanges, opts) - if !areShardsReplicated { + shardsReplicated := s.shardsReplicated(initialTopologyState) + if !shardsReplicated { s.log.Info("returning not-unfulfilled: replication is not enabled") } - - return areShardsReplicated, nil -} - -func (s commitLogSource) maybeAddToIndex( - id ident.ID, - tags ident.Tags, - shard uint32, - highestShard uint32, - blockStart time.Time, - bootstrapRangesByShard []xtime.Ranges, - indexResults result.IndexResults, - indexOptions namespace.IndexOptions, - indexBlockSize time.Duration, - resultOptions result.Options, -) error { - if !s.shouldIncludeInIndex( - shard, blockStart, highestShard, indexBlockSize, bootstrapRangesByShard) { - return nil - } - - segment, err := indexResults.GetOrAddSegment(blockStart, indexOptions, resultOptions) - if err != nil { - return err - } - - exists, err := segment.ContainsID(id.Bytes()) - if err != nil { - return err - } - if exists { - return nil - } - - // We can use the NoClone variant here because the IDs/Tags read from the commit log files - // by the ReadIndex() method won't be finalized because this code path doesn't finalize them. - d, err := convert.FromMetricNoClone(id, tags) - if err != nil { - return err - } - - _, err = segment.Insert(d) - return err + return shardsReplicated, nil } func (s *commitLogSource) logAndEmitCorruptFiles( - corruptFiles []commitlog.ErrorWithPath, isData bool) { + corruptFiles []commitlog.ErrorWithPath) { for _, f := range corruptFiles { s.log.Error("opting to skip commit log due to corruption", zap.String("error", f.Error())) - if isData { - s.metrics.data.corruptCommitlogFile.Inc(1) - } else { - s.metrics.index.corruptCommitlogFile.Inc(1) - } + s.metrics.corruptCommitlogFile.Inc(1) } } @@ -1447,16 +1046,9 @@ func (s *commitLogSource) availability( return availableShardTimeRanges, nil } -func (s *commitLogSource) areShardsReplicated( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, +func (s *commitLogSource) shardsReplicated( + initialTopologyState *topology.StateSnapshot, ) bool { - var ( - initialTopologyState = runOpts.InitialTopologyState() - majorityReplicas = initialTopologyState.MajorityReplicas - ) - // In any situation where we could actually stream data from our peers // the replication factor would be 2 or larger which means that the // value of majorityReplicas would be 2 or larger also. This heuristic can @@ -1464,95 +1056,22 @@ func (s *commitLogSource) areShardsReplicated( // cannot be used to determine what the actual replication factor is in all // situations because it can be ambiguous. For example, both R.F 2 and 3 will // have majority replica values of 2. + majorityReplicas := initialTopologyState.MajorityReplicas return majorityReplicas > 1 } -func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { - nsID := ns.ID() - return func(id ident.ID, namespace ident.ID) bool { - return nsID.Equal(namespace) - } -} - -type shardData struct { - series *Map - ranges xtime.Ranges -} - -type metadataAndEncodersByTime struct { - id ident.ID - tags ident.Tags - // int64 instead of time.Time because there is an optimized map access pattern - // for i64's - encoders map[xtime.UnixNano][]encoder -} - -// encoderArg contains all the information a worker go-routine needs to encode -// a data point as M3TSZ -type encoderArg struct { - series ts.Series - dp ts.Datapoint - unit xtime.Unit - annotation ts.Annotation - blockStart time.Time -} - -type ioReaders []xio.SegmentReader - -func newIOReadersFromEncodersAndBlock( - segmentReaderPool xio.SegmentReaderPool, - encoders []encoder, - dbBlock block.DatabaseBlock, -) (ioReaders, error) { - numReaders := len(encoders) - if dbBlock != nil { - numReaders++ - } - - readers := make(ioReaders, 0, numReaders) - if dbBlock != nil { - blockSegment := dbBlock.Discard() - blockReader := segmentReaderPool.Get() - blockReader.Reset(blockSegment) - readers = append(readers, blockReader) - } - - for _, encoder := range encoders { - segmentReader := segmentReaderPool.Get() - segmentReader.Reset(encoder.enc.Discard()) - readers = append(readers, segmentReader) - } - - return readers, nil -} - -func (ir ioReaders) close() { - for _, r := range ir { - r.(xio.SegmentReader).Finalize() - } -} - -type commitLogSourceDataAndIndexMetrics struct { - data commitLogSourceMetrics - index commitLogSourceMetrics -} - -func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDataAndIndexMetrics { - return commitLogSourceDataAndIndexMetrics{ - data: newCommitLogSourceMetrics(scope.Tagged(map[string]string{ - "source_type": "data", - })), - index: newCommitLogSourceMetrics(scope.Tagged(map[string]string{ - "source_type": "index", - })), - } -} - type commitLogSourceMetrics struct { corruptCommitlogFile tally.Counter bootstrapping tally.Gauge } +func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { + return commitLogSourceMetrics{ + corruptCommitlogFile: scope.SubScope("commitlog").Counter("corrupt"), + bootstrapping: scope.SubScope("status").Gauge("bootstrapping"), + } +} + type gaugeLoopCloserFn func() func (m commitLogSourceMetrics) emitBootstrapping() gaugeLoopCloserFn { @@ -1572,10 +1091,3 @@ func (m commitLogSourceMetrics) emitBootstrapping() gaugeLoopCloserFn { return func() { close(doneCh) } } - -func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { - return commitLogSourceMetrics{ - corruptCommitlogFile: scope.SubScope("commitlog").Counter("corrupt"), - bootstrapping: scope.SubScope("status").Gauge("bootstrapping"), - } -} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_proto_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_proto_test.go index ca7fd4f4b0..14b6e15ec0 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_proto_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_proto_test.go @@ -22,9 +22,10 @@ package commitlog import ( "testing" - "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/dbnode/testdata/prototest" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/testdata/prototest" + "github.com/m3db/m3/src/x/ident" "github.com/stretchr/testify/require" ) @@ -56,7 +57,7 @@ func testProtoNsMetadata(t *testing.T) namespace.Metadata { return md } -func setProtoAnnotation(value []testValue) []testValue { +func setProtoAnnotation(value testValues) testValues { protoIter := prototest.NewProtoMessageIterator(testProtoMessages) for i := 0; i < len(value); i++ { value[i].v = 0 @@ -82,4 +83,3 @@ func TestProtoItMergesSnapshotsAndCommitLogs(t *testing.T) { md := testProtoNsMetadata(t) testItMergesSnapshotsAndCommitLogs(t, opts, md, setProtoAnnotation) } - diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go index 744ccaf3c6..4f23a5896e 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go @@ -21,22 +21,19 @@ package commitlog import ( - "fmt" + "errors" "io" - "reflect" - "sort" "testing" "time" "github.com/m3db/m3/src/dbnode/digest" - "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" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" + "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/topology" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/checked" @@ -76,20 +73,29 @@ func TestReadEmpty(t *testing.T) { opts := testDefaultOpts src := newCommitLogSource(opts, fs.Inspection{}) + md := testNsMetadata(t) + target := result.ShardTimeRanges{} + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, target, md) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) - res, err := src.ReadData(testNsMetadata(t), result.ShardTimeRanges{}, - testDefaultRunOpts) + values, err := tester.EnsureDumpAllForNamespace(md) require.NoError(t, err) - require.Equal(t, 0, len(res.ShardResults())) - require.True(t, res.Unfulfilled().IsEmpty()) + require.Equal(t, 0, len(values)) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func TestReadErrorOnNewIteratorError(t *testing.T) { opts := testDefaultOpts src := newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { - return nil, nil, fmt.Errorf("an error") + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + return nil, nil, errors.New("an error") } ranges := xtime.Ranges{} @@ -97,10 +103,17 @@ func TestReadErrorOnNewIteratorError(t *testing.T) { Start: time.Now(), End: time.Now().Add(time.Hour), }) - res, err := src.ReadData(testNsMetadata(t), result.ShardTimeRanges{0: ranges}, - testDefaultRunOpts) + + md := testNsMetadata(t) + target := result.ShardTimeRanges{0: ranges} + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, target, md) + defer tester.Finish() + + res, err := src.Read(tester.Namespaces) require.Error(t, err) - require.Nil(t, res) + require.Nil(t, res.Results) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func TestReadOrderedValues(t *testing.T) { @@ -132,7 +145,7 @@ func testReadOrderedValues(t *testing.T, opts Options, md namespace.Metadata, se bar := ts.Series{Namespace: nsCtx.ID, Shard: 1, ID: ident.StringID("bar")} baz := ts.Series{Namespace: nsCtx.ID, Shard: 2, ID: ident.StringID("baz")} - values := []testValue{ + values := testValues{ {foo, start, 1.0, xtime.Second, nil}, {foo, start.Add(1 * time.Minute), 2.0, xtime.Second, nil}, {bar, start.Add(2 * time.Minute), 1.0, xtime.Second, nil}, @@ -144,18 +157,23 @@ func testReadOrderedValues(t *testing.T, opts Options, md namespace.Metadata, se values = setAnn(values) } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} - res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 2, len(res.ShardResults())) - require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(nsCtx, - values[:4], blockSize, res.ShardResults(), opts)) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + read := tester.EnsureDumpWritesForNamespace(md) + require.Equal(t, 2, len(read)) + enforceValuesAreCorrect(t, values[:4], read) + tester.EnsureNoLoadedBlocks() } func TestReadUnorderedValues(t *testing.T) { @@ -184,7 +202,7 @@ func testReadUnorderedValues(t *testing.T, opts Options, md namespace.Metadata, foo := ts.Series{Namespace: nsCtx.ID, Shard: 0, ID: ident.StringID("foo")} - values := []testValue{ + values := testValues{ {foo, start.Add(10 * time.Minute), 1.0, xtime.Second, nil}, {foo, start.Add(1 * time.Minute), 2.0, xtime.Second, nil}, {foo, start.Add(2 * time.Minute), 3.0, xtime.Second, nil}, @@ -195,32 +213,35 @@ func testReadUnorderedValues(t *testing.T, opts Options, md namespace.Metadata, values = setAnn(values) } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} - res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 1, len(res.ShardResults())) - require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(nsCtx, - values, blockSize, res.ShardResults(), opts)) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + read := tester.EnsureDumpWritesForNamespace(md) + require.Equal(t, 1, len(read)) + enforceValuesAreCorrect(t, values, read) + tester.EnsureNoLoadedBlocks() } +// TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex was added as a +// regression test to make sure that the commit log bootstrapper does not make +// any assumptions about series having a unique index because that only holds +// for the duration that an M3DB node is on, but commit log files can span +// multiple M3DB processes which means that unique indexes could be re-used +// for multiple different series. func TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T) { opts := testDefaultOpts md := testNsMetadata(t) - testReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t, opts, md, nil) -} -// TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex was added as a regression test to make -// sure that the commit log bootstrapper does not make any assumptions about series having a unique -// unique index because that only holds for the duration that an M3DB node is on, but commit log -// files can span multiple M3DB processes which means that unique indexes could be re-used for multiple -// different series. -func testReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T, opts Options, md namespace.Metadata, setAnn setAnnotation) { nsCtx := namespace.NewContextFrom(md) src := newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) @@ -242,26 +263,28 @@ func testReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T, opts O bar := ts.Series{ Namespace: nsCtx.ID, Shard: 0, ID: ident.StringID("bar"), UniqueIndex: 0} - values := []testValue{ + values := testValues{ {foo, start, 1.0, xtime.Second, nil}, {bar, start, 2.0, xtime.Second, nil}, } - if setAnn != nil { - values = setAnn(values) - } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} - res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 1, len(res.ShardResults())) - require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(nsCtx, - values, blockSize, res.ShardResults(), opts)) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + read := tester.EnsureDumpWritesForNamespace(md) + require.Equal(t, 2, len(read)) + enforceValuesAreCorrect(t, values, read) + tester.EnsureNoLoadedBlocks() } func TestReadTrimsToRanges(t *testing.T) { @@ -289,29 +312,34 @@ func testReadTrimsToRanges(t *testing.T, opts Options, md namespace.Metadata, se }) foo := ts.Series{Namespace: nsCtx.ID, Shard: 0, ID: ident.StringID("foo")} - - values := []testValue{ + values := testValues{ {foo, start.Add(-1 * time.Minute), 1.0, xtime.Nanosecond, nil}, {foo, start, 2.0, xtime.Nanosecond, nil}, {foo, start.Add(1 * time.Minute), 3.0, xtime.Nanosecond, nil}, {foo, end.Truncate(blockSize).Add(blockSize).Add(time.Nanosecond), 4.0, xtime.Nanosecond, nil}, } + if setAnn != nil { values = setAnn(values) } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} - res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 1, len(res.ShardResults())) - require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(nsCtx, - values[1:3], blockSize, res.ShardResults(), opts)) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + read := tester.EnsureDumpWritesForNamespace(md) + require.Equal(t, 1, len(read)) + enforceValuesAreCorrect(t, values[1:3], read) + tester.EnsureNoLoadedBlocks() } func TestItMergesSnapshotsAndCommitLogs(t *testing.T) { @@ -321,7 +349,8 @@ func TestItMergesSnapshotsAndCommitLogs(t *testing.T) { testItMergesSnapshotsAndCommitLogs(t, opts, md, nil) } -func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, md namespace.Metadata, setAnn setAnnotation) { +func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, + md namespace.Metadata, setAnn setAnnotation) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -335,7 +364,7 @@ func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, md namespace ranges = xtime.Ranges{} foo = ts.Series{Namespace: nsCtx.ID, Shard: 0, ID: ident.StringID("foo")} - commitLogValues = []testValue{ + commitLogValues = testValues{ {foo, start.Add(2 * time.Minute), 1.0, xtime.Nanosecond, nil}, {foo, start.Add(3 * time.Minute), 2.0, xtime.Nanosecond, nil}, {foo, start.Add(4 * time.Minute), 3.0, xtime.Nanosecond, nil}, @@ -357,10 +386,17 @@ func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, md namespace End: end, }) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(commitLogValues, nil), nil, nil } - src.snapshotFilesFn = func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) { + + src.snapshotFilesFn = func( + filePathPrefix string, + namespace ident.ID, + shard uint32, + ) (fs.FileSetFilesSlice, error) { return fs.FileSetFilesSlice{ fs.FileSetFile{ ID: fs.FileSetFileIdentifier{ @@ -390,14 +426,15 @@ func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, md namespace mockReader.EXPECT().Entries().Return(1).AnyTimes() mockReader.EXPECT().Close().Return(nil).AnyTimes() - snapshotValues := []testValue{ + snapshotValues := testValues{ {foo, start.Add(1 * time.Minute), 1.0, xtime.Nanosecond, nil}, } if setAnn != nil { snapshotValues = setAnn(snapshotValues) } - encoder := opts.ResultOptions().DatabaseBlockOptions().EncoderPool().Get() + encoderPool := opts.ResultOptions().DatabaseBlockOptions().EncoderPool() + encoder := encoderPool.Get() encoder.Reset(snapshotValues[0].t, 10, nsCtx.Schema) for _, value := range snapshotValues { dp := ts.Datapoint{ @@ -429,24 +466,40 @@ func testItMergesSnapshotsAndCommitLogs(t *testing.T, opts Options, md namespace ) mockReader.EXPECT().Read().Return(nil, nil, nil, uint32(0), io.EOF) - src.newReaderFn = func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) { + src.newReaderFn = func( + bytesPool pool.CheckedBytesPool, + opts fs.Options, + ) (fs.DataFileSetReader, error) { return mockReader, nil } targetRanges := result.ShardTimeRanges{0: ranges} - res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 1, len(res.ShardResults())) - require.Equal(t, 0, len(res.Unfulfilled())) - expectedValues := append([]testValue{}, snapshotValues...) - expectedValues = append(expectedValues, commitLogValues[0:3]...) - - require.NoError(t, verifyShardResultsAreCorrect(nsCtx, - expectedValues, blockSize, res.ShardResults(), opts)) + tester := bootstrap.BuildNamespacesTesterWithReaderIteratorPool( + t, + testDefaultRunOpts, + targetRanges, + opts.ResultOptions().DatabaseBlockOptions().MultiReaderIteratorPool(), + md, + ) + + defer tester.Finish() + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + // NB: this case is a little tricky in that this test is combining writes + // that come through both the `LoadBlock()` methods (for snapshotted data), + // and the `Write()` method (for data that is not snapshotted) into the + // namespace data accumulator. Thus writes into the accumulated series should + // be verified against both of these methods. + read := tester.EnsureDumpWritesForNamespace(md) + require.Equal(t, 1, len(read)) + enforceValuesAreCorrect(t, commitLogValues[0:3], read) + + read = tester.EnsureDumpLoadedBlocksForNamespace(md) + enforceValuesAreCorrect(t, snapshotValues, read) } -type setAnnotation func([]testValue) []testValue +type setAnnotation func(testValues) testValues type annotationEqual func([]byte, []byte) bool type testValue struct { @@ -457,287 +510,53 @@ type testValue struct { a ts.Annotation } -type seriesShardResultBlock struct { - encoder encoding.Encoder -} - -type seriesShardResult struct { - blocks map[xtime.UnixNano]*seriesShardResultBlock - result block.DatabaseSeriesBlocks -} - -func verifyShardResultsAreCorrect( - nsCtx namespace.Context, - values []testValue, - blockSize time.Duration, - actual result.ShardResults, - opts Options, -) error { - if actual == nil { - if len(values) == 0 { - return nil - } - - return fmt.Errorf( - "shard result is nil, but expected: %d values", len(values)) - } - // First create what result should be constructed for test values - expected, err := createExpectedShardResult(nsCtx, values, blockSize, opts) - if err != nil { - return err - } - - // Assert the values - if len(expected) != len(actual) { - return fmt.Errorf( - "number of shards do not match, expected: %d, but got: %d", - len(expected), len(actual), - ) - } - - for shard, expectedResult := range expected { - actualResult, ok := actual[shard] - if !ok { - return fmt.Errorf("shard: %d present in expected, but not actual", shard) +type testValues []testValue + +func (v testValues) toDecodedBlockMap() bootstrap.DecodedBlockMap { + blockMap := make(bootstrap.DecodedBlockMap, len(v)) + for _, bl := range v { + id := bl.s.ID.String() + val := series.DecodedTestValue{ + Timestamp: bl.t, + Value: bl.v, + Unit: bl.u, + Annotation: bl.a, } - err = verifyShardResultsAreEqual(nsCtx, opts, shard, actualResult, expectedResult) - if err != nil { - return err + if values, found := blockMap[id]; found { + blockMap[id] = append(values, val) + } else { + blockMap[id] = bootstrap.DecodedValues{val} } } - return nil -} -func createExpectedShardResult( - nsCtx namespace.Context, - values []testValue, - blockSize time.Duration, - opts Options, -) (result.ShardResults, error) { - bopts := opts.ResultOptions() - blopts := bopts.DatabaseBlockOptions() - - expected := result.ShardResults{} - - // Sort before iterating to ensure encoding to blocks is correct order - sort.Stable(testValuesByTime(values)) - - allResults := make(map[string]*seriesShardResult) - for _, v := range values { - shardResult, ok := expected[v.s.Shard] - if !ok { - shardResult = result.NewShardResult(0, bopts) - expected[v.s.Shard] = shardResult - } - _, exists := shardResult.AllSeries().Get(v.s.ID) - if !exists { - // Trigger blocks to be created for series - shardResult.AddSeries(v.s.ID, v.s.Tags, nil) - } - - series, _ := shardResult.AllSeries().Get(v.s.ID) - blocks := series.Blocks - blockStart := v.t.Truncate(blockSize) - - r, ok := allResults[v.s.ID.String()] - if !ok { - r = &seriesShardResult{ - blocks: make(map[xtime.UnixNano]*seriesShardResultBlock), - result: blocks, - } - allResults[v.s.ID.String()] = r - } - - b, ok := r.blocks[xtime.ToUnixNano(blockStart)] - if !ok { - encoder := bopts.DatabaseBlockOptions().EncoderPool().Get() - encoder.Reset(v.t, 0, nsCtx.Schema) - b = &seriesShardResultBlock{ - encoder: encoder, - } - r.blocks[xtime.ToUnixNano(blockStart)] = b - } - - err := b.encoder.Encode(ts.Datapoint{ - Timestamp: v.t, - Value: v.v, - }, v.u, v.a) - if err != nil { - return expected, err - } - } - - for _, r := range allResults { - for start, blockResult := range r.blocks { - enc := blockResult.encoder - bl := block.NewDatabaseBlock(start.ToTime(), blockSize, enc.Discard(), blopts, nsCtx) - if r.result != nil { - r.result.AddBlock(bl) - } - } - - } - - return expected, nil + return blockMap } -func verifyShardResultsAreEqual(nsCtx namespace.Context, opts Options, shard uint32, actualResult, expectedResult result.ShardResult) error { - expectedSeries := expectedResult.AllSeries() - actualSeries := actualResult.AllSeries() - if expectedSeries.Len() != actualSeries.Len() { - return fmt.Errorf( - "different number of series for shard: %v . expected: %d , actual: %d", - shard, - expectedSeries.Len(), - actualSeries.Len(), - ) - } - - for _, entry := range expectedSeries.Iter() { - expectedID, expectedBlocks := entry.Key(), entry.Value() - actualBlocks, ok := actualSeries.Get(expectedID) - if !ok { - return fmt.Errorf("series: %v present in expected but not actual", expectedID) - } - - if !expectedBlocks.Tags.Equal(actualBlocks.Tags) { - return fmt.Errorf( - "series: %v present in expected and actual, but tags do not match", expectedID) - } - - expectedAllBlocks := expectedBlocks.Blocks.AllBlocks() - actualAllBlocks := actualBlocks.Blocks.AllBlocks() - if len(expectedAllBlocks) != len(actualAllBlocks) { - return fmt.Errorf( - "number of expected blocks: %d does not match number of actual blocks: %d for series: %s", - len(expectedAllBlocks), - len(actualAllBlocks), - expectedID, - ) - } - - err := verifyBlocksAreEqual(nsCtx, opts, expectedAllBlocks, actualAllBlocks) - if err != nil { - return err - } - } - - return nil +func enforceValuesAreCorrect( + t *testing.T, + values testValues, + actual bootstrap.DecodedBlockMap, +) { + require.NoError(t, verifyValuesAreCorrect(values, actual)) } -func verifyBlocksAreEqual(nsCtx namespace.Context, opts Options, expectedAllBlocks, actualAllBlocks map[xtime.UnixNano]block.DatabaseBlock) error { - blopts := opts.ResultOptions().DatabaseBlockOptions() - for start, expectedBlock := range expectedAllBlocks { - actualBlock, ok := actualAllBlocks[start] - if !ok { - return fmt.Errorf("Expected block for start time: %v", start) - } - - ctx := blopts.ContextPool().Get() - defer ctx.Close() - - expectedStream, expectedStreamErr := expectedBlock.Stream(ctx) - if expectedStreamErr != nil { - return fmt.Errorf("err creating expected stream: %s", expectedStreamErr.Error()) - } - - actualStream, actualStreamErr := actualBlock.Stream(ctx) - if actualStreamErr != nil { - return fmt.Errorf("err creating actual stream: %s", actualStreamErr.Error()) - } - - readerIteratorPool := blopts.ReaderIteratorPool() - - expectedIter := readerIteratorPool.Get() - expectedIter.Reset(expectedStream, nsCtx.Schema) - defer expectedIter.Close() - - actualIter := readerIteratorPool.Get() - actualIter.Reset(actualStream, nsCtx.Schema) - defer actualIter.Close() - - for { - expectedNext := expectedIter.Next() - actualNext := actualIter.Next() - if !expectedNext && !actualNext { - break - } - - if !(expectedNext && actualNext) { - return fmt.Errorf( - "err: expectedNext was: %v, but actualNext was: %v", - expectedNext, - actualNext, - ) - } - - expectedValue, expectedUnit, expectedAnnotation := expectedIter.Current() - actualValue, actualUnit, actualAnnotation := actualIter.Current() - - if expectedValue.Timestamp != actualValue.Timestamp { - return fmt.Errorf( - "expectedValue.Timestamp was: %v, but actualValue.Timestamp was: %v", - expectedValue.Timestamp, - actualValue.Timestamp, - ) - } - - if expectedValue.Value != actualValue.Value { - return fmt.Errorf( - "expectedValue.Value was: %v, but actualValue.Value was: %v", - expectedValue.Value, - actualValue.Value, - ) - } - - if expectedUnit != actualUnit { - return fmt.Errorf( - "expectedUnit was: %v, but actualUnit was: %v", - expectedUnit, - actualUnit, - ) - } - - if nsCtx.Schema == nil { - if !reflect.DeepEqual(expectedAnnotation, actualAnnotation) { - return fmt.Errorf( - "expectedAnnotation was: %v, but actualAnnotation was: %v", - expectedAnnotation, - actualAnnotation, - ) - } - } else { - if !testProtoEqual(expectedAnnotation, actualAnnotation) { - return fmt.Errorf( - "expectedAnnotation was: %v, but actualAnnotation was: %v", - expectedAnnotation, - actualAnnotation, - ) - } - } - } - } - - return nil +func verifyValuesAreCorrect( + values testValues, + actual bootstrap.DecodedBlockMap, +) error { + expected := values.toDecodedBlockMap() + return expected.VerifyEquals(actual) } type testCommitLogIterator struct { - values []testValue + values testValues idx int err error closed bool } -type testValuesByTime []testValue - -func (v testValuesByTime) Len() int { return len(v) } -func (v testValuesByTime) Swap(i, j int) { v[i], v[j] = v[j], v[i] } -func (v testValuesByTime) Less(i, j int) bool { - return v[i].t.Before(v[j].t) -} - -func newTestCommitLogIterator(values []testValue, err error) *testCommitLogIterator { +func newTestCommitLogIterator(values testValues, err error) *testCommitLogIterator { return &testCommitLogIterator{values: values, idx: -1, err: err} } @@ -746,13 +565,22 @@ func (i *testCommitLogIterator) Next() bool { return i.idx < len(i.values) } -func (i *testCommitLogIterator) Current() (ts.Series, ts.Datapoint, xtime.Unit, ts.Annotation) { +func (i *testCommitLogIterator) Current() commitlog.LogEntry { idx := i.idx if idx == -1 { idx = 0 } v := i.values[idx] - return v.s, ts.Datapoint{Timestamp: v.t, Value: v.v}, v.u, v.a + return commitlog.LogEntry{ + Series: v.s, + Datapoint: ts.Datapoint{Timestamp: v.t, Value: v.v}, + Unit: v.u, + Annotation: v.a, + Metadata: commitlog.LogEntryMetadata{ + FileReadID: uint64(idx) + 1, + SeriesUniqueIndex: v.s.UniqueIndex, + }, + } } func (i *testCommitLogIterator) Err() error { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go index 3fb2cb8319..b006595faf 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go @@ -25,12 +25,15 @@ import ( "testing" "time" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + "github.com/m3db/m3/src/x/serialize" xtime "github.com/m3db/m3/src/x/time" "github.com/stretchr/testify/require" @@ -38,15 +41,21 @@ import ( var namespaceOptions = namespace.NewOptions() -func TestBootstrapIndex(t *testing.T) { - testBootstrapIndex(t, false) -} - -func TestBootstrapIndexAfterBootstrapData(t *testing.T) { - testBootstrapIndex(t, true) +func toEncodedBytes( + t *testing.T, + pool serialize.TagEncoderPool, + tags ...ident.Tag, +) []byte { + encoder := pool.Get() + seriesTags := ident.NewTags(tags...) + err := encoder.Encode(ident.NewTagsIterator(seriesTags)) + require.NoError(t, err) + data, ok := encoder.Data() + require.True(t, ok) + return data.Bytes() } -func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { +func TestBootstrapIndex(t *testing.T) { var ( opts = testDefaultOpts src = newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) @@ -79,33 +88,46 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { now := time.Now() start := now.Truncate(indexBlockSize) - fooTags := ident.NewTags(ident.StringTag("city", "ny"), ident.StringTag("conference", "monitoroma")) - barTags := ident.NewTags(ident.StringTag("city", "sf")) - bazTags := ident.NewTags(ident.StringTag("city", "oakland")) + testTagEncodingPool := serialize. + NewTagEncoderPool(serialize.NewTagEncoderOptions(), + pool.NewObjectPoolOptions().SetSize(1)) + testTagEncodingPool.Init() + + fooTags := toEncodedBytes(t, testTagEncodingPool, + ident.StringTag("city", "ny"), + ident.StringTag("conference", "monitoroma"), + ) + + barTags := toEncodedBytes(t, testTagEncodingPool, + ident.StringTag("city", "sf")) + bazTags := toEncodedBytes(t, testTagEncodingPool, + ident.StringTag("city", "oakland")) shardn := func(n int) uint32 { return uint32(n) } - foo := ts.Series{UniqueIndex: 0, Namespace: testNamespaceID, Shard: shardn(0), ID: ident.StringID("foo"), Tags: fooTags} - bar := ts.Series{UniqueIndex: 1, Namespace: testNamespaceID, Shard: shardn(0), ID: ident.StringID("bar"), Tags: barTags} - baz := ts.Series{UniqueIndex: 2, Namespace: testNamespaceID, Shard: shardn(5), ID: ident.StringID("baz"), Tags: bazTags} + foo := ts.Series{UniqueIndex: 0, Namespace: testNamespaceID, Shard: shardn(0), + ID: ident.StringID("foo"), EncodedTags: fooTags} + bar := ts.Series{UniqueIndex: 1, Namespace: testNamespaceID, Shard: shardn(0), + ID: ident.StringID("bar"), EncodedTags: barTags} + baz := ts.Series{UniqueIndex: 2, Namespace: testNamespaceID, Shard: shardn(5), + ID: ident.StringID("baz"), EncodedTags: bazTags} // Make sure we can handle series that don't have tags. - untagged := ts.Series{UniqueIndex: 3, Namespace: testNamespaceID, Shard: shardn(5), ID: ident.StringID("untagged"), Tags: ident.Tags{}} + untagged := ts.Series{UniqueIndex: 3, Namespace: testNamespaceID, + Shard: shardn(5), ID: ident.StringID("untagged"), Tags: ident.Tags{}} // Make sure we skip series that are not within the bootstrap range. - outOfRange := ts.Series{UniqueIndex: 4, Namespace: testNamespaceID, Shard: shardn(3), ID: ident.StringID("outOfRange"), Tags: ident.Tags{}} + outOfRange := ts.Series{UniqueIndex: 4, Namespace: testNamespaceID, + Shard: shardn(3), ID: ident.StringID("outOfRange"), Tags: ident.Tags{}} // Make sure we skip and dont panic on writes for shards that are higher than the maximum we're trying to bootstrap. - shardTooHigh := ts.Series{UniqueIndex: 5, Namespace: testNamespaceID, Shard: shardn(100), ID: ident.StringID("shardTooHigh"), Tags: ident.Tags{}} + shardTooHigh := ts.Series{UniqueIndex: 5, Namespace: testNamespaceID, + Shard: shardn(100), ID: ident.StringID("shardTooHigh"), Tags: ident.Tags{}} // Make sure we skip series for shards that have no requested bootstrap ranges. The shard for this write needs // to be less than the highest shard we actually plan to bootstrap. - noShardBootstrapRange := ts.Series{UniqueIndex: 6, Namespace: testNamespaceID, Shard: shardn(4), ID: ident.StringID("noShardBootstrapRange"), Tags: ident.Tags{}} + noShardBootstrapRange := ts.Series{UniqueIndex: 6, Namespace: testNamespaceID, + Shard: shardn(4), ID: ident.StringID("noShardBootstrapRange"), Tags: ident.Tags{}} // Make sure it handles multiple namespaces - someOtherNamespace := ts.Series{UniqueIndex: 7, Namespace: testNamespaceID2, Shard: shardn(0), ID: ident.StringID("someOtherNamespace"), Tags: ident.Tags{}} + someOtherNamespace := ts.Series{UniqueIndex: 7, Namespace: testNamespaceID2, + Shard: shardn(0), ID: ident.StringID("series_OtherNamespace"), Tags: ident.Tags{}} - seriesNotToExpect := map[string]struct{}{ - outOfRange.ID.String(): struct{}{}, - shardTooHigh.ID.String(): struct{}{}, - noShardBootstrapRange.ID.String(): struct{}{}, - } - - values := []testValue{ + valuesNs := testValues{ {foo, start, 1.0, xtime.Second, nil}, {foo, start, 2.0, xtime.Second, nil}, {foo, start.Add(dataBlockSize), 3.0, xtime.Second, nil}, @@ -114,13 +136,23 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { {baz, start.Add(2 * dataBlockSize), 1.0, xtime.Second, nil}, {baz, start.Add(2 * dataBlockSize), 2.0, xtime.Second, nil}, {untagged, start.Add(2 * dataBlockSize), 1.0, xtime.Second, nil}, + } + + invalidNs := testValues{ {outOfRange, start.Add(-dataBlockSize), 1.0, xtime.Second, nil}, {shardTooHigh, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, {noShardBootstrapRange, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, + } + + valuesOtherNs := testValues{ {someOtherNamespace, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + values := append(valuesNs, invalidNs...) + values = append(values, valuesOtherNs...) + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -142,95 +174,23 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { targetRanges := result.ShardTimeRanges{ shardn(0): ranges, shardn(1): ranges, shardn(2): ranges, shardn(5): ranges} - if bootstrapDataFirst { - // Bootstrap the data to exercise the metadata caching path. - - // Bootstrap some arbitrary time range to make sure that it can handle multiple - // calls to ReadData() with different ranges and only a subset of the shards to make - // sure it can handle multiple calls to ReadData() with different shards / ranges (it needs - // to merge the ranges / shards across calls.) - var ( - targetRangesCopy = targetRanges.Copy() - highestShard = uint32(0) - ) - for key := range targetRangesCopy { - // The time-range here is "arbitrary", but it does need to be one for which - // there is data so that we can actually exercise the blockStart merging - // logic. - targetRangesCopy[key] = xtime.NewRanges(xtime.Range{ - Start: start, - End: start.Add(dataBlockSize), - }) - - if key > highestShard { - highestShard = key - } - } - - // Delete the highest hard number to ensure we're bootstrapping a subset, and the subsequent call - // to ReadData() will exercise the slice extending loggic. - delete(targetRangesCopy, highestShard) - _, err = src.ReadData(md1, targetRangesCopy, testDefaultRunOpts) - require.NoError(t, err) - - // Bootstrap the actual time ranges to actually cache the metadata. - _, err = src.ReadData(md1, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - } - - res, err := src.ReadIndex(md1, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - - // Data blockSize is 2 hours and index blockSize is four hours so the data blocks - // will span two different index blocks. - indexResults := res.IndexResults() - require.Equal(t, 2, len(indexResults)) - require.Equal(t, 0, len(res.Unfulfilled())) - - err = verifyIndexResultsAreCorrect(values, seriesNotToExpect, indexResults, indexBlockSize) - require.NoError(t, err) - - // Update the iterator function to only return values for the second namespace because - // the real commit log reader does this (via the ReadSeries predicate). - otherNamespaceValues := []testValue{} - for _, value := range values { - if value.s.Namespace.Equal(testNamespaceID2) { - otherNamespaceValues = append(otherNamespaceValues, value) - } - } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { - return newTestCommitLogIterator(otherNamespaceValues, nil), nil, nil - } - - res, err = src.ReadIndex(md2, targetRanges, testDefaultRunOpts) - require.NoError(t, err) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md1, md2, md3) + defer tester.Finish() - // Only one series so there should only be one index result. - indexResults = res.IndexResults() - require.Equal(t, 1, len(indexResults)) - require.Equal(t, 0, len(res.Unfulfilled())) + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md1) + tester.TestUnfulfilledForNamespaceIsEmpty(md2) + tester.TestUnfulfilledForNamespaceIsEmpty(md3) - err = verifyIndexResultsAreCorrect(otherNamespaceValues, seriesNotToExpect, indexResults, indexBlockSize) - require.NoError(t, err) + nsWrites := tester.EnsureDumpWritesForNamespace(md1) + enforceValuesAreCorrect(t, valuesNs, nsWrites) - // Update the iterator function to return no values (since this namespace has no data) - // because the real commit log reader does this (via the ReadSeries predicate). - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { - return newTestCommitLogIterator([]testValue{}, nil), nil, nil - } + otherNamespaceWrites := tester.EnsureDumpWritesForNamespace(md2) + enforceValuesAreCorrect(t, valuesOtherNs, otherNamespaceWrites) - res, err = src.ReadIndex(md3, targetRanges, testDefaultRunOpts) - require.NoError(t, err) - - // Zero series so there should be no index results. - indexResults = res.IndexResults() - require.Equal(t, 0, len(indexResults)) - require.Equal(t, 0, len(res.Unfulfilled())) - seg, err := indexResults.GetOrAddSegment(start, namespaceOptions.IndexOptions(), result.NewOptions()) - require.NoError(t, err) - require.Equal(t, int64(0), seg.Size()) - err = verifyIndexResultsAreCorrect([]testValue{}, seriesNotToExpect, indexResults, indexBlockSize) - require.NoError(t, err) + noWrites := tester.EnsureDumpWritesForNamespace(md3) + require.Equal(t, 0, len(noWrites)) + tester.EnsureNoLoadedBlocks() } func TestBootstrapIndexEmptyShardTimeRanges(t *testing.T) { @@ -255,41 +215,23 @@ func TestBootstrapIndexEmptyShardTimeRanges(t *testing.T) { md, err := namespace.NewMetadata(testNamespaceID, namespaceOptions) require.NoError(t, err) - values := []testValue{} + values := testValues{} src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } - res, err := src.ReadIndex(md, result.ShardTimeRanges{}, testDefaultRunOpts) - require.NoError(t, err) - - // Data blockSize is 2 hours and index blockSize is four hours so the data blocks - // will span two different index blocks. - indexResults := res.IndexResults() - require.Equal(t, 0, len(indexResults)) - require.Equal(t, 0, len(res.Unfulfilled())) -} - -func TestBootstrapIndexNamespaceIndexNotEnabled(t *testing.T) { - var ( - opts = testDefaultOpts - src = newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) - namespaceOptions = namespace.NewOptions(). - SetIndexOptions( - namespace.NewOptions(). - IndexOptions(). - SetEnabled(false), - ) - ) - md, err := namespace.NewMetadata(testNamespaceID, namespaceOptions) - require.NoError(t, err) + target := result.ShardTimeRanges{} + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, target, md) + defer tester.Finish() - _, err = src.ReadIndex(md, result.ShardTimeRanges{}, testDefaultRunOpts) - require.Equal(t, err, errIndexingNotEnableForNamespace) + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func verifyIndexResultsAreCorrect( - values []testValue, + values testValues, seriesNotToExpect map[string]struct{}, indexResults result.IndexResults, indexBlockSize time.Duration, @@ -398,3 +340,72 @@ func verifyIndexResultsAreCorrect( return nil } + +func TestBootstrapIndexFailsForDecodedTags(t *testing.T) { + var ( + opts = testDefaultOpts + src = newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) + dataBlockSize = 2 * time.Hour + indexBlockSize = 4 * time.Hour + namespaceOptions = namespaceOptions. + SetRetentionOptions( + namespaceOptions. + RetentionOptions(). + SetBlockSize(dataBlockSize), + ). + SetIndexOptions( + namespaceOptions. + IndexOptions(). + SetBlockSize(indexBlockSize). + SetEnabled(true), + ) + ) + md1, err := namespace.NewMetadata(testNamespaceID, namespaceOptions) + require.NoError(t, err) + + now := time.Now() + start := now.Truncate(indexBlockSize) + + fooTags := ident.NewTags(ident.StringTag("city", "ny")) + + shardn := func(n int) uint32 { return uint32(n) } + foo := ts.Series{UniqueIndex: 0, Namespace: testNamespaceID, Shard: shardn(0), + ID: ident.StringID("foo"), Tags: fooTags} + + values := testValues{ + {foo, start, 1.0, xtime.Second, nil}, + } + + src.newIteratorFn = func( + _ commitlog.IteratorOpts, + ) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { + return newTestCommitLogIterator(values, nil), nil, nil + } + + ranges := xtime.Ranges{} + ranges = ranges.AddRange(xtime.Range{ + Start: start, + End: start.Add(dataBlockSize), + }) + ranges = ranges.AddRange(xtime.Range{ + Start: start.Add(dataBlockSize), + End: start.Add(2 * dataBlockSize), + }) + ranges = ranges.AddRange(xtime.Range{ + Start: start.Add(2 * dataBlockSize), + End: start.Add(3 * dataBlockSize), + }) + + // Don't include ranges for shard 4 as thats how we're testing the noShardBootstrapRange series. + targetRanges := result.ShardTimeRanges{ + shardn(0): ranges, shardn(1): ranges, shardn(2): ranges, shardn(5): ranges} + + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, targetRanges, md1) + defer tester.Finish() + + _, err = src.Read(tester.Namespaces) + require.Error(t, err) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 3669bb52f1..0d38a6e4b5 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -40,6 +40,7 @@ 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/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/topology" tu "github.com/m3db/m3/src/dbnode/topology/testutil" @@ -80,7 +81,6 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { parameters.Rng.Seed(seed) nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts) require.NoError(t, err) - nsCtx := namespace.NewContextFrom(nsMeta) props.Property("Commitlog bootstrapping properly bootstraps the entire commitlog", prop.ForAll( func(input propTestInput) (bool, error) { @@ -280,6 +280,7 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { if err != nil { return false, err } + writesCh <- struct{}{} } close(writesCh) @@ -369,16 +370,21 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { tu.SelfID: tu.Shards(allShardsSlice, shard.Available), }) } + runOpts := testDefaultRunOpts.SetInitialTopologyState(initialTopoState) - dataResult, err := source.BootstrapData(nsMeta, shardTimeRanges, runOpts) + tester := bootstrap.BuildNamespacesTester(t, runOpts, shardTimeRanges, nsMeta) + + bootstrapResults, err := source.Bootstrap(tester.Namespaces) if err != nil { return false, err } // Create testValues for each datapoint for comparison - values := []testValue{} + values := testValues{} for _, write := range input.writes { - values = append(values, testValue{write.series, write.datapoint.Timestamp, write.datapoint.Value, write.unit, write.annotation}) + values = append(values, testValue{ + write.series, write.datapoint.Timestamp, + write.datapoint.Value, write.unit, write.annotation}) } commitLogFiles, corruptFiles, err := commitlog.Files(commitLogOpts) @@ -401,6 +407,12 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { !shardTimeRanges.IsEmpty() ) + nsResult, found := bootstrapResults.Results.Get(nsMeta.ID()) + if !found { + return false, fmt.Errorf("could not find id: %s", nsMeta.ID().String()) + } + + dataResult := nsResult.DataResult if shouldReturnUnfulfilled { if dataResult.Unfulfilled().IsEmpty() { return false, fmt.Errorf( @@ -413,23 +425,13 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { dataResult.Unfulfilled().String()) } } - err = verifyShardResultsAreCorrect(nsCtx, values, blockSize, dataResult.ShardResults(), bootstrapOpts) - if err != nil { - return false, err - } - indexResult, err := source.BootstrapIndex(nsMeta, shardTimeRanges, runOpts) - if err != nil { - return false, err - } - - indexBlockSize := nsMeta.Options().IndexOptions().BlockSize() - err = verifyIndexResultsAreCorrect( - values, map[string]struct{}{}, indexResult.IndexResults(), indexBlockSize) + written, err := tester.EnsureDumpAllForNamespace(nsMeta) if err != nil { return false, err } + indexResult := nsResult.IndexResult if shouldReturnUnfulfilled { if indexResult.Unfulfilled().IsEmpty() { return false, fmt.Errorf( @@ -443,6 +445,11 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } } + err = verifyValuesAreCorrect(values, written) + if err != nil { + return false, err + } + return true, nil }, genPropTestInputs(nsMeta, startTime), diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go index b2a09039bf..604c1c21d4 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go @@ -26,34 +26,28 @@ import ( "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" ) -// Options represents the options for bootstrapping from commit logs +// Options represents the options for bootstrapping from commit logs. type Options interface { - // Validate validates the options + // Validate validates the options. Validate() error - // SetResultOptions sets the result options + // SetResultOptions sets the result options. SetResultOptions(value result.Options) Options - // ResultOptions returns the result options + // ResultOptions returns the result options. ResultOptions() result.Options - // SetCommitLogOptions sets the commit log options + // SetCommitLogOptions sets the commit log options. SetCommitLogOptions(value commitlog.Options) Options - // CommitLogOptions returns the commit log options + // CommitLogOptions returns the commit log options. CommitLogOptions() commitlog.Options - // SetEncodingConcurrency sets the concurrency for encoding - SetEncodingConcurrency(value int) Options + // SetAccumulateConcurrency sets the concurrency for accumulating. + SetAccumulateConcurrency(value int) Options - // EncodingConcurrency returns the concurrency for encoding - EncodingConcurrency() int - - // SetMergeShardConcurrency sets the concurrency for merging shards - SetMergeShardsConcurrency(value int) Options - - // MergeShardConcurrency returns the concurrency for merging shards - MergeShardsConcurrency() int + // AccumulateConcurrency returns the concurrency for accumulating. + AccumulateConcurrency() int // SetReturnUnfulfilledForCorruptCommitLogFiles sets whether the bootstrapper // should return unfulfilled if it encounters corrupt commitlog files. diff --git a/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go b/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go index 74034c9599..e01a114536 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go @@ -97,7 +97,7 @@ func newFileSystemSource(opts Options) bootstrap.Source { s := &fileSystemSource{ opts: opts, fsopts: opts.FilesystemOptions(), - log: iopts.Logger(), + log: iopts.Logger().With(zap.String("bootstrapper", "filesystem")), idPool: opts.IdentifierPool(), newReaderFn: fs.NewReader, dataProcessors: dataProcessors, @@ -115,14 +115,6 @@ func newFileSystemSource(opts Options) bootstrap.Source { return s } -func (s *fileSystemSource) Can(strategy bootstrap.Strategy) bool { - switch strategy { - case bootstrap.BootstrapSequential: - return true - } - return false -} - func (s *fileSystemSource) AvailableData( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -131,18 +123,6 @@ func (s *fileSystemSource) AvailableData( return s.availability(md, shardsTimeRanges) } -func (s *fileSystemSource) ReadData( - md namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - r, err := s.read(md, shardsTimeRanges, bootstrapDataRunType, runOpts) - if err != nil { - return nil, err - } - return r.data, nil -} - func (s *fileSystemSource) AvailableIndex( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -151,23 +131,79 @@ func (s *fileSystemSource) AvailableIndex( return s.availability(md, shardsTimeRanges) } -func (s *fileSystemSource) ReadIndex( - md namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - r, err := s.read(md, shardsTimeRanges, bootstrapIndexRunType, runOpts) - if err != nil { - return nil, err +func (s *fileSystemSource) Read( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + results := bootstrap.NamespaceResults{ + Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}), } - return r.index, nil + + // NB(r): Perform all data bootstrapping first then index bootstrapping + // to more clearly deliniate which process is slower than the other. + nowFn := s.opts.ResultOptions().ClockOptions().NowFn() + start := nowFn() + s.log.Info("bootstrapping time series data start") + for _, elem := range namespaces.Namespaces.Iter() { + namespace := elem.Value() + md := namespace.Metadata + + r, err := s.read(bootstrapDataRunType, md, namespace.DataAccumulator, + namespace.DataRunOptions.ShardTimeRanges, + namespace.DataRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + results.Results.Set(md.ID(), bootstrap.NamespaceResult{ + Metadata: md, + Shards: namespace.Shards, + DataResult: r.data, + }) + } + s.log.Info("bootstrapping time series data success", + zap.Duration("took", nowFn().Sub(start))) + + start = nowFn() + s.log.Info("bootstrapping index metadata start") + for _, elem := range namespaces.Namespaces.Iter() { + namespace := elem.Value() + md := namespace.Metadata + if !md.Options().IndexOptions().Enabled() { + // Not bootstrapping for index. + s.log.Info("bootstrapping for namespace disabled by options", + zap.String("ns", md.ID().String())) + continue + } + + r, err := s.read(bootstrapIndexRunType, md, namespace.DataAccumulator, + namespace.IndexRunOptions.ShardTimeRanges, + namespace.IndexRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + result, ok := results.Results.Get(md.ID()) + if !ok { + err = fmt.Errorf("missing expected result for namespace: %s", + md.ID().String()) + return bootstrap.NamespaceResults{}, err + } + + result.IndexResult = r.index + + results.Results.Set(md.ID(), result) + } + s.log.Info("bootstrapping index metadata success", + zap.Stringer("took", nowFn().Sub(start))) + + return results, nil } func (s *fileSystemSource) availability( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, ) (result.ShardTimeRanges, error) { - result := make(map[uint32]xtime.Ranges) + result := make(map[uint32]xtime.Ranges, len(shardsTimeRanges)) for shard, ranges := range shardsTimeRanges { result[shard] = s.shardAvailability(md.ID(), shard, ranges) } @@ -211,8 +247,8 @@ func (s *fileSystemSource) shardAvailability( } func (s *fileSystemSource) enqueueReaders( - ns namespace.Metadata, run runType, + ns namespace.Metadata, runOpts bootstrap.RunOptions, shardsTimeRanges result.ShardTimeRanges, readerPool *readerPool, @@ -224,7 +260,7 @@ func (s *fileSystemSource) enqueueReaders( shouldPersistIndexBootstrap := run == bootstrapIndexRunType && s.shouldPersist(runOpts) if !shouldPersistIndexBootstrap { // Normal run, open readers - s.enqueueReadersGroupedByBlockSize(ns, run, runOpts, + s.enqueueReadersGroupedByBlockSize(run, ns, runOpts, shardsTimeRanges, readerPool, readersCh) return } @@ -254,14 +290,14 @@ func (s *fileSystemSource) enqueueReaders( // greater than the number of shards. continue } - s.enqueueReadersGroupedByBlockSize(ns, run, runOpts, + s.enqueueReadersGroupedByBlockSize(run, ns, runOpts, bucket, readerPool, readersCh) } } func (s *fileSystemSource) enqueueReadersGroupedByBlockSize( - ns namespace.Metadata, run runType, + ns namespace.Metadata, runOpts bootstrap.RunOptions, shardTimeRanges result.ShardTimeRanges, readerPool *readerPool, @@ -367,23 +403,19 @@ func (s *fileSystemSource) newShardReaders( } func (s *fileSystemSource) bootstrapFromReaders( - ns namespace.Metadata, run runType, + ns namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, runOpts bootstrap.RunOptions, readerPool *readerPool, - retriever block.DatabaseBlockRetriever, readersCh <-chan timeWindowReaders, ) *runResult { var ( - runResult = newRunResult() - resultOpts = s.opts.ResultOptions() - shardRetrieverMgr block.DatabaseShardBlockRetrieverManager - wg sync.WaitGroup - processors xsync.WorkerPool + runResult = newRunResult() + resultOpts = s.opts.ResultOptions() + wg sync.WaitGroup + processors xsync.WorkerPool ) - if retriever != nil { - shardRetrieverMgr = block.NewDatabaseShardBlockRetrieverManager(retriever) - } switch run { case bootstrapDataRunType: @@ -398,20 +430,13 @@ func (s *fileSystemSource) bootstrapFromReaders( timeWindowReaders := timeWindowReaders wg.Add(1) processors.Go(func() { - s.loadShardReadersDataIntoShardResult(ns, run, runOpts, runResult, - resultOpts, shardRetrieverMgr, timeWindowReaders, readerPool) + s.loadShardReadersDataIntoShardResult(run, ns, accumulator, + runOpts, runResult, resultOpts, timeWindowReaders, readerPool) wg.Done() }) } wg.Wait() - shardResults := runResult.data.ShardResults() - for shard, results := range shardResults { - if results.NumSeries() == 0 { - delete(shardResults, shard) - } - } - return runResult } @@ -436,33 +461,9 @@ func (s *fileSystemSource) markRunResultErrorsAndUnfulfilled( for i := range timesWithErrors { timesWithErrorsString[i] = timesWithErrors[i].String() } - s.log.Info("deleting entries from results for times with errors", + s.log.Info("encounted errors for range", zap.String("requestedRanges", requestedRanges.SummaryString()), - zap.Strings("timesWithErrors", timesWithErrorsString), - ) - - runResult.Lock() - for shard := range requestedRanges { - // Delete all affected times from the data results. - shardResult, ok := runResult.data.ShardResults()[shard] - if ok { - for _, entry := range shardResult.AllSeries().Iter() { - series := entry.Value() - for _, t := range timesWithErrors { - shardResult.RemoveBlockAt(series.ID, t) - } - } - } - } - // NB(r): We explicitly do not remove entries from the index results - // as they are additive and get merged together with results from other - // bootstrappers by just appending the result (unlike data bootstrap - // results that when merged replace the block with the current block). - // It would also be difficult to remove only series that were added to the - // index block as results from data files can be subsets of the index block - // and there's no way to definitively delete the entry we added as a result - // of just this data file failing. - runResult.Unlock() + zap.Strings("timesWithErrors", timesWithErrorsString)) } if !remainingRanges.IsEmpty() { @@ -478,12 +479,12 @@ func (s *fileSystemSource) markRunResultErrorsAndUnfulfilled( } func (s *fileSystemSource) loadShardReadersDataIntoShardResult( - ns namespace.Metadata, run runType, + ns namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, runOpts bootstrap.RunOptions, runResult *runResult, ropts result.Options, - shardRetrieverMgr block.DatabaseShardBlockRetrieverManager, timeWindowReaders timeWindowReaders, readerPool *readerPool, ) { @@ -492,8 +493,6 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( seriesCachePolicy = ropts.SeriesCachePolicy() indexBlockSegment segment.MutableSegment timesWithErrors []time.Time - shardResult result.ShardResult - shardRetriever block.DatabaseShardBlockRetriever nsCtx = namespace.NewContextFrom(ns) ) @@ -504,13 +503,6 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( shard := uint32(shard) readers := shardReaders.readers - if run == bootstrapDataRunType { - // For the bootstrap data case we need the shard retriever. - if shardRetrieverMgr != nil { - shardRetriever = shardRetrieverMgr.ShardRetriever(shard) - } - } - for _, r := range readers { var ( timeRange = r.Range() @@ -520,8 +512,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( ) switch run { case bootstrapDataRunType: - capacity := r.Entries() - shardResult = runResult.getOrAddDataShardResult(shard, capacity, ropts) + // Pass, since nothing to do. case bootstrapIndexRunType: indexBlockSegment, err = runResult.getOrAddIndexSegment(start, ns, ropts) default: @@ -533,8 +524,8 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( for i := 0; err == nil && i < numEntries; i++ { switch run { case bootstrapDataRunType: - err = s.readNextEntryAndRecordBlock(nsCtx, r, runResult, start, blockSize, shardResult, - shardRetriever, blockPool, seriesCachePolicy) + err = s.readNextEntryAndRecordBlock(nsCtx, accumulator, shard, r, + runResult, start, blockSize, blockPool, seriesCachePolicy) case bootstrapIndexRunType: // We can just read the entry and index if performing an index run. err = s.readNextEntryAndIndex(r, runResult, indexBlockSegment) @@ -549,10 +540,9 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( var validateErr error switch run { case bootstrapDataRunType: - switch seriesCachePolicy { - case series.CacheAll: + if seriesCachePolicy == series.CacheAll { validateErr = r.Validate() - default: + } else { err = fmt.Errorf("invalid series cache policy: %s", seriesCachePolicy.String()) } case bootstrapIndexRunType: @@ -598,8 +588,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( l.Error("persist fs index bootstrap failed", zap.Stringer("namespace", ns.ID()), zap.Stringer("requestedRanges", requestedRanges), - zap.Error(err), - ) + zap.Error(err)) }) } } @@ -619,12 +608,12 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( func (s *fileSystemSource) readNextEntryAndRecordBlock( nsCtx namespace.Context, + accumulator bootstrap.NamespaceDataAccumulator, + shardID uint32, r fs.DataFileSetReader, runResult *runResult, blockStart time.Time, blockSize time.Duration, - shardResult result.ShardResult, - shardRetriever block.DatabaseShardBlockRetriever, blockPool block.DatabaseBlockPool, seriesCachePolicy series.CachePolicy, ) error { @@ -636,6 +625,16 @@ func (s *fileSystemSource) readNextEntryAndRecordBlock( err error ) + defer func() { + // Can finalize the ID and tags always. + if id != nil { + id.Finalize() + } + if tagsIter != nil { + tagsIter.Close() + } + }() + switch seriesCachePolicy { case series.CacheAll: id, tagsIter, data, _, err = r.Read() @@ -646,43 +645,17 @@ func (s *fileSystemSource) readNextEntryAndRecordBlock( return fmt.Errorf("error reading data file: %v", err) } - var ( - entry result.DatabaseSeriesBlocks - tags ident.Tags - exists bool - ) - runResult.Lock() - defer runResult.Unlock() - - entry, exists = shardResult.AllSeries().Get(id) - if exists { - // NB(r): In the case the series is already inserted - // we can avoid holding onto this ID and use the already - // allocated ID. - id.Finalize() - id = entry.ID - tags = entry.Tags - } else { - tags, err = convert.TagsFromTagsIter(id, tagsIter, s.idPool) - if err != nil { - return fmt.Errorf("unable to decode tags: %v", err) - } + ref, err := accumulator.CheckoutSeriesWithLock(shardID, id, tagsIter) + if err != nil { + return fmt.Errorf("unable to checkout series: %v", err) } - tagsIter.Close() - switch seriesCachePolicy { - case series.CacheAll: - seg := ts.NewSegment(data, nil, ts.FinalizeHead) - seriesBlock.Reset(blockStart, blockSize, seg, nsCtx) - default: - return fmt.Errorf("invalid series cache policy: %s", seriesCachePolicy.String()) + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + seriesBlock.Reset(blockStart, blockSize, seg, nsCtx) + if err := ref.Series.LoadBlock(seriesBlock, series.WarmWrite); err != nil { + return fmt.Errorf("unable to load block: %v", err) } - if exists { - entry.Blocks.AddBlock(seriesBlock) - } else { - shardResult.AddBlock(id, tags, seriesBlock) - } return nil } @@ -920,14 +893,14 @@ func (s *fileSystemSource) persistBootstrapIndexSegment( } func (s *fileSystemSource) read( + run runType, md namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, shardsTimeRanges result.ShardTimeRanges, - run runType, runOpts bootstrap.RunOptions, ) (*runResult, error) { var ( seriesCachePolicy = s.opts.ResultOptions().SeriesCachePolicy() - blockRetriever block.DatabaseBlockRetriever res *runResult ) if shardsTimeRanges.IsEmpty() { @@ -946,33 +919,10 @@ func (s *fileSystemSource) read( } if run == bootstrapDataRunType { - // NB(r): We only need to marks blocks as fulfilled when bootstrapping data, - // because the data can be retrieved lazily from disk during reads. - // On the other hand, if we're bootstrapping the index then currently we - // need to rebuild it from scratch by reading all the IDs/tags until - // we can natively bootstrap persisted segments from disk and compact them - // with series metadata from other shards if topology has changed. - if mgr := s.opts.DatabaseBlockRetrieverManager(); mgr != nil { - shards := make([]uint32, 0, len(shardsTimeRanges)) - for shard := range shardsTimeRanges { - shards = append(shards, shard) - } - var err error - blockRetriever, err = s.resolveBlockRetriever(md, - mgr, shards) - if err != nil { - return nil, err - } - } - - switch seriesCachePolicy { - case series.CacheAll: - // No checks necessary. - default: + if seriesCachePolicy != series.CacheAll { // Unless we're caching all series (or all series metadata) in memory, we // return just the availability of the files we have. - return s.bootstrapDataRunResultFromAvailability(md, - shardsTimeRanges), nil + return s.bootstrapDataRunResultFromAvailability(md, shardsTimeRanges), nil } } @@ -998,10 +948,10 @@ func (s *fileSystemSource) read( // hence why its created on demand each time. readerPool := newReaderPool(s.newReaderPoolOpts) readersCh := make(chan timeWindowReaders) - go s.enqueueReaders(md, run, runOpts, shardsTimeRanges, + go s.enqueueReaders(run, md, runOpts, shardsTimeRanges, readerPool, readersCh) - bootstrapFromDataReadersResult := s.bootstrapFromReaders(md, run, runOpts, - readerPool, blockRetriever, readersCh) + bootstrapFromDataReadersResult := s.bootstrapFromReaders(run, md, + accumulator, runOpts, readerPool, readersCh) // Merge any existing results if necessary. setOrMergeResult(bootstrapFromDataReadersResult) @@ -1014,28 +964,6 @@ func (s *fileSystemSource) newReader() (fs.DataFileSetReader, error) { return s.newReaderFn(bytesPool, s.fsopts) } -func (s *fileSystemSource) resolveBlockRetriever( - md namespace.Metadata, - blockRetrieverMgr block.DatabaseBlockRetrieverManager, - shards []uint32, -) ( - block.DatabaseBlockRetriever, - error, -) { - var blockRetriever block.DatabaseBlockRetriever - - s.log.Info("filesystem bootstrapper resolving block retriever", - zap.Stringer("namespace", md.ID())) - - var err error - blockRetriever, err = blockRetrieverMgr.Retriever(md) - if err != nil { - return nil, err - } - - return blockRetriever, nil -} - func (s *fileSystemSource) bootstrapDataRunResultFromAvailability( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -1048,7 +976,11 @@ func (s *fileSystemSource) bootstrapDataRunResultFromAvailability( } availability := s.shardAvailability(md.ID(), shard, ranges) remaining := ranges.RemoveRanges(availability) - runResult.data.Add(shard, nil, remaining) + if !remaining.IsEmpty() { + unfulfilled.AddRanges(result.ShardTimeRanges{ + shard: remaining, + }) + } } runResult.data.SetUnfulfilled(unfulfilled) return runResult @@ -1234,29 +1166,6 @@ func newRunResult() *runResult { } } -func (r *runResult) getOrAddDataShardResult( - shard uint32, - capacity int, - ropts result.Options, -) result.ShardResult { - // Only called once per shard so ok to acquire write lock immediately. - r.Lock() - defer r.Unlock() - - dataResults := r.data.ShardResults() - shardResult, exists := dataResults[shard] - if exists { - return shardResult - } - - // NB(r): Wait until we have a reader to initialize the shard result - // to be able to somewhat estimate the size of it. - shardResult = result.NewShardResult(capacity, ropts) - dataResults[shard] = shardResult - - return shardResult -} - func (r *runResult) getOrAddIndexSegment( start time.Time, ns namespace.Metadata, diff --git a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_data_test.go index 95ca8f394e..0feff47d22 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_data_test.go @@ -41,9 +41,10 @@ import ( "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/pool" + xtest "github.com/m3db/m3/src/x/test" xtime "github.com/m3db/m3/src/x/time" - "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -89,11 +90,15 @@ func newTestFsOptions(filePathPrefix string) fs.Options { } func testNsMetadata(t *testing.T) namespace.Metadata { + return testNsMetadataWithIndex(t, true) +} + +func testNsMetadataWithIndex(t *testing.T, indexOn bool) namespace.Metadata { ropts := testRetentionOptions.SetBlockSize(testBlockSize) md, err := namespace.NewMetadata(testNs1ID, testNamespaceOptions. SetRetentionOptions(ropts). SetIndexOptions(testNamespaceIndexOptions. - SetEnabled(true). + SetEnabled(indexOn). SetBlockSize(testIndexBlockSize))) require.NoError(t, err) return md @@ -105,21 +110,27 @@ func createTempDir(t *testing.T) string { return dir } -func writeInfoFile(t *testing.T, prefix string, namespace ident.ID, shard uint32, start time.Time, data []byte) { +func writeInfoFile(t *testing.T, prefix string, namespace ident.ID, + shard uint32, start time.Time, data []byte) { shardDir := fs.ShardDataDirPath(prefix, namespace, shard) - filePath := path.Join(shardDir, fmt.Sprintf("fileset-%d-0-info.db", xtime.ToNanoseconds(start))) + filePath := path.Join(shardDir, + fmt.Sprintf("fileset-%d-0-info.db", xtime.ToNanoseconds(start))) writeFile(t, filePath, data) } -func writeDataFile(t *testing.T, prefix string, namespace ident.ID, shard uint32, start time.Time, data []byte) { +func writeDataFile(t *testing.T, prefix string, namespace ident.ID, + shard uint32, start time.Time, data []byte) { shardDir := fs.ShardDataDirPath(prefix, namespace, shard) - filePath := path.Join(shardDir, fmt.Sprintf("fileset-%d-0-data.db", xtime.ToNanoseconds(start))) + filePath := path.Join(shardDir, + fmt.Sprintf("fileset-%d-0-data.db", xtime.ToNanoseconds(start))) writeFile(t, filePath, data) } -func writeDigestFile(t *testing.T, prefix string, namespace ident.ID, shard uint32, start time.Time, data []byte) { +func writeDigestFile(t *testing.T, prefix string, namespace ident.ID, + shard uint32, start time.Time, data []byte) { shardDir := fs.ShardDataDirPath(prefix, namespace, shard) - filePath := path.Join(shardDir, fmt.Sprintf("fileset-%d-0-digest.db", xtime.ToNanoseconds(start))) + filePath := path.Join(shardDir, + fmt.Sprintf("fileset-%d-0-digest.db", xtime.ToNanoseconds(start))) writeFile(t, filePath, data) } @@ -141,6 +152,19 @@ func testShardTimeRanges() result.ShardTimeRanges { return map[uint32]xtime.Ranges{testShard: testTimeRanges()} } +func testBootstrappingIndexShardTimeRanges() result.ShardTimeRanges { + // NB: since index files are not corrupted on this run, it's expected that + // `testBlockSize` values should be fulfilled in the index block. This is + // `testBlockSize` rather than `testIndexSize` since the files generated + // by this test use 2 hour (which is `testBlockSize`) reader blocks. + return map[uint32]xtime.Ranges{ + testShard: xtime.Ranges{}.AddRange(xtime.Range{ + Start: testStart.Add(testBlockSize), + End: testStart.Add(11 * time.Hour), + }), + } +} + func writeGoodFiles(t *testing.T, dir string, namespace ident.ID, shard uint32) { inputs := []struct { start time.Time @@ -368,22 +392,69 @@ func TestAvailableTimeRangePartialError(t *testing.T) { validateTimeRanges(t, res[testShard], expected) } +// NB: too real :'( +func unfulfilledAndEmpty(t *testing.T, src bootstrap.Source, + md namespace.Metadata, tester bootstrap.NamespacesTester) { + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(md) + + tester.EnsureNoWrites() + tester.EnsureNoLoadedBlocks() +} + func TestReadEmptyRangeErr(t *testing.T) { src := newFileSystemSource(newTestOptions("foo")) - res, err := src.ReadData(testNsMetadata(t), nil, testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, 0, len(res.ShardResults())) - require.True(t, res.Unfulfilled().IsEmpty()) + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, nil, nsMD) + defer tester.Finish() + unfulfilledAndEmpty(t, src, nsMD, tester) } func TestReadPatternError(t *testing.T) { src := newFileSystemSource(newTestOptions("[[")) - res, err := src.ReadData(testNsMetadata(t), - map[uint32]xtime.Ranges{testShard: xtime.Ranges{}}, - testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, 0, len(res.ShardResults())) - require.True(t, res.Unfulfilled().IsEmpty()) + timeRanges := result.ShardTimeRanges{testShard: xtime.Ranges{}} + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + timeRanges, nsMD) + defer tester.Finish() + unfulfilledAndEmpty(t, src, nsMD, tester) +} + +func validateReadResults( + t *testing.T, + src bootstrap.Source, + dir string, + strs result.ShardTimeRanges, +) { + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, strs, nsMD) + defer tester.Finish() + + tester.TestReadWith(src) + readers := tester.EnsureDumpReadersForNamespace(nsMD) + require.Equal(t, 2, len(readers)) + ids := []string{"foo", "bar"} + data := [][]byte{ + {1, 2, 3}, + {4, 5, 6}, + } + + times := []time.Time{testStart, testStart.Add(10 * time.Hour)} + for i, id := range ids { + seriesReaders, ok := readers[id] + require.True(t, ok) + require.Equal(t, 1, len(seriesReaders)) + readerAtTime := seriesReaders[0] + assert.Equal(t, times[i], readerAtTime.Start) + ctx := context.NewContext() + var b [100]byte + n, err := readerAtTime.Reader.Read(b[:]) + ctx.Close() + require.NoError(t, err) + require.Equal(t, data[i], b[:n]) + } + + tester.EnsureNoWrites() } func TestReadNilTimeRanges(t *testing.T) { @@ -394,11 +465,12 @@ func TestReadNilTimeRanges(t *testing.T) { writeGoodFiles(t, dir, testNs1ID, shard) src := newFileSystemSource(newTestOptions(dir)) - - validateReadResults(t, src, dir, map[uint32]xtime.Ranges{ + timeRanges := result.ShardTimeRanges{ testShard: testTimeRanges(), 555: xtime.Ranges{}, - }) + } + + validateReadResults(t, src, dir, timeRanges) } func TestReadOpenFileError(t *testing.T) { @@ -409,23 +481,38 @@ func TestReadOpenFileError(t *testing.T) { writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ {"foo", nil, []byte{0x1}}, }) + // Intentionally truncate the info file writeInfoFile(t, dir, testNs1ID, shard, testStart, nil) src := newFileSystemSource(newTestOptions(dir)) - res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), - testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.NotNil(t, res.Unfulfilled()) - require.NotNil(t, res.Unfulfilled()[testShard]) + nsMD := testNsMetadata(t) + ranges := testShardTimeRanges() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + ranges, nsMD) + defer tester.Finish() - expected := xtime.Ranges{}. - AddRange(xtime.Range{Start: testStart, End: testStart.Add(11 * time.Hour)}) - validateTimeRanges(t, res.Unfulfilled()[testShard], expected) + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespace(nsMD, ranges, ranges) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } -func TestReadDataCorruptionError(t *testing.T) { +func TestReadDataCorruptionErrorNoIndex(t *testing.T) { + testReadDataCorruptionErrorWithIndexEnabled(t, false, testShardTimeRanges()) +} + +func TestReadDataCorruptionErrorWithIndex(t *testing.T) { + expectedIndex := testBootstrappingIndexShardTimeRanges() + testReadDataCorruptionErrorWithIndexEnabled(t, true, expectedIndex) +} + +func testReadDataCorruptionErrorWithIndexEnabled( + t *testing.T, + withIndex bool, + expectedIndexUnfulfilled result.ShardTimeRanges, +) { dir := createTempDir(t) defer os.RemoveAll(dir) @@ -438,61 +525,14 @@ func TestReadDataCorruptionError(t *testing.T) { src := newFileSystemSource(newTestOptions(dir)) strs := testShardTimeRanges() - res, err := src.ReadData(testNsMetadata(t), strs, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 0, len(res.ShardResults())) - require.Equal(t, 1, len(res.Unfulfilled())) - validateTimeRanges(t, res.Unfulfilled()[testShard], strs[testShard]) -} -func validateReadResults( - t *testing.T, - src bootstrap.Source, - dir string, - strs result.ShardTimeRanges, -) { - expected := xtime.Ranges{}. - AddRange(xtime.Range{ - Start: testStart.Add(2 * time.Hour), - End: testStart.Add(10 * time.Hour), - }) + nsMD := testNsMetadataWithIndex(t, withIndex) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, strs, nsMD) + defer tester.Finish() - res, err := src.ReadData(testNsMetadata(t), strs, testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.NotNil(t, res.ShardResults()) - require.NotNil(t, res.ShardResults()[testShard]) - allSeries := res.ShardResults()[testShard].AllSeries() - require.Equal(t, 2, allSeries.Len()) - require.NotNil(t, res.Unfulfilled()) - require.NotNil(t, res.Unfulfilled()[testShard]) - validateTimeRanges(t, res.Unfulfilled()[testShard], expected) - - require.Equal(t, 2, allSeries.Len()) - - ids := []ident.ID{ - ident.StringID("foo"), ident.StringID("bar")} - data := [][]byte{ - {1, 2, 3}, - {4, 5, 6}, - } - times := []time.Time{testStart, testStart.Add(10 * time.Hour)} - for i, id := range ids { - series, ok := allSeries.Get(id) - require.True(t, ok) - allBlocks := series.Blocks.AllBlocks() - require.Equal(t, 1, len(allBlocks)) - block := allBlocks[xtime.ToUnixNano(times[i])] - ctx := context.NewContext() - stream, err := block.Stream(ctx) - require.NoError(t, err) - var b [100]byte - n, err := stream.Read(b[:]) - ctx.Close() - require.NoError(t, err) - require.Equal(t, data[i], b[:n]) - } + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespace(nsMD, strs, expectedIndexUnfulfilled) + tester.EnsureNoWrites() } func TestReadTimeFilter(t *testing.T) { @@ -517,8 +557,21 @@ func TestReadPartialError(t *testing.T) { validateReadResults(t, src, dir, testShardTimeRanges()) } -func TestReadValidateError(t *testing.T) { - ctrl := gomock.NewController(t) +func TestReadValidateErrorNoIndex(t *testing.T) { + testReadValidateErrorWithIndexEnabled(t, false, testShardTimeRanges()) +} + +func TestReadValidateErrorWithIndex(t *testing.T) { + expectedIndex := testBootstrappingIndexShardTimeRanges() + testReadValidateErrorWithIndexEnabled(t, true, expectedIndex) +} + +func testReadValidateErrorWithIndexEnabled( + t *testing.T, + enabled bool, + expectedIndexUnfulfilled result.ShardTimeRanges, +) { + ctrl := xtest.NewController(t) defer ctrl.Finish() dir := createTempDir(t) @@ -559,26 +612,37 @@ func TestReadValidateError(t *testing.T) { Start: testStart, End: testStart.Add(2 * time.Hour), }) - reader.EXPECT().Entries().Return(0).Times(2) + reader.EXPECT().Entries().Return(0).AnyTimes() reader.EXPECT().Validate().Return(errors.New("foo")) reader.EXPECT().Close().Return(nil) - res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), - testDefaultRunOpts) + nsMD := testNsMetadataWithIndex(t, enabled) + ranges := testShardTimeRanges() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + ranges, nsMD) + defer tester.Finish() - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 0, int(res.ShardResults().NumSeries())) - require.NotNil(t, res.Unfulfilled()) - require.NotNil(t, res.Unfulfilled()[testShard]) + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespace(nsMD, ranges, expectedIndexUnfulfilled) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() +} - expected := xtime.Ranges{}. - AddRange(xtime.Range{Start: testStart, End: testStart.Add(11 * time.Hour)}) - validateTimeRanges(t, res.Unfulfilled()[testShard], expected) +func TestReadOpenErrorNoIndex(t *testing.T) { + testReadOpenError(t, false, testShardTimeRanges()) +} + +func TestReadOpenErrorWithIndex(t *testing.T) { + expectedIndex := testBootstrappingIndexShardTimeRanges() + testReadOpenError(t, true, expectedIndex) } -func TestReadOpenError(t *testing.T) { - ctrl := gomock.NewController(t) +func testReadOpenError( + t *testing.T, + enabled bool, + expectedIndexUnfulfilled result.ShardTimeRanges, +) { + ctrl := xtest.NewController(t) defer ctrl.Finish() dir := createTempDir(t) @@ -613,21 +677,20 @@ func TestReadOpenError(t *testing.T) { Open(rOpts). Return(errors.New("error")) - res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), - testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 0, len(res.ShardResults())) - require.NotNil(t, res.Unfulfilled()) - require.NotNil(t, res.Unfulfilled()[testShard]) + nsMD := testNsMetadataWithIndex(t, enabled) + ranges := testShardTimeRanges() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + ranges, nsMD) + defer tester.Finish() - expected := xtime.Ranges{}. - AddRange(xtime.Range{Start: testStart, End: testStart.Add(11 * time.Hour)}) - validateTimeRanges(t, res.Unfulfilled()[testShard], expected) + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespace(nsMD, ranges, expectedIndexUnfulfilled) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func TestReadDeleteOnError(t *testing.T) { - ctrl := gomock.NewController(t) + ctrl := xtest.NewController(t) defer ctrl.Finish() dir := createTempDir(t) @@ -654,42 +717,40 @@ func TestReadDeleteOnError(t *testing.T) { BlockStart: testStart, }, } - gomock.InOrder( - reader.EXPECT().Open(rOpts).Return(nil), - reader.EXPECT(). - Range(). - Return(xtime.Range{ - Start: testStart, - End: testStart.Add(2 * time.Hour), - }).AnyTimes(), - reader.EXPECT().Entries().Return(2).AnyTimes(), - reader.EXPECT(). - Range(). - Return(xtime.Range{ - Start: testStart, - End: testStart.Add(2 * time.Hour), - }).AnyTimes(), - reader.EXPECT().Entries().Return(2).AnyTimes(), - reader.EXPECT(). - Read(). - Return(ident.StringID("foo"), ident.EmptyTagIterator, nil, digest.Checksum(nil), nil), - reader.EXPECT(). - Read(). - Return(ident.StringID("bar"), ident.EmptyTagIterator, nil, uint32(0), errors.New("foo")), - reader.EXPECT().Close().Return(nil), - ) - res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), - testDefaultRunOpts) - require.NoError(t, err) - require.NotNil(t, res) - require.Equal(t, 0, len(res.ShardResults())) - require.NotNil(t, res.Unfulfilled()) - require.NotNil(t, res.Unfulfilled()[testShard]) + reader.EXPECT().Open(rOpts).Return(nil).AnyTimes() + reader.EXPECT().ReadMetadata().Return(ident.StringID("foo"), + ident.NewTagsIterator(ident.Tags{}), 0, uint32(0), nil) + reader.EXPECT().ReadMetadata().Return(ident.StringID("bar"), + ident.NewTagsIterator(ident.Tags{}), 0, uint32(0), errors.New("foo")) - expected := xtime.Ranges{}. - AddRange(xtime.Range{Start: testStart, End: testStart.Add(11 * time.Hour)}) - validateTimeRanges(t, res.Unfulfilled()[testShard], expected) + reader.EXPECT(). + Range(). + Return(xtime.Range{ + Start: testStart, + End: testStart.Add(2 * time.Hour), + }).AnyTimes() + reader.EXPECT().Entries().Return(2).AnyTimes() + reader.EXPECT(). + Read(). + Return(ident.StringID("foo"), ident.EmptyTagIterator, + nil, digest.Checksum(nil), nil) + + reader.EXPECT(). + Read(). + Return(ident.StringID("bar"), ident.EmptyTagIterator, + nil, uint32(0), errors.New("foo")) + reader.EXPECT().Close().Return(nil).AnyTimes() + + nsMD := testNsMetadata(t) + ranges := testShardTimeRanges() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + ranges, nsMD) + defer tester.Finish() + + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespace(nsMD, ranges, ranges) + tester.EnsureNoWrites() } func TestReadTags(t *testing.T) { @@ -708,18 +769,19 @@ func TestReadTags(t *testing.T) { }) src := newFileSystemSource(newTestOptions(dir)) - res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), - testDefaultRunOpts) - require.NoError(t, err) - - require.Equal(t, 1, len(res.ShardResults())) - require.NotNil(t, res.ShardResults()[testShard]) - - series := res.ShardResults()[testShard] - require.Equal(t, int64(1), series.NumSeries()) - - fooSeries, ok := series.AllSeries().Get(ident.StringID(id)) - require.True(t, ok) - require.True(t, fooSeries.ID.Equal(ident.StringID(id))) - require.True(t, fooSeries.Tags.Equal(sortedTagsFromTagsMap(tags))) + nsMD := testNsMetadata(t) + ranges := testShardTimeRanges() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + ranges, nsMD) + defer tester.Finish() + + tester.TestReadWith(src) + readers := tester.EnsureDumpReadersForNamespace(nsMD) + require.Equal(t, 1, len(readers)) + readersForTime, found := readers[id] + require.True(t, found) + require.Equal(t, 1, len(readersForTime)) + reader := readersForTime[0] + require.Equal(t, tags, reader.Tags) + tester.EnsureNoWrites() } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go index 116277e1a9..2bff7d33ce 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go @@ -26,17 +26,16 @@ import ( "testing" "time" + "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/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/index/convert" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/m3ninx/index/segment" "github.com/m3db/m3/src/m3ninx/index/segment/mem" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" - "github.com/stretchr/testify/require" "github.com/uber-go/tally" ) @@ -311,12 +310,15 @@ func TestBootstrapIndex(t *testing.T) { writeTSDBGoodTaggedSeriesDataFiles(t, dir, testNs1ID, times.start) src := newFileSystemSource(newTestOptions(dir)) - res, err := src.ReadIndex(testNsMetadata(t), times.shardTimeRanges, - testDefaultRunOpts) - require.NoError(t, err) + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, + times.shardTimeRanges, nsMD) + defer tester.Finish() - indexResults := res.IndexResults() + tester.TestReadWith(src) + indexResults := tester.ResultForNamespace(nsMD.ID()).IndexResult.IndexResults() validateGoodTaggedSeries(t, times.start, indexResults, timesOpts) + tester.EnsureNoWrites() } func TestBootstrapIndexWithPersist(t *testing.T) { @@ -338,9 +340,13 @@ func TestBootstrapIndexWithPersist(t *testing.T) { SetPersistConfig(bootstrap.PersistConfig{Enabled: true}) src := newFileSystemSource(opts).(*fileSystemSource) - res, err := src.ReadIndex(testNsMetadata(t), times.shardTimeRanges, - runOpts) - require.NoError(t, err) + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, runOpts, + times.shardTimeRanges, nsMD) + defer tester.Finish() + + tester.TestReadWith(src) + indexResults := tester.ResultForNamespace(nsMD.ID()).IndexResult.IndexResults() // Check that single persisted segment got written out infoFiles := fs.ReadIndexInfoFiles(src.fsopts.FilePathPrefix(), testNs1ID, @@ -357,8 +363,6 @@ func TestBootstrapIndexWithPersist(t *testing.T) { require.Equal(t, testShard, infoFile.Info.Shards[0]) } - indexResults := res.IndexResults() - // Check that the segment is not a mutable segment for this block block, ok := indexResults[xtime.ToUnixNano(times.start)] require.True(t, ok) @@ -405,17 +409,19 @@ func TestBootstrapIndexIgnoresPersistConfigIfSnapshotType(t *testing.T) { }) src := newFileSystemSource(opts).(*fileSystemSource) - res, err := src.ReadIndex(testNsMetadata(t), times.shardTimeRanges, - runOpts) - require.NoError(t, err) + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, runOpts, + times.shardTimeRanges, nsMD) + defer tester.Finish() + + tester.TestReadWith(src) + indexResults := tester.ResultForNamespace(nsMD.ID()).IndexResult.IndexResults() // Check that not segments were written out infoFiles := fs.ReadIndexInfoFiles(src.fsopts.FilePathPrefix(), testNs1ID, src.fsopts.InfoReaderBufferSize()) require.Equal(t, 0, len(infoFiles)) - indexResults := res.IndexResults() - // Check that both segments are mutable block, ok := indexResults[xtime.ToUnixNano(times.start)] require.True(t, ok) @@ -436,6 +442,7 @@ func TestBootstrapIndexIgnoresPersistConfigIfSnapshotType(t *testing.T) { counters := scope.Snapshot().Counters() require.Equal(t, int64(0), counters["fs-bootstrapper.persist-index-blocks-read+"].Value()) require.Equal(t, int64(0), counters["fs-bootstrapper.persist-index-blocks-write+"].Value()) + tester.EnsureNoWrites() } func TestBootstrapIndexWithPersistPrefersPersistedIndexBlocks(t *testing.T) { @@ -464,11 +471,13 @@ func TestBootstrapIndexWithPersistPrefersPersistedIndexBlocks(t *testing.T) { SetPersistConfig(bootstrap.PersistConfig{Enabled: true}) src := newFileSystemSource(opts).(*fileSystemSource) - res, err := src.ReadIndex(testNsMetadata(t), times.shardTimeRanges, - runOpts) - require.NoError(t, err) + nsMD := testNsMetadata(t) + tester := bootstrap.BuildNamespacesTester(t, runOpts, + times.shardTimeRanges, nsMD) + defer tester.Finish() - indexResults := res.IndexResults() + tester.TestReadWith(src) + indexResults := tester.ResultForNamespace(nsMD.ID()).IndexResult.IndexResults() // Check that the segment is not a mutable segment for this block // and came from disk @@ -493,6 +502,7 @@ func TestBootstrapIndexWithPersistPrefersPersistedIndexBlocks(t *testing.T) { counters := scope.Snapshot().Counters() require.Equal(t, int64(1), counters["fs-bootstrapper.persist-index-blocks-read+"].Value()) require.Equal(t, int64(0), counters["fs-bootstrapper.persist-index-blocks-write+"].Value()) + tester.EnsureNoWrites() } func TestBootstrapIndexWithPersistForIndexBlockAtRetentionEdge(t *testing.T) { @@ -526,9 +536,12 @@ func TestBootstrapIndexWithPersistForIndexBlockAtRetentionEdge(t *testing.T) { SetBlockSize(testIndexBlockSize))) require.NoError(t, err) - res, err := src.ReadIndex(ns, times.shardTimeRanges, - runOpts) - require.NoError(t, err) + tester := bootstrap.BuildNamespacesTester(t, runOpts, + times.shardTimeRanges, ns) + defer tester.Finish() + + tester.TestReadWith(src) + indexResults := tester.ResultForNamespace(ns.ID()).IndexResult.IndexResults() // Check that single persisted segment got written out infoFiles := fs.ReadIndexInfoFiles(src.fsopts.FilePathPrefix(), testNs1ID, @@ -559,8 +572,6 @@ func TestBootstrapIndexWithPersistForIndexBlockAtRetentionEdge(t *testing.T) { require.Equal(t, testShard, infoFile.Info.Shards[0]) } - indexResults := res.IndexResults() - // Check that the segment is not a mutable segment block, ok := indexResults[xtime.ToUnixNano(firstIndexBlockStart)] require.True(t, ok) @@ -583,4 +594,5 @@ func TestBootstrapIndexWithPersistForIndexBlockAtRetentionEdge(t *testing.T) { counters := scope.Snapshot().Counters() require.Equal(t, int64(0), counters["fs-bootstrapper.persist-index-blocks-read+"].Value()) require.Equal(t, int64(2), counters["fs-bootstrapper.persist-index-blocks-write+"].Value()) + tester.EnsureNoWrites() } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/noop.go b/src/dbnode/storage/bootstrap/bootstrapper/noop.go index 14e3b70c3f..24aad52fd5 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/noop.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/noop.go @@ -21,9 +21,9 @@ package bootstrapper import ( + "fmt" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" ) const ( @@ -60,28 +60,34 @@ func (noop noOpNoneBootstrapper) String() string { return NoOpNoneBootstrapperName } -func (noop noOpNoneBootstrapper) Can(strategy bootstrap.Strategy) bool { - return true -} - -func (noop noOpNoneBootstrapper) BootstrapData( - _ namespace.Metadata, - targetRanges result.ShardTimeRanges, - _ bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - res := result.NewDataBootstrapResult() - res.SetUnfulfilled(targetRanges) - return res, nil -} - -func (noop noOpNoneBootstrapper) BootstrapIndex( - ns namespace.Metadata, - targetRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - res := result.NewIndexBootstrapResult() - res.SetUnfulfilled(targetRanges) - return res, nil +func (noop noOpNoneBootstrapper) Bootstrap( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + results := bootstrap.NewNamespaceResults(namespaces) + for _, elem := range results.Results.Iter() { + id := elem.Key() + namespace := elem.Value() + + requested, ok := namespaces.Namespaces.Get(id) + if !ok { + return bootstrap.NamespaceResults{}, + fmt.Errorf("missing request namespace: %s", id.String()) + } + + // Set everything as unfulfilled. + shardTimeRanges := requested.DataRunOptions.ShardTimeRanges + namespace.DataResult = shardTimeRanges.ToUnfulfilledDataResult() + + if namespace.Metadata.Options().IndexOptions().Enabled() { + shardTimeRanges := requested.IndexRunOptions.ShardTimeRanges + namespace.IndexResult = shardTimeRanges.ToUnfulfilledIndexResult() + } + + // Set value after modifications (map is by value). + results.Results.Set(id, namespace) + } + + return results, nil } // noOpAllBootstrapperProvider is the no-op bootstrapper provider that pretends @@ -110,22 +116,8 @@ func (noop noOpAllBootstrapper) String() string { return NoOpAllBootstrapperName } -func (noop noOpAllBootstrapper) Can(strategy bootstrap.Strategy) bool { - return true -} - -func (noop noOpAllBootstrapper) BootstrapData( - _ namespace.Metadata, - _ result.ShardTimeRanges, - _ bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - return result.NewDataBootstrapResult(), nil -} - -func (noop noOpAllBootstrapper) BootstrapIndex( - _ namespace.Metadata, - _ result.ShardTimeRanges, - _ bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - return result.NewIndexBootstrapResult(), nil +func (noop noOpAllBootstrapper) Bootstrap( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + return bootstrap.NewNamespaceResults(namespaces), nil } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/noop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/noop_test.go index 3e948735f0..776112cb49 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/noop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/noop_test.go @@ -23,25 +23,54 @@ package bootstrapper import ( "testing" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/stretchr/testify/require" ) func TestNoOpNoneBootstrapperBootstrapProvider(t *testing.T) { + testNoOpNoneBootstrapperBootstrapProvider(t, false) +} + +func TestNoOpNoneBootstrapperBootstrapProviderWithIndex(t *testing.T) { + testNoOpNoneBootstrapperBootstrapProvider(t, true) +} + +func testNoOpNoneBootstrapperBootstrapProvider(t *testing.T, indexEnabled bool) { bs := NewNoOpNoneBootstrapperProvider() ranges := testShardTimeRanges() bootstrapper, err := bs.Provide() require.NoError(t, err) - res, err := bootstrapper.BootstrapData(testNsMetadata(t), ranges, testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, ranges, res.Unfulfilled()) + mds := namespace.MustBuildMetadatas(indexEnabled, "foo", "bar") + opts := bootstrap.NewRunOptions() + ns := bootstrap.BuildNamespacesTester(t, opts, ranges, mds...) + defer ns.Finish() + ns.TestBootstrapWith(bootstrapper) + for _, md := range mds { + ns.TestUnfulfilledForNamespace(md, ranges, ranges) + } } func TestNoOpAllBootstrapperBootstrapProvider(t *testing.T) { + testNoOpAllBootstrapperBootstrapProvider(t, false) +} + +func TestNoOpAllBootstrapperBootstrapProviderWithIndex(t *testing.T) { + testNoOpAllBootstrapperBootstrapProvider(t, true) +} + +func testNoOpAllBootstrapperBootstrapProvider(t *testing.T, indexEnabled bool) { bs := NewNoOpAllBootstrapperProvider() ranges := testShardTimeRanges() bootstrapper, err := bs.Provide() require.NoError(t, err) - res, err := bootstrapper.BootstrapData(testNsMetadata(t), ranges, testDefaultRunOpts) - require.NoError(t, err) - require.True(t, res.Unfulfilled().IsEmpty()) + mds := namespace.MustBuildMetadatas(indexEnabled, "foo", "bar") + opts := bootstrap.NewRunOptions() + ns := bootstrap.BuildNamespacesTester(t, opts, ranges, mds...) + defer ns.Finish() + ns.TestBootstrapWith(bootstrapper) + for _, md := range mds { + ns.TestUnfulfilledForNamespaceIsEmpty(md) + } } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/source.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/source.go index 35126cdc01..d2504c2082 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/source.go @@ -36,6 +36,8 @@ import ( "github.com/m3db/m3/src/dbnode/storage/index/convert" "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/topology" + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" @@ -58,21 +60,14 @@ type persistenceFlush struct { } func newPeersSource(opts Options) (bootstrap.Source, error) { + iopts := opts.ResultOptions().InstrumentOptions() return &peersSource{ opts: opts, - log: opts.ResultOptions().InstrumentOptions().Logger(), + log: iopts.Logger().With(zap.String("bootstrapper", "peers")), nowFn: opts.ResultOptions().ClockOptions().NowFn(), }, nil } -func (s *peersSource) Can(strategy bootstrap.Strategy) bool { - switch strategy { - case bootstrap.BootstrapSequential: - return true - } - return false -} - type shardPeerAvailability struct { numPeers int numAvailablePeers int @@ -89,8 +84,89 @@ func (s *peersSource) AvailableData( return s.peerAvailability(nsMetadata, shardsTimeRanges, runOpts) } -func (s *peersSource) ReadData( +func (s *peersSource) AvailableIndex( + nsMetadata namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + runOpts bootstrap.RunOptions, +) (result.ShardTimeRanges, error) { + if err := s.validateRunOpts(runOpts); err != nil { + return nil, err + } + return s.peerAvailability(nsMetadata, shardsTimeRanges, runOpts) +} + +func (s *peersSource) Read( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + results := bootstrap.NamespaceResults{ + Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}), + } + + // NB(r): Perform all data bootstrapping first then index bootstrapping + // to more clearly deliniate which process is slower than the other. + nowFn := s.opts.ResultOptions().ClockOptions().NowFn() + start := nowFn() + s.log.Info("bootstrapping time series data start") + for _, elem := range namespaces.Namespaces.Iter() { + namespace := elem.Value() + md := namespace.Metadata + + r, err := s.readData(md, namespace.DataAccumulator, + namespace.DataRunOptions.ShardTimeRanges, + namespace.DataRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + results.Results.Set(md.ID(), bootstrap.NamespaceResult{ + Metadata: md, + Shards: namespace.Shards, + DataResult: r, + }) + } + s.log.Info("bootstrapping time series data success", + zap.Duration("took", nowFn().Sub(start))) + + start = nowFn() + s.log.Info("bootstrapping index metadata start") + for _, elem := range namespaces.Namespaces.Iter() { + namespace := elem.Value() + md := namespace.Metadata + if !md.Options().IndexOptions().Enabled() { + s.log.Info("skipping bootstrap for namespace based on options", + zap.Stringer("namespace", md.ID())) + + // Not bootstrapping for index. + continue + } + + r, err := s.readIndex(md, + namespace.IndexRunOptions.ShardTimeRanges, + namespace.IndexRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } + + result, ok := results.Results.Get(md.ID()) + if !ok { + err = fmt.Errorf("missing expected result for namespace: %s", + md.ID().String()) + return bootstrap.NamespaceResults{}, err + } + + result.IndexResult = r + + results.Results.Set(md.ID(), result) + } + s.log.Info("bootstrapping index metadata success", + zap.Duration("took", nowFn().Sub(start))) + + return results, nil +} + +func (s *peersSource) readData( nsMetadata namespace.Metadata, + accumulator bootstrap.NamespaceDataAccumulator, shardsTimeRanges result.ShardTimeRanges, opts bootstrap.RunOptions, ) (result.DataBootstrapResult, error) { @@ -110,6 +186,7 @@ func (s *peersSource) ReadData( seriesCachePolicy = s.opts.ResultOptions().SeriesCachePolicy() persistConfig = opts.PersistConfig() ) + if persistConfig.Enabled && (seriesCachePolicy == series.CacheRecentlyRead || seriesCachePolicy == series.CacheLRU) && persistConfig.FileSetType == persist.FileSetFlushType { @@ -184,8 +261,8 @@ func (s *peersSource) ReadData( workers.Go(func() { defer wg.Done() s.fetchBootstrapBlocksFromPeers(shard, ranges, nsMetadata, session, - resultOpts, result, &resultLock, shouldPersist, persistenceQueue, - shardRetrieverMgr, blockSize) + accumulator, resultOpts, result, &resultLock, shouldPersist, + persistenceQueue, shardRetrieverMgr, blockSize) }) } @@ -218,10 +295,6 @@ func (s *peersSource) startPersistenceQueueWorkerLoop( err := s.flush(opts, persistFlush, flush.nsMetadata, flush.shard, flush.shardRetrieverMgr, flush.shardResult, flush.timeRange) if err == nil { - // Safe to add to the shared bootstrap result now. - lock.Lock() - bootstrapResult.Add(flush.shard, flush.shardResult, xtime.Ranges{}) - lock.Unlock() continue } @@ -231,7 +304,11 @@ func (s *peersSource) startPersistenceQueueWorkerLoop( // Make unfulfilled. lock.Lock() - bootstrapResult.Add(flush.shard, nil, xtime.NewRanges(flush.timeRange)) + unfulfilled := bootstrapResult.Unfulfilled().Copy() + unfulfilled.AddRanges(result.ShardTimeRanges{ + flush.shard: xtime.NewRanges(flush.timeRange), + }) + bootstrapResult.SetUnfulfilled(unfulfilled) lock.Unlock() } close(doneCh) @@ -248,6 +325,7 @@ func (s *peersSource) fetchBootstrapBlocksFromPeers( ranges xtime.Ranges, nsMetadata namespace.Metadata, session client.AdminSession, + accumulator bootstrap.NamespaceDataAccumulator, bopts result.Options, bootstrapResult result.DataBootstrapResult, lock *sync.Mutex, @@ -257,6 +335,13 @@ func (s *peersSource) fetchBootstrapBlocksFromPeers( blockSize time.Duration, ) { it := ranges.Iter() + tagsIter := ident.NewTagsIterator(ident.Tags{}) + unfulfill := func(r xtime.Range) { + lock.Lock() + unfulfilled := bootstrapResult.Unfulfilled() + unfulfilled.AddRanges(result.ShardTimeRanges{shard: xtime.NewRanges(r)}) + lock.Unlock() + } for it.Next() { currRange := it.Value() @@ -264,14 +349,11 @@ func (s *peersSource) fetchBootstrapBlocksFromPeers( blockEnd := blockStart.Add(blockSize) shardResult, err := session.FetchBootstrapBlocksFromPeers( nsMetadata, shard, blockStart, blockEnd, bopts) - s.logFetchBootstrapBlocksFromPeersOutcome(shard, shardResult, err) if err != nil { - // Do not add result at all to the bootstrap result - lock.Lock() - bootstrapResult.Add(shard, nil, xtime.NewRanges(currRange)) - lock.Unlock() + // No result to add for this bootstrap. + unfulfill(currRange) continue } @@ -286,10 +368,28 @@ func (s *peersSource) fetchBootstrapBlocksFromPeers( continue } - // If not waiting to flush, add straight away to bootstrap result - lock.Lock() - bootstrapResult.Add(shard, shardResult, xtime.Ranges{}) - lock.Unlock() + // If not waiting to flush, add straight away to bootstrap result. + for _, elem := range shardResult.AllSeries().Iter() { + entry := elem.Value() + tagsIter.Reset(entry.Tags) + ref, err := accumulator.CheckoutSeriesWithLock(shard, entry.ID, tagsIter) + if err != nil { + unfulfill(currRange) + s.log.Error("could not checkout series", zap.Error(err)) + continue + } + + for _, block := range entry.Blocks.AllBlocks() { + if err := ref.Series.LoadBlock(block, series.WarmWrite); err != nil { + unfulfill(currRange) + s.log.Error("could not load series block", zap.Error(err)) + } + } + + // Safe to finalize these IDs and Tags, shard result no longer used. + entry.ID.Finalize() + entry.Tags.Finalize() + } } } } @@ -316,7 +416,7 @@ func (s *peersSource) logFetchBootstrapBlocksFromPeersOutcome( ) } } else { - s.log.Error("error fetching bootstrap blocks from peers", + s.log.Error("error fetching bootstrap blocks", zap.Uint32("shard", shard), zap.Error(err), ) @@ -505,18 +605,7 @@ func (s *peersSource) flush( return nil } -func (s *peersSource) AvailableIndex( - nsMetadata namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.ShardTimeRanges, error) { - if err := s.validateRunOpts(runOpts); err != nil { - return nil, err - } - return s.peerAvailability(nsMetadata, shardsTimeRanges, runOpts) -} - -func (s *peersSource) ReadIndex( +func (s *peersSource) readIndex( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, opts bootstrap.RunOptions, @@ -586,7 +675,6 @@ func (s *peersSource) ReadIndex( markUnfulfilled(err) continue } - for metadata.Next() { _, dataBlock := metadata.Current() diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_data_test.go index d071ce5ef7..c0ef037198 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_data_test.go @@ -21,6 +21,7 @@ package peers import ( + "errors" "fmt" "testing" "time" @@ -53,6 +54,8 @@ var ( for _, opt := range opts { namespaceOpts = opt(namespaceOpts) } + idxOpts := namespaceOpts.IndexOptions() + namespaceOpts = namespaceOpts.SetIndexOptions(idxOpts.SetEnabled(true)) ns, err := namespace.NewMetadata(testNamespace, namespaceOpts) require.NoError(t, err) return ns @@ -105,17 +108,6 @@ func newValidMockRuntimeOptionsManager(t *testing.T, ctrl *gomock.Controller) m3 type namespaceOption func(namespace.Options) namespace.Options -func TestPeersSourceCan(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - src, err := newPeersSource(newTestDefaultOpts(t, ctrl)) - require.NoError(t, err) - - assert.True(t, src.Can(bootstrap.BootstrapSequential)) - assert.False(t, src.Can(bootstrap.BootstrapParallel)) -} - func TestPeersSourceEmptyShardTimeRanges(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -127,18 +119,20 @@ func TestPeersSourceEmptyShardTimeRanges(t *testing.T) { require.NoError(t, err) var ( - nsMetdata = testNamespaceMetadata(t) - target = result.ShardTimeRanges{} - runOpts = testDefaultRunOpts.SetInitialTopologyState(&topology.StateSnapshot{}) + nsMetadata = testNamespaceMetadata(t) + target = result.ShardTimeRanges{} + runOpts = testDefaultRunOpts.SetInitialTopologyState(&topology.StateSnapshot{}) ) - available, err := src.AvailableData(nsMetdata, target, runOpts) + available, err := src.AvailableData(nsMetadata, target, runOpts) require.NoError(t, err) require.Equal(t, target, available) - r, err := src.ReadData(nsMetdata, target, testDefaultRunOpts) - require.NoError(t, err) - require.Equal(t, 0, len(r.ShardResults())) - require.True(t, r.Unfulfilled().IsEmpty()) + tester := bootstrap.BuildNamespacesTester(t, runOpts, target, nsMetadata) + defer tester.Finish() + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(nsMetadata) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func TestPeersSourceReturnsErrorForAdminSession(t *testing.T) { @@ -148,7 +142,7 @@ func TestPeersSourceReturnsErrorForAdminSession(t *testing.T) { nsMetadata := testNamespaceMetadata(t) ropts := nsMetadata.Options().RetentionOptions() - expectedErr := fmt.Errorf("an error") + expectedErr := errors.New("an error") mockAdminClient := client.NewMockAdminClient(ctrl) mockAdminClient.EXPECT().DefaultAdminSession().Return(nil, expectedErr) @@ -165,12 +159,16 @@ func TestPeersSourceReturnsErrorForAdminSession(t *testing.T) { 1: xtime.NewRanges(xtime.Range{Start: start, End: end}), } - _, err = src.ReadData(nsMetadata, target, testDefaultRunOpts) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, target, nsMetadata) + defer tester.Finish() + _, err = src.Read(tester.Namespaces) require.Error(t, err) assert.Equal(t, expectedErr, err) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } -func TestPeersSourceReturnsFulfilledAndUnfulfilled(t *testing.T) { +func TestPeersSourceReturnsUnfulfilled(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -183,21 +181,25 @@ func TestPeersSourceReturnsFulfilledAndUnfulfilled(t *testing.T) { goodResult := result.NewShardResult(0, opts.ResultOptions()) fooBlock := block.NewDatabaseBlock(start, ropts.BlockSize(), ts.Segment{}, testBlockOpts, namespace.Context{}) - goodResult.AddBlock(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) - badErr := fmt.Errorf("an error") + goodID := ident.StringID("foo") + goodResult.AddBlock(goodID, ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) mockAdminSession := client.NewMockAdminSession(ctrl) mockAdminSession.EXPECT(). FetchBootstrapBlocksFromPeers(namespace.NewMetadataMatcher(nsMetadata), uint32(0), start, end, gomock.Any()). Return(goodResult, nil) + + peerMetaIter := client.NewMockPeerBlockMetadataIter(ctrl) + peerMetaIter.EXPECT().Next().Return(false).AnyTimes() + peerMetaIter.EXPECT().Err().Return(nil).AnyTimes() mockAdminSession.EXPECT(). - FetchBootstrapBlocksFromPeers(namespace.NewMetadataMatcher(nsMetadata), - uint32(1), start, end, gomock.Any()). - Return(nil, badErr) + FetchBootstrapBlocksMetadataFromPeers(gomock.Any(), gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any()). + Return(peerMetaIter, nil).AnyTimes() mockAdminClient := client.NewMockAdminClient(ctrl) - mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil) + mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil).AnyTimes() opts = opts.SetAdminClient(mockAdminClient) @@ -206,28 +208,22 @@ func TestPeersSourceReturnsFulfilledAndUnfulfilled(t *testing.T) { target := result.ShardTimeRanges{ 0: xtime.NewRanges(xtime.Range{Start: start, End: end}), - 1: xtime.NewRanges(xtime.Range{Start: start, End: end}), } - r, err := src.ReadData(nsMetadata, target, testDefaultRunOpts) - assert.NoError(t, err) - - assert.Equal(t, 1, len(r.ShardResults())) - require.NotNil(t, r.ShardResults()[0]) - require.Nil(t, r.ShardResults()[1]) - - require.True(t, r.Unfulfilled()[0].IsEmpty()) - require.False(t, r.Unfulfilled()[1].IsEmpty()) - require.Equal(t, 1, r.Unfulfilled()[1].Len()) - - block, ok := r.ShardResults()[0].BlockAt(ident.StringID("foo"), start) - require.True(t, ok) - require.Equal(t, fooBlock, block) - - rangeIter := r.Unfulfilled()[1].Iter() - require.True(t, rangeIter.Next()) - require.Equal(t, xtime.Range{Start: start, End: end}, rangeIter.Value()) - require.Equal(t, ropts.BlockSize(), block.BlockSize()) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, target, nsMetadata) + defer tester.Finish() + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(nsMetadata) + vals := tester.DumpLoadedBlocks() + assert.Equal(t, 1, len(vals)) + series, found := vals[nsMetadata.ID().String()] + require.True(t, found) + + assert.Equal(t, 1, len(series)) + points, found := series[goodID.String()] + require.True(t, found) + assert.Equal(t, 0, len(points)) + tester.EnsureNoWrites() } func TestPeersSourceRunWithPersist(t *testing.T) { @@ -242,6 +238,7 @@ func TestPeersSourceRunWithPersist(t *testing.T) { resultOpts := testDefaultResultOpts.SetSeriesCachePolicy(cachePolicy) opts := testDefaultOpts.SetResultOptions(resultOpts) ropts := testNsMd.Options().RetentionOptions() + testNsMd.Options() blockSize := ropts.BlockSize() start := time.Now().Add(-ropts.RetentionPeriod()).Truncate(ropts.BlockSize()) @@ -283,8 +280,16 @@ func TestPeersSourceRunWithPersist(t *testing.T) { uint32(1), start.Add(blockSize), start.Add(blockSize*2), gomock.Any()). Return(shard1ResultBlock2, nil) + peerMetaIter := client.NewMockPeerBlockMetadataIter(ctrl) + peerMetaIter.EXPECT().Next().Return(false).AnyTimes() + peerMetaIter.EXPECT().Err().Return(nil).AnyTimes() + mockAdminSession.EXPECT(). + FetchBootstrapBlocksMetadataFromPeers(gomock.Any(), gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any()). + Return(peerMetaIter, nil).AnyTimes() + mockAdminClient := client.NewMockAdminClient(ctrl) - mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil) + mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil).AnyTimes() opts = opts.SetAdminClient(mockAdminClient) @@ -396,16 +401,10 @@ func TestPeersSourceRunWithPersist(t *testing.T) { 1: xtime.NewRanges(xtime.Range{Start: start, End: end}), } - r, err := src.ReadData(testNsMd, target, testRunOptsWithPersist) - assert.NoError(t, err) - - require.True(t, r.Unfulfilled()[0].IsEmpty()) - require.True(t, r.Unfulfilled()[1].IsEmpty()) - - assert.Equal(t, 0, len(r.ShardResults())) - require.Nil(t, r.ShardResults()[0]) - require.Nil(t, r.ShardResults()[1]) - + tester := bootstrap.BuildNamespacesTester(t, testRunOptsWithPersist, target, testNsMd) + defer tester.Finish() + tester.TestReadWith(src) + tester.TestUnfulfilledForNamespaceIsEmpty(testNsMd) assert.Equal(t, map[string]int{ "foo": 1, "bar": 1, "baz": 1, }, persists) @@ -413,6 +412,9 @@ func TestPeersSourceRunWithPersist(t *testing.T) { assert.Equal(t, map[string]int{ "foo": 1, "bar": 1, "baz": 1, "empty": 1, }, closes) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } } @@ -433,35 +435,39 @@ func TestPeersSourceMarksUnfulfilledOnPersistenceErrors(t *testing.T) { end := start.Add(2 * ropts.BlockSize()) type resultsKey struct { - shard uint32 - start int64 - end int64 + shard uint32 + start int64 + end int64 + expectedErr bool } results := make(map[resultsKey]result.ShardResult) - addResult := func(shard uint32, id string, b block.DatabaseBlock) { + addResult := func(shard uint32, id string, b block.DatabaseBlock, expectedErr bool) { r := result.NewShardResult(0, opts.ResultOptions()) r.AddBlock(ident.StringID(id), ident.NewTags(ident.StringTag(id, id)), b) start := b.StartTime() end := start.Add(ropts.BlockSize()) - results[resultsKey{shard, start.UnixNano(), end.UnixNano()}] = r + results[resultsKey{shard, start.UnixNano(), end.UnixNano(), expectedErr}] = r } + segmentError := errors.New("segment err") + // foo results var fooBlocks [2]block.DatabaseBlock fooBlocks[0] = block.NewMockDatabaseBlock(ctrl) fooBlocks[0].(*block.MockDatabaseBlock).EXPECT().StartTime().Return(start).AnyTimes() - fooBlocks[0].(*block.MockDatabaseBlock).EXPECT().Stream(gomock.Any()).Return(xio.EmptyBlockReader, fmt.Errorf("stream err")) - addResult(0, "foo", fooBlocks[0]) + fooBlocks[0].(*block.MockDatabaseBlock).EXPECT(). + Stream(gomock.Any()).Return(xio.EmptyBlockReader, errors.New("stream err")) + addResult(0, "foo", fooBlocks[0], true) fooBlocks[1] = block.NewDatabaseBlock(midway, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{1, 2, 3}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(0, "foo", fooBlocks[1]) + addResult(0, "foo", fooBlocks[1], false) // bar results mockStream := xio.NewMockSegmentReader(ctrl) - mockStream.EXPECT().Segment().Return(ts.Segment{}, fmt.Errorf("segment err")) + mockStream.EXPECT().Segment().Return(ts.Segment{}, segmentError) b := xio.BlockReader{ SegmentReader: mockStream, @@ -471,36 +477,36 @@ func TestPeersSourceMarksUnfulfilledOnPersistenceErrors(t *testing.T) { barBlocks[0] = block.NewMockDatabaseBlock(ctrl) barBlocks[0].(*block.MockDatabaseBlock).EXPECT().StartTime().Return(start).AnyTimes() barBlocks[0].(*block.MockDatabaseBlock).EXPECT().Stream(gomock.Any()).Return(b, nil) - addResult(1, "bar", barBlocks[0]) + addResult(1, "bar", barBlocks[0], false) barBlocks[1] = block.NewDatabaseBlock(midway, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{4, 5, 6}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(1, "bar", barBlocks[1]) + addResult(1, "bar", barBlocks[1], false) // baz results var bazBlocks [2]block.DatabaseBlock bazBlocks[0] = block.NewDatabaseBlock(start, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{7, 8, 9}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(2, "baz", bazBlocks[0]) + addResult(2, "baz", bazBlocks[0], false) bazBlocks[1] = block.NewDatabaseBlock(midway, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{10, 11, 12}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(2, "baz", bazBlocks[1]) + addResult(2, "baz", bazBlocks[1], false) // qux results var quxBlocks [2]block.DatabaseBlock quxBlocks[0] = block.NewDatabaseBlock(start, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{13, 14, 15}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(3, "qux", quxBlocks[0]) + addResult(3, "qux", quxBlocks[0], false) quxBlocks[1] = block.NewDatabaseBlock(midway, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{16, 17, 18}, nil), nil, ts.FinalizeNone), testBlockOpts, namespace.Context{}) - addResult(3, "qux", quxBlocks[1]) + addResult(3, "qux", quxBlocks[1], false) mockAdminSession := client.NewMockAdminSession(ctrl) @@ -510,10 +516,24 @@ func TestPeersSourceMarksUnfulfilledOnPersistenceErrors(t *testing.T) { key.shard, time.Unix(0, key.start), time.Unix(0, key.end), gomock.Any()). Return(result, nil) + + peerError := segmentError + if !key.expectedErr { + peerError = nil + } + + peerMetaIter := client.NewMockPeerBlockMetadataIter(ctrl) + peerMetaIter.EXPECT().Next().Return(false).AnyTimes() + peerMetaIter.EXPECT().Err().Return(nil).AnyTimes() + mockAdminSession.EXPECT(). + FetchBootstrapBlocksMetadataFromPeers(testNsMd.ID(), + key.shard, time.Unix(0, key.start), time.Unix(0, key.end), gomock.Any()). + Return(peerMetaIter, peerError).AnyTimes() } mockAdminClient := client.NewMockAdminClient(ctrl) - mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil) + mockAdminClient.EXPECT().DefaultAdminSession(). + Return(mockAdminSession, nil).AnyTimes() opts = opts.SetAdminClient(mockAdminClient) @@ -706,18 +726,22 @@ func TestPeersSourceMarksUnfulfilledOnPersistenceErrors(t *testing.T) { AddRange(xtime.Range{Start: midway, End: end}), } - r, err := src.ReadData(testNsMd, target, testRunOptsWithPersist) - assert.NoError(t, err) + tester := bootstrap.BuildNamespacesTester(t, testRunOptsWithPersist, target, testNsMd) + defer tester.Finish() + tester.TestReadWith(src) - assert.Equal(t, 0, len(r.ShardResults())) - for i := uint32(0); i < uint32(len(target)); i++ { - require.False(t, r.Unfulfilled()[i].IsEmpty()) - require.Equal(t, xtime.NewRanges(xtime.Range{ - Start: start, - End: midway, - }).String(), r.Unfulfilled()[i].String()) + expectedRanges := result.ShardTimeRanges{ + 0: xtime.Ranges{}.AddRange(xtime.Range{Start: start, End: midway}), + 1: xtime.Ranges{}.AddRange(xtime.Range{Start: start, End: midway}), + 2: xtime.Ranges{}.AddRange(xtime.Range{Start: start, End: midway}), + 3: xtime.Ranges{}.AddRange(xtime.Range{Start: start, End: midway}), } + expectedIndexRanges := result.ShardTimeRanges{ + 0: xtime.Ranges{}.AddRange(xtime.Range{Start: start, End: midway}), + } + + tester.TestUnfulfilledForNamespace(testNsMd, expectedRanges, expectedIndexRanges) assert.Equal(t, map[string]int{ "foo": 1, "bar": 1, "baz": 2, "qux": 2, }, persists) @@ -725,6 +749,9 @@ func TestPeersSourceMarksUnfulfilledOnPersistenceErrors(t *testing.T) { assert.Equal(t, map[string]int{ "foo": 2, "bar": 2, "baz": 2, "qux": 2, }, closes) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } func assertBlockChecksum(t *testing.T, expectedChecksum uint32, block block.DatabaseBlock) { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_index_test.go index f582ebd33c..b3bb5109d4 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_index_test.go @@ -27,10 +27,12 @@ import ( "time" "github.com/m3db/m3/src/dbnode/client" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/block" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -149,6 +151,24 @@ func TestBootstrapIndex(t *testing.T) { mockAdminSession := client.NewMockAdminSession(ctrl) mockAdminSessionCalls := []*gomock.Call{} + mockAdminSession.EXPECT(). + FetchBootstrapBlocksFromPeers(gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func( + _ namespace.Metadata, + _ uint32, + _ time.Time, + _ time.Time, + _ result.Options, + ) (result.ShardResult, error) { + goodID := ident.StringID("foo") + goodResult := result.NewShardResult(0, opts.ResultOptions()) + fooBlock := block.NewDatabaseBlock(start, ropts.BlockSize(), + ts.Segment{}, testBlockOpts, namespace.Context{}) + goodResult.AddBlock(goodID, ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) + return goodResult, nil + }).AnyTimes() + for blockStart := start; blockStart.Before(end); blockStart = blockStart.Add(blockSize) { // Find and expect calls for blocks matchedBlock := false @@ -209,17 +229,19 @@ func TestBootstrapIndex(t *testing.T) { gomock.InOrder(mockAdminSessionCalls...) mockAdminClient := client.NewMockAdminClient(ctrl) - mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil) + mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil).AnyTimes() opts = opts.SetAdminClient(mockAdminClient) src, err := newPeersSource(opts) require.NoError(t, err) - res, err := src.ReadIndex(nsMetadata, shardTimeRanges, - testDefaultRunOpts) - require.NoError(t, err) + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, shardTimeRanges, nsMetadata) + defer tester.Finish() + tester.TestReadWith(src) - indexResults := res.IndexResults() + tester.TestUnfulfilledForNamespaceIsEmpty(nsMetadata) + results := tester.ResultForNamespace(nsMetadata.ID()) + indexResults := results.IndexResult.IndexResults() numBlocksWithData := 0 for _, b := range indexResults { if len(b.Segments()) != 0 { @@ -297,15 +319,15 @@ func TestBootstrapIndex(t *testing.T) { t2 := indexStart.Add(indexBlockSize) t3 := t2.Add(indexBlockSize) - blk1, ok := res.IndexResults()[xtime.ToUnixNano(t1)] + blk1, ok := indexResults[xtime.ToUnixNano(t1)] require.True(t, ok) assertShardRangesEqual(t, result.NewShardTimeRanges(t1, t2, 0), blk1.Fulfilled()) - blk2, ok := res.IndexResults()[xtime.ToUnixNano(t2)] + blk2, ok := indexResults[xtime.ToUnixNano(t2)] require.True(t, ok) assertShardRangesEqual(t, result.NewShardTimeRanges(t2, t3, 0), blk2.Fulfilled()) - for _, blk := range res.IndexResults() { + for _, blk := range indexResults { if blk.BlockStart().Equal(t1) || blk.BlockStart().Equal(t2) { continue // already checked above } @@ -315,6 +337,8 @@ func TestBootstrapIndex(t *testing.T) { end := start.Add(indexBlockSize) assertShardRangesEqual(t, result.NewShardTimeRanges(start, end, 0), blk.Fulfilled()) } + + tester.EnsureNoWrites() } func TestBootstrapIndexErr(t *testing.T) { @@ -386,6 +410,24 @@ func TestBootstrapIndexErr(t *testing.T) { mockAdminSession := client.NewMockAdminSession(ctrl) mockAdminSessionCalls := []*gomock.Call{} + mockAdminSession.EXPECT(). + FetchBootstrapBlocksFromPeers(gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func( + _ namespace.Metadata, + _ uint32, + _ time.Time, + _ time.Time, + _ result.Options, + ) (result.ShardResult, error) { + goodID := ident.StringID("foo") + goodResult := result.NewShardResult(0, opts.ResultOptions()) + fooBlock := block.NewDatabaseBlock(start, ropts.BlockSize(), + ts.Segment{}, testBlockOpts, namespace.Context{}) + goodResult.AddBlock(goodID, ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) + return goodResult, nil + }).AnyTimes() + for blockStart := start; blockStart.Before(end); blockStart = blockStart.Add(blockSize) { // Find and expect calls for blocks matchedBlock := false @@ -446,16 +488,20 @@ func TestBootstrapIndexErr(t *testing.T) { gomock.InOrder(mockAdminSessionCalls...) mockAdminClient := client.NewMockAdminClient(ctrl) - mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil).Times(1) + mockAdminClient.EXPECT().DefaultAdminSession().Return(mockAdminSession, nil).AnyTimes() opts = opts.SetAdminClient(mockAdminClient) src, err := newPeersSource(opts) require.NoError(t, err) - res, err := src.ReadIndex(nsMetadata, shardTimeRanges, testDefaultRunOpts) - require.NoError(t, err) - indexResults := res.IndexResults() + tester := bootstrap.BuildNamespacesTester(t, testDefaultRunOpts, shardTimeRanges, nsMetadata) + defer tester.Finish() + tester.TestReadWith(src) + + tester.TestUnfulfilledForNamespaceIsEmpty(nsMetadata) + results := tester.ResultForNamespace(nsMetadata.ID()) + indexResults := results.IndexResult.IndexResults() numBlocksWithData := 0 for _, b := range indexResults { if len(b.Segments()) != 0 { @@ -466,11 +512,11 @@ func TestBootstrapIndexErr(t *testing.T) { t1 := indexStart - blk1, ok := res.IndexResults()[xtime.ToUnixNano(t1)] + blk1, ok := indexResults[xtime.ToUnixNano(t1)] require.True(t, ok) require.True(t, blk1.Fulfilled().IsEmpty()) - for _, blk := range res.IndexResults() { + for _, blk := range indexResults { if blk.BlockStart().Equal(t1) { continue // already checked above } @@ -480,6 +526,8 @@ func TestBootstrapIndexErr(t *testing.T) { end := start.Add(indexBlockSize) assertShardRangesEqual(t, result.NewShardTimeRanges(start, end, 0), blk.Fulfilled()) } + + tester.EnsureNoWrites() } func assertShardRangesEqual(t *testing.T, a, b result.ShardTimeRanges) { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_test.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_test.go index 562cc6158a..cf57d7cfad 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/source_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/source_test.go @@ -195,11 +195,11 @@ func TestPeersSourceReturnsErrorIfUnknownPersistenceFileSetType(t *testing.T) { } runOpts := testRunOptsWithPersist.SetPersistConfig(bootstrap.PersistConfig{Enabled: true, FileSetType: 999}) - _, err = src.ReadData(testNsMd, target, runOpts) - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "unknown persist config fileset file type")) - - _, err = src.ReadIndex(testNsMd, target, runOpts) + tester := bootstrap.BuildNamespacesTester(t, runOpts, target, testNsMd) + defer tester.Finish() + _, err = src.Read(tester.Namespaces) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "unknown persist config fileset file type")) + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source.go b/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source.go index 902b77ac92..c8dfa698fb 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source.go @@ -24,9 +24,9 @@ import ( "fmt" "github.com/m3db/m3/src/cluster/shard" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/topology" ) @@ -50,15 +50,6 @@ func newTopologyUninitializedSource(opts Options) bootstrap.Source { } } -func (s *uninitializedTopologySource) Can(strategy bootstrap.Strategy) bool { - switch strategy { - case bootstrap.BootstrapSequential: - return true - } - - return false -} - func (s *uninitializedTopologySource) AvailableData( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -142,42 +133,45 @@ func (s *uninitializedTopologySource) availability( return availableShardTimeRanges, nil } -func (s *uninitializedTopologySource) ReadData( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { - availability, err := s.availability(ns, shardsTimeRanges, runOpts) - if err != nil { - return nil, err +func (s *uninitializedTopologySource) Read( + namespaces bootstrap.Namespaces, +) (bootstrap.NamespaceResults, error) { + results := bootstrap.NamespaceResults{ + Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}), } + for _, elem := range namespaces.Namespaces.Iter() { + ns := elem.Value() - missing := shardsTimeRanges.Copy() - missing.Subtract(availability) + namespaceResult := bootstrap.NamespaceResult{ + Metadata: ns.Metadata, + Shards: ns.Shards, + } - if missing.IsEmpty() { - return result.NewDataBootstrapResult(), nil - } + availability, err := s.availability(ns.Metadata, + ns.DataRunOptions.ShardTimeRanges, ns.DataRunOptions.RunOptions) + if err != nil { + return bootstrap.NamespaceResults{}, err + } - return missing.ToUnfulfilledDataResult(), nil -} + missing := ns.DataRunOptions.ShardTimeRanges.Copy() + missing.Subtract(availability) -func (s *uninitializedTopologySource) ReadIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - availability, err := s.availability(ns, shardsTimeRanges, runOpts) - if err != nil { - return nil, err - } + if missing.IsEmpty() { + namespaceResult.DataResult = result.NewDataBootstrapResult() + } else { + namespaceResult.DataResult = missing.ToUnfulfilledDataResult() + } - missing := shardsTimeRanges.Copy() - missing.Subtract(availability) + if ns.Metadata.Options().IndexOptions().Enabled() { + if missing.IsEmpty() { + namespaceResult.IndexResult = result.NewIndexBootstrapResult() + } else { + namespaceResult.IndexResult = missing.ToUnfulfilledIndexResult() + } + } - if missing.IsEmpty() { - return result.NewIndexBootstrapResult(), nil + results.Results.Set(ns.Metadata.ID(), namespaceResult) } - return missing.ToUnfulfilledIndexResult(), nil + return results, nil } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source_test.go b/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source_test.go index 33b6b113ee..ec3a8a3553 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source_test.go @@ -26,9 +26,9 @@ import ( "time" "github.com/m3db/m3/src/cluster/shard" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/topology" tu "github.com/m3db/m3/src/dbnode/topology/testutil" "github.com/m3db/m3/src/x/ident" @@ -57,7 +57,9 @@ func TestUnitializedTopologySourceAvailableDataAndAvailableIndex(t *testing.T) { End: blockStart.Add(blockSize), }) ) - nsMetadata, err := namespace.NewMetadata(testNamespaceID, namespace.NewOptions()) + nsOpts := namespace.NewOptions() + nsOpts = nsOpts.SetIndexOptions(nsOpts.IndexOptions().SetEnabled(true)) + nsMetadata, err := namespace.NewMetadata(testNamespaceID, nsOpts) require.NoError(t, err) for i := 0; i < int(numShards); i++ { @@ -194,19 +196,23 @@ func TestUnitializedTopologySourceAvailableDataAndAvailableIndex(t *testing.T) { require.Equal(t, tc.expectedAvailableShardsTimeRanges, dataAvailabilityResult) require.Equal(t, tc.expectedAvailableShardsTimeRanges, indexAvailabilityResult) - // Make sure ReadData marks anything that AvailableData wouldn't return as unfulfilled - dataResult, err := src.ReadData(nsMetadata, tc.shardsTimeRangesToBootstrap, runOpts) - require.NoError(t, err) + // Make sure Read marks anything that available ranges wouldn't return as unfulfilled + tester := bootstrap.BuildNamespacesTester(t, runOpts, tc.shardsTimeRangesToBootstrap, nsMetadata) + defer tester.Finish() + tester.TestReadWith(src) + expectedDataUnfulfilled := tc.shardsTimeRangesToBootstrap.Copy() expectedDataUnfulfilled.Subtract(tc.expectedAvailableShardsTimeRanges) - require.Equal(t, expectedDataUnfulfilled, dataResult.Unfulfilled()) - - // Make sure ReadIndex marks anything that AvailableIndex wouldn't return as unfulfilled - indexResult, err := src.ReadIndex(nsMetadata, tc.shardsTimeRangesToBootstrap, runOpts) - require.NoError(t, err) expectedIndexUnfulfilled := tc.shardsTimeRangesToBootstrap.Copy() expectedIndexUnfulfilled.Subtract(tc.expectedAvailableShardsTimeRanges) - require.Equal(t, expectedIndexUnfulfilled, indexResult.Unfulfilled()) + tester.TestUnfulfilledForNamespace( + nsMetadata, + expectedDataUnfulfilled, + expectedIndexUnfulfilled, + ) + + tester.EnsureNoLoadedBlocks() + tester.EnsureNoWrites() } }) } diff --git a/src/dbnode/storage/bootstrap/namespace_results_map_gen.go b/src/dbnode/storage/bootstrap/namespace_results_map_gen.go new file mode 100644 index 0000000000..7409949b29 --- /dev/null +++ b/src/dbnode/storage/bootstrap/namespace_results_map_gen.go @@ -0,0 +1,299 @@ +// Copyright (c) 2019 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package bootstrap + +import ( + "github.com/m3db/m3/src/x/ident" +) + +// Copyright (c) 2019 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// NamespaceResultsMapHash is the hash for a given map entry, this is public to support +// iterating over the map using a native Go for loop. +type NamespaceResultsMapHash uint64 + +// NamespaceResultsMapHashFn is the hash function to execute when hashing a key. +type NamespaceResultsMapHashFn func(ident.ID) NamespaceResultsMapHash + +// NamespaceResultsMapEqualsFn is the equals key function to execute when detecting equality of a key. +type NamespaceResultsMapEqualsFn func(ident.ID, ident.ID) bool + +// NamespaceResultsMapCopyFn is the copy key function to execute when copying the key. +type NamespaceResultsMapCopyFn func(ident.ID) ident.ID + +// NamespaceResultsMapFinalizeFn is the finalize key function to execute when finished with a key. +type NamespaceResultsMapFinalizeFn func(ident.ID) + +// NamespaceResultsMap uses the genny package to provide a generic hash map that can be specialized +// by running the following command from this root of the repository: +// ``` +// make hashmap-gen pkg=outpkg key_type=Type value_type=Type out_dir=/tmp +// ``` +// Or if you would like to use bytes or ident.ID as keys you can use the +// partially specialized maps to generate your own maps as well: +// ``` +// make byteshashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// make idhashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// ``` +// This will output to stdout the generated source file to use for your map. +// It uses linear probing by incrementing the number of the hash created when +// hashing the identifier if there is a collision. +// NamespaceResultsMap is a value type and not an interface to allow for less painful +// upgrades when adding/removing methods, it is not likely to need mocking so +// an interface would not be super useful either. +type NamespaceResultsMap struct { + _NamespaceResultsMapOptions + + // lookup uses hash of the identifier for the key and the MapEntry value + // wraps the value type and the key (used to ensure lookup is correct + // when dealing with collisions), we use uint64 for the hash partially + // because lookups of maps with uint64 keys has a fast path for Go. + lookup map[NamespaceResultsMapHash]NamespaceResultsMapEntry +} + +// _NamespaceResultsMapOptions is a set of options used when creating an identifier map, it is kept +// private so that implementers of the generated map can specify their own options +// that partially fulfill these options. +type _NamespaceResultsMapOptions struct { + // hash is the hash function to execute when hashing a key. + hash NamespaceResultsMapHashFn + // equals is the equals key function to execute when detecting equality. + equals NamespaceResultsMapEqualsFn + // copy is the copy key function to execute when copying the key. + copy NamespaceResultsMapCopyFn + // finalize is the finalize key function to execute when finished with a + // key, this is optional to specify. + finalize NamespaceResultsMapFinalizeFn + // initialSize is the initial size for the map, use zero to use Go's std map + // initial size and consequently is optional to specify. + initialSize int +} + +// NamespaceResultsMapEntry is an entry in the map, this is public to support iterating +// over the map using a native Go for loop. +type NamespaceResultsMapEntry struct { + // key is used to check equality on lookups to resolve collisions + key _NamespaceResultsMapKey + // value type stored + value NamespaceResult +} + +type _NamespaceResultsMapKey struct { + key ident.ID + finalize bool +} + +// Key returns the map entry key. +func (e NamespaceResultsMapEntry) Key() ident.ID { + return e.key.key +} + +// Value returns the map entry value. +func (e NamespaceResultsMapEntry) Value() NamespaceResult { + return e.value +} + +// _NamespaceResultsMapAlloc is a non-exported function so that when generating the source code +// for the map you can supply a public constructor that sets the correct +// hash, equals, copy, finalize options without users of the map needing to +// implement them themselves. +func _NamespaceResultsMapAlloc(opts _NamespaceResultsMapOptions) *NamespaceResultsMap { + m := &NamespaceResultsMap{_NamespaceResultsMapOptions: opts} + m.Reallocate() + return m +} + +func (m *NamespaceResultsMap) newMapKey(k ident.ID, opts _NamespaceResultsMapKeyOptions) _NamespaceResultsMapKey { + key := _NamespaceResultsMapKey{key: k, finalize: opts.finalizeKey} + if !opts.copyKey { + return key + } + + key.key = m.copy(k) + return key +} + +func (m *NamespaceResultsMap) removeMapKey(hash NamespaceResultsMapHash, key _NamespaceResultsMapKey) { + delete(m.lookup, hash) + if key.finalize { + m.finalize(key.key) + } +} + +// Get returns a value in the map for an identifier if found. +func (m *NamespaceResultsMap) Get(k ident.ID) (NamespaceResult, bool) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + return entry.value, true + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + var empty NamespaceResult + return empty, false +} + +// Set will set the value for an identifier. +func (m *NamespaceResultsMap) Set(k ident.ID, v NamespaceResult) { + m.set(k, v, _NamespaceResultsMapKeyOptions{ + copyKey: true, + finalizeKey: m.finalize != nil, + }) +} + +// NamespaceResultsMapSetUnsafeOptions is a set of options to use when setting a value with +// the SetUnsafe method. +type NamespaceResultsMapSetUnsafeOptions struct { + NoCopyKey bool + NoFinalizeKey bool +} + +// SetUnsafe will set the value for an identifier with unsafe options for how +// the map treats the key. +func (m *NamespaceResultsMap) SetUnsafe(k ident.ID, v NamespaceResult, opts NamespaceResultsMapSetUnsafeOptions) { + m.set(k, v, _NamespaceResultsMapKeyOptions{ + copyKey: !opts.NoCopyKey, + finalizeKey: !opts.NoFinalizeKey, + }) +} + +type _NamespaceResultsMapKeyOptions struct { + copyKey bool + finalizeKey bool +} + +func (m *NamespaceResultsMap) set(k ident.ID, v NamespaceResult, opts _NamespaceResultsMapKeyOptions) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.lookup[hash] = NamespaceResultsMapEntry{ + key: entry.key, + value: v, + } + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + + m.lookup[hash] = NamespaceResultsMapEntry{ + key: m.newMapKey(k, opts), + value: v, + } +} + +// Iter provides the underlying map to allow for using a native Go for loop +// to iterate the map, however callers should only ever read and not write +// the map. +func (m *NamespaceResultsMap) Iter() map[NamespaceResultsMapHash]NamespaceResultsMapEntry { + return m.lookup +} + +// Len returns the number of map entries in the map. +func (m *NamespaceResultsMap) Len() int { + return len(m.lookup) +} + +// Contains returns true if value exists for key, false otherwise, it is +// shorthand for a call to Get that doesn't return the value. +func (m *NamespaceResultsMap) Contains(k ident.ID) bool { + _, ok := m.Get(k) + return ok +} + +// Delete will remove a value set in the map for the specified key. +func (m *NamespaceResultsMap) Delete(k ident.ID) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.removeMapKey(hash, entry.key) + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } +} + +// Reset will reset the map by simply deleting all keys to avoid +// allocating a new map. +func (m *NamespaceResultsMap) Reset() { + for hash, entry := range m.lookup { + m.removeMapKey(hash, entry.key) + } +} + +// Reallocate will avoid deleting all keys and reallocate a new +// map, this is useful if you believe you have a large map and +// will not need to grow back to a similar size. +func (m *NamespaceResultsMap) Reallocate() { + if m.initialSize > 0 { + m.lookup = make(map[NamespaceResultsMapHash]NamespaceResultsMapEntry, m.initialSize) + } else { + m.lookup = make(map[NamespaceResultsMapHash]NamespaceResultsMapEntry) + } +} diff --git a/src/dbnode/storage/bootstrap/namespace_results_new_map_gen.go b/src/dbnode/storage/bootstrap/namespace_results_new_map_gen.go new file mode 100644 index 0000000000..8b02405418 --- /dev/null +++ b/src/dbnode/storage/bootstrap/namespace_results_new_map_gen.go @@ -0,0 +1,95 @@ +// Copyright (c) 2019 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package bootstrap + +import ( + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + + "github.com/cespare/xxhash" +) + +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// NamespaceResultsMapOptions provides options used when created the map. +type NamespaceResultsMapOptions struct { + InitialSize int + KeyCopyPool pool.BytesPool +} + +// NewNamespaceResultsMap returns a new byte keyed map. +func NewNamespaceResultsMap(opts NamespaceResultsMapOptions) *NamespaceResultsMap { + var ( + copyFn NamespaceResultsMapCopyFn + finalizeFn NamespaceResultsMapFinalizeFn + ) + if pool := opts.KeyCopyPool; pool == nil { + copyFn = func(k ident.ID) ident.ID { + return ident.BytesID(append([]byte(nil), k.Bytes()...)) + } + } else { + copyFn = func(k ident.ID) ident.ID { + bytes := k.Bytes() + keyLen := len(bytes) + pooled := pool.Get(keyLen)[:keyLen] + copy(pooled, bytes) + return ident.BytesID(pooled) + } + finalizeFn = func(k ident.ID) { + if slice, ok := k.(ident.BytesID); ok { + pool.Put(slice) + } + } + } + return _NamespaceResultsMapAlloc(_NamespaceResultsMapOptions{ + hash: func(id ident.ID) NamespaceResultsMapHash { + return NamespaceResultsMapHash(xxhash.Sum64(id.Bytes())) + }, + equals: func(x, y ident.ID) bool { + return x.Equal(y) + }, + copy: copyFn, + finalize: finalizeFn, + initialSize: opts.InitialSize, + }) +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go b/src/dbnode/storage/bootstrap/namespaces_map_gen.go similarity index 73% rename from src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go rename to src/dbnode/storage/bootstrap/namespaces_map_gen.go index 8217daf776..c4f23409f7 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go +++ b/src/dbnode/storage/bootstrap/namespaces_map_gen.go @@ -22,7 +22,7 @@ // Any changes will be lost if this file is regenerated. // see https://github.com/mauricelam/genny -package commitlog +package bootstrap import ( "github.com/m3db/m3/src/x/ident" @@ -72,23 +72,23 @@ import ( // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -// MapHash is the hash for a given map entry, this is public to support +// NamespacesMapHash is the hash for a given map entry, this is public to support // iterating over the map using a native Go for loop. -type MapHash uint64 +type NamespacesMapHash uint64 -// HashFn is the hash function to execute when hashing a key. -type HashFn func(ident.ID) MapHash +// NamespacesMapHashFn is the hash function to execute when hashing a key. +type NamespacesMapHashFn func(ident.ID) NamespacesMapHash -// EqualsFn is the equals key function to execute when detecting equality of a key. -type EqualsFn func(ident.ID, ident.ID) bool +// NamespacesMapEqualsFn is the equals key function to execute when detecting equality of a key. +type NamespacesMapEqualsFn func(ident.ID, ident.ID) bool -// CopyFn is the copy key function to execute when copying the key. -type CopyFn func(ident.ID) ident.ID +// NamespacesMapCopyFn is the copy key function to execute when copying the key. +type NamespacesMapCopyFn func(ident.ID) ident.ID -// FinalizeFn is the finalize key function to execute when finished with a key. -type FinalizeFn func(ident.ID) +// NamespacesMapFinalizeFn is the finalize key function to execute when finished with a key. +type NamespacesMapFinalizeFn func(ident.ID) -// Map uses the genny package to provide a generic hash map that can be specialized +// NamespacesMap uses the genny package to provide a generic hash map that can be specialized // by running the following command from this root of the repository: // ``` // make hashmap-gen pkg=outpkg key_type=Type value_type=Type out_dir=/tmp @@ -102,73 +102,73 @@ type FinalizeFn func(ident.ID) // This will output to stdout the generated source file to use for your map. // It uses linear probing by incrementing the number of the hash created when // hashing the identifier if there is a collision. -// Map is a value type and not an interface to allow for less painful +// NamespacesMap is a value type and not an interface to allow for less painful // upgrades when adding/removing methods, it is not likely to need mocking so // an interface would not be super useful either. -type Map struct { - mapOptions +type NamespacesMap struct { + _NamespacesMapOptions // lookup uses hash of the identifier for the key and the MapEntry value // wraps the value type and the key (used to ensure lookup is correct // when dealing with collisions), we use uint64 for the hash partially // because lookups of maps with uint64 keys has a fast path for Go. - lookup map[MapHash]MapEntry + lookup map[NamespacesMapHash]NamespacesMapEntry } -// mapOptions is a set of options used when creating an identifier map, it is kept +// _NamespacesMapOptions is a set of options used when creating an identifier map, it is kept // private so that implementers of the generated map can specify their own options // that partially fulfill these options. -type mapOptions struct { +type _NamespacesMapOptions struct { // hash is the hash function to execute when hashing a key. - hash HashFn + hash NamespacesMapHashFn // equals is the equals key function to execute when detecting equality. - equals EqualsFn + equals NamespacesMapEqualsFn // copy is the copy key function to execute when copying the key. - copy CopyFn + copy NamespacesMapCopyFn // finalize is the finalize key function to execute when finished with a // key, this is optional to specify. - finalize FinalizeFn + finalize NamespacesMapFinalizeFn // initialSize is the initial size for the map, use zero to use Go's std map // initial size and consequently is optional to specify. initialSize int } -// MapEntry is an entry in the map, this is public to support iterating +// NamespacesMapEntry is an entry in the map, this is public to support iterating // over the map using a native Go for loop. -type MapEntry struct { +type NamespacesMapEntry struct { // key is used to check equality on lookups to resolve collisions - key mapKey + key _NamespacesMapKey // value type stored - value metadataAndEncodersByTime + value Namespace } -type mapKey struct { +type _NamespacesMapKey struct { key ident.ID finalize bool } // Key returns the map entry key. -func (e MapEntry) Key() ident.ID { +func (e NamespacesMapEntry) Key() ident.ID { return e.key.key } // Value returns the map entry value. -func (e MapEntry) Value() metadataAndEncodersByTime { +func (e NamespacesMapEntry) Value() Namespace { return e.value } -// mapAlloc is a non-exported function so that when generating the source code +// _NamespacesMapAlloc is a non-exported function so that when generating the source code // for the map you can supply a public constructor that sets the correct // hash, equals, copy, finalize options without users of the map needing to // implement them themselves. -func mapAlloc(opts mapOptions) *Map { - m := &Map{mapOptions: opts} +func _NamespacesMapAlloc(opts _NamespacesMapOptions) *NamespacesMap { + m := &NamespacesMap{_NamespacesMapOptions: opts} m.Reallocate() return m } -func (m *Map) newMapKey(k ident.ID, opts mapKeyOptions) mapKey { - key := mapKey{key: k, finalize: opts.finalizeKey} +func (m *NamespacesMap) newMapKey(k ident.ID, opts _NamespacesMapKeyOptions) _NamespacesMapKey { + key := _NamespacesMapKey{key: k, finalize: opts.finalizeKey} if !opts.copyKey { return key } @@ -177,7 +177,7 @@ func (m *Map) newMapKey(k ident.ID, opts mapKeyOptions) mapKey { return key } -func (m *Map) removeMapKey(hash MapHash, key mapKey) { +func (m *NamespacesMap) removeMapKey(hash NamespacesMapHash, key _NamespacesMapKey) { delete(m.lookup, hash) if key.finalize { m.finalize(key.key) @@ -185,7 +185,7 @@ func (m *Map) removeMapKey(hash MapHash, key mapKey) { } // Get returns a value in the map for an identifier if found. -func (m *Map) Get(k ident.ID) (metadataAndEncodersByTime, bool) { +func (m *NamespacesMap) Get(k ident.ID) (Namespace, bool) { hash := m.hash(k) for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { if m.equals(entry.key.key, k) { @@ -194,44 +194,44 @@ func (m *Map) Get(k ident.ID) (metadataAndEncodersByTime, bool) { // Linear probe to "next" to this entry (really a rehash) hash++ } - var empty metadataAndEncodersByTime + var empty Namespace return empty, false } // Set will set the value for an identifier. -func (m *Map) Set(k ident.ID, v metadataAndEncodersByTime) { - m.set(k, v, mapKeyOptions{ +func (m *NamespacesMap) Set(k ident.ID, v Namespace) { + m.set(k, v, _NamespacesMapKeyOptions{ copyKey: true, finalizeKey: m.finalize != nil, }) } -// SetUnsafeOptions is a set of options to use when setting a value with +// NamespacesMapSetUnsafeOptions is a set of options to use when setting a value with // the SetUnsafe method. -type SetUnsafeOptions struct { +type NamespacesMapSetUnsafeOptions struct { NoCopyKey bool NoFinalizeKey bool } // SetUnsafe will set the value for an identifier with unsafe options for how // the map treats the key. -func (m *Map) SetUnsafe(k ident.ID, v metadataAndEncodersByTime, opts SetUnsafeOptions) { - m.set(k, v, mapKeyOptions{ +func (m *NamespacesMap) SetUnsafe(k ident.ID, v Namespace, opts NamespacesMapSetUnsafeOptions) { + m.set(k, v, _NamespacesMapKeyOptions{ copyKey: !opts.NoCopyKey, finalizeKey: !opts.NoFinalizeKey, }) } -type mapKeyOptions struct { +type _NamespacesMapKeyOptions struct { copyKey bool finalizeKey bool } -func (m *Map) set(k ident.ID, v metadataAndEncodersByTime, opts mapKeyOptions) { +func (m *NamespacesMap) set(k ident.ID, v Namespace, opts _NamespacesMapKeyOptions) { hash := m.hash(k) for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { if m.equals(entry.key.key, k) { - m.lookup[hash] = MapEntry{ + m.lookup[hash] = NamespacesMapEntry{ key: entry.key, value: v, } @@ -241,7 +241,7 @@ func (m *Map) set(k ident.ID, v metadataAndEncodersByTime, opts mapKeyOptions) { hash++ } - m.lookup[hash] = MapEntry{ + m.lookup[hash] = NamespacesMapEntry{ key: m.newMapKey(k, opts), value: v, } @@ -250,24 +250,24 @@ func (m *Map) set(k ident.ID, v metadataAndEncodersByTime, opts mapKeyOptions) { // Iter provides the underlying map to allow for using a native Go for loop // to iterate the map, however callers should only ever read and not write // the map. -func (m *Map) Iter() map[MapHash]MapEntry { +func (m *NamespacesMap) Iter() map[NamespacesMapHash]NamespacesMapEntry { return m.lookup } // Len returns the number of map entries in the map. -func (m *Map) Len() int { +func (m *NamespacesMap) Len() int { return len(m.lookup) } // Contains returns true if value exists for key, false otherwise, it is // shorthand for a call to Get that doesn't return the value. -func (m *Map) Contains(k ident.ID) bool { +func (m *NamespacesMap) Contains(k ident.ID) bool { _, ok := m.Get(k) return ok } // Delete will remove a value set in the map for the specified key. -func (m *Map) Delete(k ident.ID) { +func (m *NamespacesMap) Delete(k ident.ID) { hash := m.hash(k) for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { if m.equals(entry.key.key, k) { @@ -281,7 +281,7 @@ func (m *Map) Delete(k ident.ID) { // Reset will reset the map by simply deleting all keys to avoid // allocating a new map. -func (m *Map) Reset() { +func (m *NamespacesMap) Reset() { for hash, entry := range m.lookup { m.removeMapKey(hash, entry.key) } @@ -290,10 +290,10 @@ func (m *Map) Reset() { // Reallocate will avoid deleting all keys and reallocate a new // map, this is useful if you believe you have a large map and // will not need to grow back to a similar size. -func (m *Map) Reallocate() { +func (m *NamespacesMap) Reallocate() { if m.initialSize > 0 { - m.lookup = make(map[MapHash]MapEntry, m.initialSize) + m.lookup = make(map[NamespacesMapHash]NamespacesMapEntry, m.initialSize) } else { - m.lookup = make(map[MapHash]MapEntry) + m.lookup = make(map[NamespacesMapHash]NamespacesMapEntry) } } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go b/src/dbnode/storage/bootstrap/namespaces_new_map_gen.go similarity index 86% rename from src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go rename to src/dbnode/storage/bootstrap/namespaces_new_map_gen.go index 34c6b4ba02..5fb41e3e47 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go +++ b/src/dbnode/storage/bootstrap/namespaces_new_map_gen.go @@ -1,4 +1,4 @@ -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2019 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 @@ -22,7 +22,7 @@ // Any changes will be lost if this file is regenerated. // see https://github.com/mauricelam/genny -package commitlog +package bootstrap import ( "github.com/m3db/m3/src/x/ident" @@ -51,17 +51,17 @@ import ( // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -// MapOptions provides options used when created the map. -type MapOptions struct { +// NamespacesMapOptions provides options used when created the map. +type NamespacesMapOptions struct { InitialSize int KeyCopyPool pool.BytesPool } -// NewMap returns a new byte keyed map. -func NewMap(opts MapOptions) *Map { +// NewNamespacesMap returns a new byte keyed map. +func NewNamespacesMap(opts NamespacesMapOptions) *NamespacesMap { var ( - copyFn CopyFn - finalizeFn FinalizeFn + copyFn NamespacesMapCopyFn + finalizeFn NamespacesMapFinalizeFn ) if pool := opts.KeyCopyPool; pool == nil { copyFn = func(k ident.ID) ident.ID { @@ -81,9 +81,9 @@ func NewMap(opts MapOptions) *Map { } } } - return mapAlloc(mapOptions{ - hash: func(id ident.ID) MapHash { - return MapHash(xxhash.Sum64(id.Bytes())) + return _NamespacesMapAlloc(_NamespacesMapOptions{ + hash: func(id ident.ID) NamespacesMapHash { + return NamespacesMapHash(xxhash.Sum64(id.Bytes())) }, equals: func(x, y ident.ID) bool { return x.Equal(y) diff --git a/src/dbnode/storage/bootstrap/noop.go b/src/dbnode/storage/bootstrap/noop.go index d176dbf5eb..297e7a4615 100644 --- a/src/dbnode/storage/bootstrap/noop.go +++ b/src/dbnode/storage/bootstrap/noop.go @@ -22,9 +22,6 @@ package bootstrap import ( "time" - - "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" ) type noOpBootstrapProcessProvider struct{} @@ -49,11 +46,7 @@ type noOpBootstrapProcess struct{} func (b noOpBootstrapProcess) Run( start time.Time, - ns namespace.Metadata, - shards []uint32, -) (ProcessResult, error) { - return ProcessResult{ - DataResult: result.NewDataBootstrapResult(), - IndexResult: result.NewIndexBootstrapResult(), - }, nil + namespaces []ProcessNamespace, +) (NamespaceResults, error) { + return NewNamespaceResults(NewNamespaces(namespaces)), nil } diff --git a/src/dbnode/storage/bootstrap/process.go b/src/dbnode/storage/bootstrap/process.go index b802ca0b69..77d1e9f0ae 100644 --- a/src/dbnode/storage/bootstrap/process.go +++ b/src/dbnode/storage/bootstrap/process.go @@ -21,14 +21,15 @@ package bootstrap import ( + "fmt" "sync" "time" "github.com/m3db/m3/src/dbnode/clock" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/topology" xtime "github.com/m3db/m3/src/x/time" @@ -150,105 +151,118 @@ type bootstrapProcess struct { } func (b bootstrapProcess) Run( - start time.Time, - namespace namespace.Metadata, - shards []uint32, -) (ProcessResult, error) { - dataResult, err := b.bootstrapData(start, namespace, shards) - if err != nil { - return ProcessResult{}, err + at time.Time, + namespaces []ProcessNamespace, +) (NamespaceResults, error) { + namespacesRunFirst := Namespaces{ + Namespaces: NewNamespacesMap(NamespacesMapOptions{}), } - - indexResult, err := b.bootstrapIndex(start, namespace, shards) - if err != nil { - return ProcessResult{}, err + namespacesRunSecond := Namespaces{ + Namespaces: NewNamespacesMap(NamespacesMapOptions{}), + } + for _, namespace := range namespaces { + ropts := namespace.Metadata.Options().RetentionOptions() + idxopts := namespace.Metadata.Options().IndexOptions() + dataRanges := b.targetRangesForData(at, ropts) + indexRanges := b.targetRangesForIndex(at, ropts, idxopts) + + namespacesRunFirst.Namespaces.Set(namespace.Metadata.ID(), Namespace{ + Metadata: namespace.Metadata, + Shards: namespace.Shards, + DataAccumulator: namespace.DataAccumulator, + DataTargetRange: dataRanges.firstRangeWithPersistTrue, + IndexTargetRange: indexRanges.firstRangeWithPersistTrue, + DataRunOptions: NamespaceRunOptions{ + ShardTimeRanges: b.newShardTimeRanges( + dataRanges.firstRangeWithPersistTrue.Range, namespace.Shards), + RunOptions: dataRanges.firstRangeWithPersistTrue.RunOptions, + }, + IndexRunOptions: NamespaceRunOptions{ + ShardTimeRanges: b.newShardTimeRanges( + indexRanges.firstRangeWithPersistTrue.Range, namespace.Shards), + RunOptions: indexRanges.firstRangeWithPersistTrue.RunOptions, + }, + }) + namespacesRunSecond.Namespaces.Set(namespace.Metadata.ID(), Namespace{ + Metadata: namespace.Metadata, + Shards: namespace.Shards, + DataAccumulator: namespace.DataAccumulator, + DataTargetRange: dataRanges.secondRangeWithPersistFalse, + IndexTargetRange: indexRanges.secondRangeWithPersistFalse, + DataRunOptions: NamespaceRunOptions{ + ShardTimeRanges: b.newShardTimeRanges( + dataRanges.secondRangeWithPersistFalse.Range, namespace.Shards), + RunOptions: dataRanges.secondRangeWithPersistFalse.RunOptions, + }, + IndexRunOptions: NamespaceRunOptions{ + ShardTimeRanges: b.newShardTimeRanges( + indexRanges.secondRangeWithPersistFalse.Range, namespace.Shards), + RunOptions: indexRanges.secondRangeWithPersistFalse.RunOptions, + }, + }) } - return ProcessResult{ - DataResult: dataResult, - IndexResult: indexResult, - }, nil -} - -func (b bootstrapProcess) bootstrapData( - at time.Time, - namespace namespace.Metadata, - shards []uint32, -) (result.DataBootstrapResult, error) { - bootstrapResult := result.NewDataBootstrapResult() - ropts := namespace.Options().RetentionOptions() - targetRanges := b.targetRangesForData(at, ropts) - for _, target := range targetRanges { - logFields := b.logFields(bootstrapDataRunType, namespace, - shards, target.Range) - b.logBootstrapRun(logFields) + bootstrapResult := NewNamespaceResults(namespacesRunFirst) + for _, namespaces := range []Namespaces{ + namespacesRunFirst, + namespacesRunSecond, + } { + for _, entry := range namespaces.Namespaces.Iter() { + namespace := entry.Value() + logFields := b.logFields(namespace.Metadata, namespace.Shards, + namespace.DataTargetRange.Range, namespace.IndexTargetRange.Range) + b.logBootstrapRun(logFields) + } begin := b.nowFn() - shardsTimeRanges := b.newShardTimeRanges(target.Range, shards) - res, err := b.bootstrapper.BootstrapData(namespace, - shardsTimeRanges, target.RunOptions) + res, err := b.bootstrapper.Bootstrap(namespaces) + took := b.nowFn().Sub(begin) + for _, entry := range namespaces.Namespaces.Iter() { + namespace := entry.Value() + nsID := namespace.Metadata.ID() + result, ok := res.Results.Get(nsID) + if !ok { + return NamespaceResults{}, + fmt.Errorf("result missing for namespace: %v", nsID.String()) + } - b.logBootstrapResult(logFields, err, begin) - if err != nil { - return nil, err + logFields := b.logFields(namespace.Metadata, namespace.Shards, + namespace.DataTargetRange.Range, namespace.IndexTargetRange.Range) + b.logBootstrapResult(result, logFields, err, took) } - bootstrapResult = result.MergedDataBootstrapResult(bootstrapResult, res) - } - - return bootstrapResult, nil -} - -func (b bootstrapProcess) bootstrapIndex( - at time.Time, - namespace namespace.Metadata, - shards []uint32, -) (result.IndexBootstrapResult, error) { - bootstrapResult := result.NewIndexBootstrapResult() - ropts := namespace.Options().RetentionOptions() - idxopts := namespace.Options().IndexOptions() - if !idxopts.Enabled() { - // NB(r): If indexing not enable we just return an empty result - return result.NewIndexBootstrapResult(), nil - } - - targetRanges := b.targetRangesForIndex(at, ropts, idxopts) - for _, target := range targetRanges { - logFields := b.logFields(bootstrapIndexRunType, namespace, - shards, target.Range) - b.logBootstrapRun(logFields) - - begin := b.nowFn() - shardsTimeRanges := b.newShardTimeRanges(target.Range, shards) - res, err := b.bootstrapper.BootstrapIndex(namespace, - shardsTimeRanges, target.RunOptions) - - b.logBootstrapResult(logFields, err, begin) if err != nil { - return nil, err + return NamespaceResults{}, err } - bootstrapResult = result.MergedIndexBootstrapResult(bootstrapResult, res) + bootstrapResult = MergeNamespaceResults(bootstrapResult, res) } return bootstrapResult, nil } func (b bootstrapProcess) logFields( - runType bootstrapRunType, namespace namespace.Metadata, shards []uint32, - window xtime.Range, + dataTimeWindow xtime.Range, + indexTimeWindow xtime.Range, ) []zapcore.Field { - return []zapcore.Field{ - zap.String("run", string(runType)), + fields := []zapcore.Field{ zap.String("bootstrapper", b.bootstrapper.String()), - zap.String("namespace", namespace.ID().String()), + zap.Stringer("namespace", namespace.ID()), zap.Int("numShards", len(shards)), - zap.Time("from", window.Start), - zap.Time("to", window.End), - zap.Duration("range", window.End.Sub(window.Start)), + zap.Time("dataFrom", dataTimeWindow.Start), + zap.Time("dataTo", dataTimeWindow.End), + zap.Duration("dataRange", dataTimeWindow.End.Sub(dataTimeWindow.Start)), } + if namespace.Options().IndexOptions().Enabled() { + fields = append(fields, []zapcore.Field{ + zap.Time("indexFrom", indexTimeWindow.Start), + zap.Time("indexTo", indexTimeWindow.End), + zap.Duration("indexRange", indexTimeWindow.End.Sub(indexTimeWindow.Start)), + }...) + } + return fields } func (b bootstrapProcess) newShardTimeRanges( @@ -266,28 +280,34 @@ func (b bootstrapProcess) newShardTimeRanges( func (b bootstrapProcess) logBootstrapRun( logFields []zapcore.Field, ) { - b.log.Info("bootstrapping shards for range starting", logFields...) + b.log.Info("bootstrap range starting", logFields...) } func (b bootstrapProcess) logBootstrapResult( + result NamespaceResult, logFields []zapcore.Field, err error, - begin time.Time, + took time.Duration, ) { - logFields = append(logFields, zap.Duration("took", b.nowFn().Sub(begin))) + logFields = append(logFields, + zap.Duration("took", took)) + if result.IndexResult != nil { + logFields = append(logFields, + zap.Int("numIndexBlocks", len(result.IndexResult.IndexResults()))) + } if err != nil { logFields = append(logFields, zap.Error(err)) - b.log.Info("bootstrapping shards for range completed with error", logFields...) + b.log.Info("bootstrap range completed with error", logFields...) return } - b.log.Info("bootstrapping shards for range completed successfully", logFields...) + b.log.Info("bootstrap range completed successfully", logFields...) } func (b bootstrapProcess) targetRangesForData( at time.Time, ropts retention.Options, -) []TargetRange { +) targetRangesResult { return b.targetRanges(at, targetRangesOptions{ retentionPeriod: ropts.RetentionPeriod(), futureRetentionPeriod: ropts.FutureRetentionPeriod(), @@ -301,7 +321,7 @@ func (b bootstrapProcess) targetRangesForIndex( at time.Time, ropts retention.Options, idxopts namespace.IndexOptions, -) []TargetRange { +) targetRangesResult { return b.targetRanges(at, targetRangesOptions{ retentionPeriod: ropts.RetentionPeriod(), futureRetentionPeriod: ropts.FutureRetentionPeriod(), @@ -319,10 +339,15 @@ type targetRangesOptions struct { bufferFuture time.Duration } +type targetRangesResult struct { + firstRangeWithPersistTrue TargetRange + secondRangeWithPersistFalse TargetRange +} + func (b bootstrapProcess) targetRanges( at time.Time, opts targetRangesOptions, -) []TargetRange { +) targetRangesResult { start := at.Add(-opts.retentionPeriod). Truncate(opts.blockSize) midPoint := at. @@ -340,8 +365,8 @@ func (b bootstrapProcess) targetRanges( // bootstrap with persistence so we don't keep the full raw // data in process until we finish bootstrapping which could // cause the process to OOM. - return []TargetRange{ - { + return targetRangesResult{ + firstRangeWithPersistTrue: TargetRange{ Range: xtime.Range{Start: start, End: midPoint}, RunOptions: b.newRunOptions().SetPersistConfig(PersistConfig{ Enabled: true, @@ -351,7 +376,7 @@ func (b bootstrapProcess) targetRanges( FileSetType: persist.FileSetFlushType, }), }, - { + secondRangeWithPersistFalse: TargetRange{ Range: xtime.Range{Start: midPoint, End: cutover}, RunOptions: b.newRunOptions().SetPersistConfig(PersistConfig{ Enabled: true, @@ -372,3 +397,70 @@ func (b bootstrapProcess) newRunOptions() RunOptions { ). SetInitialTopologyState(b.initialTopologyState) } + +// NewNamespaces returns a new set of bootstrappable namespaces. +func NewNamespaces( + namespaces []ProcessNamespace, +) Namespaces { + namespacesMap := NewNamespacesMap(NamespacesMapOptions{}) + for _, ns := range namespaces { + namespacesMap.Set(ns.Metadata.ID(), Namespace{ + Metadata: ns.Metadata, + Shards: ns.Shards, + DataAccumulator: ns.DataAccumulator, + }) + } + return Namespaces{ + Namespaces: namespacesMap, + } +} + +// NewNamespaceResults creates a +// namespace results map with an entry for each +// namespace spoecified by a namespaces map. +func NewNamespaceResults( + namespaces Namespaces, +) NamespaceResults { + resultsMap := NewNamespaceResultsMap(NamespaceResultsMapOptions{}) + for _, entry := range namespaces.Namespaces.Iter() { + key := entry.Key() + value := entry.Value() + resultsMap.Set(key, NamespaceResult{ + Metadata: value.Metadata, + Shards: value.Shards, + DataResult: result.NewDataBootstrapResult(), + IndexResult: result.NewIndexBootstrapResult(), + }) + } + return NamespaceResults{ + Results: resultsMap, + } +} + +// MergeNamespaceResults merges two namespace results, this will mutate +// both a and b and return a merged copy of them reusing one of the results. +func MergeNamespaceResults(a, b NamespaceResults) NamespaceResults { + for _, entry := range a.Results.Iter() { + id := entry.Key() + elem := entry.Value() + other, ok := b.Results.Get(id) + if !ok { + continue + } + elem.DataResult = result.MergedDataBootstrapResult(elem.DataResult, + other.DataResult) + elem.IndexResult = result.MergedIndexBootstrapResult(elem.IndexResult, + other.IndexResult) + + // Save back the merged results. + a.Results.Set(id, elem) + + // Remove from b, then can directly add to a all non-merged results. + b.Results.Delete(id) + } + // All overlapping between a and b have been merged, add rest to a. + for _, entry := range b.Results.Iter() { + a.Results.Set(entry.Key(), entry.Value()) + } + return a +} diff --git a/src/dbnode/storage/bootstrap/result/result_data.go b/src/dbnode/storage/bootstrap/result/result_data.go index 7ce966187c..87838d7ae6 100644 --- a/src/dbnode/storage/bootstrap/result/result_data.go +++ b/src/dbnode/storage/bootstrap/result/result_data.go @@ -25,35 +25,23 @@ import ( "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/x/ident" - xtime "github.com/m3db/m3/src/x/time" ) type dataBootstrapResult struct { - results ShardResults unfulfilled ShardTimeRanges } // NewDataBootstrapResult creates a new result. func NewDataBootstrapResult() DataBootstrapResult { return &dataBootstrapResult{ - results: make(ShardResults), unfulfilled: make(ShardTimeRanges), } } -func (r *dataBootstrapResult) ShardResults() ShardResults { - return r.results -} - func (r *dataBootstrapResult) Unfulfilled() ShardTimeRanges { return r.unfulfilled } -func (r *dataBootstrapResult) Add(shard uint32, result ShardResult, unfulfilled xtime.Ranges) { - r.results.AddResults(ShardResults{shard: result}) - r.unfulfilled.AddRanges(ShardTimeRanges{shard: unfulfilled}) -} - func (r *dataBootstrapResult) SetUnfulfilled(unfulfilled ShardTimeRanges) { r.unfulfilled = unfulfilled } @@ -68,21 +56,8 @@ func MergedDataBootstrapResult(i, j DataBootstrapResult) DataBootstrapResult { if j == nil { return i } - sizeI, sizeJ := 0, 0 - for _, sr := range i.ShardResults() { - sizeI += int(sr.NumSeries()) - } - for _, sr := range j.ShardResults() { - sizeJ += int(sr.NumSeries()) - } - if sizeI >= sizeJ { - i.ShardResults().AddResults(j.ShardResults()) - i.Unfulfilled().AddRanges(j.Unfulfilled()) - return i - } - j.ShardResults().AddResults(i.ShardResults()) - j.Unfulfilled().AddRanges(i.Unfulfilled()) - return j + i.Unfulfilled().AddRanges(j.Unfulfilled()) + return i } type shardResult struct { diff --git a/src/dbnode/storage/bootstrap/result/result_data_test.go b/src/dbnode/storage/bootstrap/result/result_data_test.go index d148aaa470..2e5eb242b3 100644 --- a/src/dbnode/storage/bootstrap/result/result_data_test.go +++ b/src/dbnode/storage/bootstrap/result/result_data_test.go @@ -42,65 +42,87 @@ func testResultOptions() Options { return NewOptions() } -func TestDataResultAddMergesExistingShardResults(t *testing.T) { - opts := testResultOptions() - blopts := opts.DatabaseBlockOptions() - +func TestDataResultSetUnfulfilledMergeShardResults(t *testing.T) { start := time.Now().Truncate(testBlockSize) - - blocks := []block.DatabaseBlock{ - block.NewDatabaseBlock(start, testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(1*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(2*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), + rangeOne := ShardTimeRanges{ + 0: xtime.NewRanges(xtime.Range{ + Start: start, + End: start.Add(8 * testBlockSize), + }), + 1: xtime.NewRanges(xtime.Range{ + Start: start, + End: start.Add(testBlockSize), + }), } - srs := []ShardResult{ - NewShardResult(0, opts), - NewShardResult(0, opts), + rangeTwo := ShardTimeRanges{ + 0: xtime.NewRanges(xtime.Range{ + Start: start.Add(6 * testBlockSize), + End: start.Add(10 * testBlockSize), + }), + 2: xtime.NewRanges(xtime.Range{ + Start: start.Add(testBlockSize), + End: start.Add(2 * testBlockSize), + }), } - fooTags := ident.NewTags(ident.StringTag("foo", "foe")) - barTags := ident.NewTags(ident.StringTag("bar", "baz")) + r := NewDataBootstrapResult() + r.SetUnfulfilled(rangeOne) + rTwo := NewDataBootstrapResult() + rTwo.SetUnfulfilled(rangeTwo) - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) - srs[1].AddBlock(ident.StringID("bar"), barTags, blocks[2]) + rMerged := MergedDataBootstrapResult(nil, nil) + assert.Nil(t, rMerged) - r := NewDataBootstrapResult() - r.Add(0, srs[0], xtime.Ranges{}) - r.Add(0, srs[1], xtime.Ranges{}) + rMerged = MergedDataBootstrapResult(r, nil) + assert.True(t, rMerged.Unfulfilled().Equal(rangeOne)) - srMerged := NewShardResult(0, opts) - srMerged.AddBlock(ident.StringID("foo"), fooTags, blocks[0]) - srMerged.AddBlock(ident.StringID("foo"), fooTags, blocks[1]) - srMerged.AddBlock(ident.StringID("bar"), barTags, blocks[2]) + rMerged = MergedDataBootstrapResult(nil, r) + assert.True(t, rMerged.Unfulfilled().Equal(rangeOne)) - merged := NewDataBootstrapResult() - merged.Add(0, srMerged, xtime.Ranges{}) + rMerged = MergedDataBootstrapResult(r, rTwo) + expected := ShardTimeRanges{ + 0: xtime.NewRanges(xtime.Range{ + Start: start, + End: start.Add(10 * testBlockSize), + }), + 1: xtime.NewRanges(xtime.Range{ + Start: start, + End: start.Add(testBlockSize), + }), + 2: xtime.NewRanges(xtime.Range{ + Start: start.Add(testBlockSize), + End: start.Add(testBlockSize * 2), + })} - assert.True(t, r.ShardResults().Equal(merged.ShardResults())) + assert.True(t, rMerged.Unfulfilled().Equal(expected)) } -func TestDataResultAddMergesUnfulfilled(t *testing.T) { +func TestDataResultSetUnfulfilledOverwitesUnfulfilled(t *testing.T) { start := time.Now().Truncate(testBlockSize) - r := NewDataBootstrapResult() + r.SetUnfulfilled(ShardTimeRanges{ + 0: xtime.NewRanges(xtime.Range{ + Start: start, + End: start.Add(8 * testBlockSize), + }), + }) - r.Add(0, nil, xtime.NewRanges(xtime.Range{ + expected := ShardTimeRanges{0: xtime.NewRanges(xtime.Range{ Start: start, End: start.Add(8 * testBlockSize), - })) + })} - r.Add(0, nil, xtime.NewRanges(xtime.Range{ - Start: start, - End: start.Add(2 * testBlockSize), - }).AddRange(xtime.Range{ - Start: start.Add(6 * testBlockSize), - End: start.Add(10 * testBlockSize), - })) + assert.True(t, r.Unfulfilled().Equal(expected)) + r.SetUnfulfilled(ShardTimeRanges{ + 0: xtime.NewRanges(xtime.Range{ + Start: start.Add(6 * testBlockSize), + End: start.Add(10 * testBlockSize), + }), + }) - expected := ShardTimeRanges{0: xtime.NewRanges(xtime.Range{ - Start: start, + expected = ShardTimeRanges{0: xtime.NewRanges(xtime.Range{ + Start: start.Add(6 * testBlockSize), End: start.Add(10 * testBlockSize), })} @@ -136,100 +158,6 @@ func TestResultSetUnfulfilled(t *testing.T) { })) } -func TestResultNumSeries(t *testing.T) { - opts := testResultOptions() - blopts := opts.DatabaseBlockOptions() - - start := time.Now().Truncate(testBlockSize) - - blocks := []block.DatabaseBlock{ - block.NewDatabaseBlock(start, testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(1*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(2*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - } - - srs := []ShardResult{ - NewShardResult(0, opts), - NewShardResult(0, opts), - } - - fooTags := ident.NewTags(ident.StringTag("foo", "foe")) - barTags := ident.NewTags(ident.StringTag("bar", "baz")) - - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) - srs[1].AddBlock(ident.StringID("bar"), barTags, blocks[2]) - - r := NewDataBootstrapResult() - r.Add(0, srs[0], xtime.Ranges{}) - r.Add(1, srs[1], xtime.Ranges{}) - - require.Equal(t, int64(2), r.ShardResults().NumSeries()) -} - -func TestResultAddResult(t *testing.T) { - opts := testResultOptions() - blopts := opts.DatabaseBlockOptions() - - start := time.Now().Truncate(testBlockSize) - - blocks := []block.DatabaseBlock{ - block.NewDatabaseBlock(start, testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(1*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - block.NewDatabaseBlock(start.Add(2*testBlockSize), testBlockSize, ts.Segment{}, blopts, namespace.Context{}), - } - - srs := []ShardResult{ - NewShardResult(0, opts), - NewShardResult(0, opts), - } - fooTags := ident.NewTags(ident.StringTag("foo", "foe")) - barTags := ident.NewTags(ident.StringTag("bar", "baz")) - - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) - srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) - srs[1].AddBlock(ident.StringID("bar"), barTags, blocks[2]) - - rs := []DataBootstrapResult{ - NewDataBootstrapResult(), - NewDataBootstrapResult(), - } - - rs[0].Add(0, srs[0], xtime.NewRanges(xtime.Range{ - Start: start.Add(4 * testBlockSize), - End: start.Add(6 * testBlockSize), - })) - - rs[1].Add(0, srs[1], xtime.NewRanges(xtime.Range{ - Start: start.Add(6 * testBlockSize), - End: start.Add(8 * testBlockSize), - })) - - r := MergedDataBootstrapResult(rs[0], rs[1]) - - srMerged := NewShardResult(0, opts) - srMerged.AddBlock(ident.StringID("foo"), fooTags, blocks[0]) - srMerged.AddBlock(ident.StringID("foo"), fooTags, blocks[1]) - srMerged.AddBlock(ident.StringID("bar"), barTags, blocks[2]) - - expected := struct { - shardResults ShardResults - unfulfilled ShardTimeRanges - }{ - ShardResults{0: srMerged}, - ShardTimeRanges{0: xtime.NewRanges(xtime.Range{ - Start: start.Add(4 * testBlockSize), - End: start.Add(6 * testBlockSize), - }).AddRange(xtime.Range{ - Start: start.Add(6 * testBlockSize), - End: start.Add(8 * testBlockSize), - })}, - } - - assert.True(t, r.ShardResults().Equal(expected.shardResults)) - assert.True(t, r.Unfulfilled().Equal(expected.unfulfilled)) -} - func TestShardResultIsEmpty(t *testing.T) { opts := testResultOptions() sr := NewShardResult(0, opts) @@ -377,7 +305,6 @@ func TestShardTimeRangesToUnfulfilledDataResult(t *testing.T) { }), } r := str.ToUnfulfilledDataResult() - assert.Equal(t, 0, len(r.ShardResults())) assert.True(t, r.Unfulfilled().Equal(str)) } diff --git a/src/dbnode/storage/bootstrap/result/result_index.go b/src/dbnode/storage/bootstrap/result/result_index.go index b7589c49c9..66369d57a2 100644 --- a/src/dbnode/storage/bootstrap/result/result_index.go +++ b/src/dbnode/storage/bootstrap/result/result_index.go @@ -68,6 +68,16 @@ func (r *indexBootstrapResult) Add(block IndexBlock, unfulfilled ShardTimeRanges r.unfulfilled.AddRanges(unfulfilled) } +func (r *indexBootstrapResult) NumSeries() int { + var size int64 + for _, b := range r.results { + for _, s := range b.segments { + size += s.Size() + } + } + return int(size) +} + // Add will add an index block to the collection, merging if one already // exists. func (r IndexResults) Add(block IndexBlock) { diff --git a/src/dbnode/storage/bootstrap/result/shard_ranges.go b/src/dbnode/storage/bootstrap/result/shard_ranges.go index b3444f5b96..2d547b913a 100644 --- a/src/dbnode/storage/bootstrap/result/shard_ranges.go +++ b/src/dbnode/storage/bootstrap/result/shard_ranges.go @@ -103,9 +103,7 @@ func (r ShardTimeRanges) AddRanges(other ShardTimeRanges) { // unfufilled time ranges from the set of shard time ranges. func (r ShardTimeRanges) ToUnfulfilledDataResult() DataBootstrapResult { result := NewDataBootstrapResult() - for shard, ranges := range r { - result.Add(shard, nil, ranges) - } + result.SetUnfulfilled(r.Copy()) return result } @@ -113,7 +111,7 @@ func (r ShardTimeRanges) ToUnfulfilledDataResult() DataBootstrapResult { // unfufilled time ranges from the set of shard time ranges. func (r ShardTimeRanges) ToUnfulfilledIndexResult() IndexBootstrapResult { result := NewIndexBootstrapResult() - result.SetUnfulfilled(r) + result.SetUnfulfilled(r.Copy()) return result } @@ -156,6 +154,12 @@ func (r ShardTimeRanges) MinMax() (time.Time, time.Time) { return min, max } +// MinMaxRange returns the min and max times, and the duration for this range. +func (r ShardTimeRanges) MinMaxRange() (time.Time, time.Time, time.Duration) { + min, max := r.MinMax() + return min, max, max.Sub(min) +} + type summaryFn func(xtime.Ranges) string func (r ShardTimeRanges) summarize(sfn summaryFn) string { diff --git a/src/dbnode/storage/bootstrap/result/types.go b/src/dbnode/storage/bootstrap/result/types.go index e5ff9ade98..cbe546b9c0 100644 --- a/src/dbnode/storage/bootstrap/result/types.go +++ b/src/dbnode/storage/bootstrap/result/types.go @@ -34,15 +34,9 @@ import ( // DataBootstrapResult is the result of a bootstrap of series data. type DataBootstrapResult interface { - // ShardResults is the results of all shards for the bootstrap. - ShardResults() ShardResults - // Unfulfilled is the unfulfilled time ranges for the bootstrap. Unfulfilled() ShardTimeRanges - // Add adds a shard result with any unfulfilled time ranges. - Add(shard uint32, result ShardResult, unfulfilled xtime.Ranges) - // SetUnfulfilled sets the current unfulfilled shard time ranges. SetUnfulfilled(unfulfilled ShardTimeRanges) } @@ -60,6 +54,9 @@ type IndexBootstrapResult interface { // Add adds an index block result. Add(block IndexBlock, unfulfilled ShardTimeRanges) + + // NumSeries returns the total number of series across all segments. + NumSeries() int } // IndexResults is a set of index blocks indexed by block start. diff --git a/src/dbnode/storage/bootstrap/types.go b/src/dbnode/storage/bootstrap/types.go index c53f801076..3fc90fae55 100644 --- a/src/dbnode/storage/bootstrap/types.go +++ b/src/dbnode/storage/bootstrap/types.go @@ -23,10 +23,12 @@ package bootstrap import ( "time" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" - "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/topology" + "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" ) @@ -50,11 +52,110 @@ type ProcessProvider interface { // with the mindset that it will always be set to default values from the constructor. type Process interface { // Run runs the bootstrap process, returning the bootstrap result and any error encountered. - Run(start time.Time, ns namespace.Metadata, shards []uint32) (ProcessResult, error) + Run(start time.Time, namespaces []ProcessNamespace) (NamespaceResults, error) +} + +// ProcessNamespace is a namespace to pass to the bootstrap process. +type ProcessNamespace struct { + // Metadata of the namespace being bootstrapped. + Metadata namespace.Metadata + // Shards is the shards to bootstrap for the bootstrap process. + Shards []uint32 + // DataAccumulator is the data accumulator for the shards. + DataAccumulator NamespaceDataAccumulator +} + +// Namespaces are a set of namespaces being bootstrapped. +type Namespaces struct { + // Namespaces are the namespaces being bootstrapped. + Namespaces *NamespacesMap +} + +// Namespace is a namespace that is being bootstrapped. +type Namespace struct { + // Metadata of the namespace being bootstrapped. + Metadata namespace.Metadata + // Shards is the shards for the namespace being bootstrapped. + Shards []uint32 + // DataAccumulator is the data accumulator for the shards. + DataAccumulator NamespaceDataAccumulator + // DataTargetRange is the data target bootstrap range. + DataTargetRange TargetRange + // IndexTargetRange is the index target bootstrap range. + IndexTargetRange TargetRange + // DataRunOptions are the options for the data bootstrap for this + // namespace. + DataRunOptions NamespaceRunOptions + // IndexRunOptions are the options for the index bootstrap for this + // namespace. + IndexRunOptions NamespaceRunOptions +} + +// NamespaceRunOptions are the run options for a bootstrap process run. +type NamespaceRunOptions struct { + // ShardTimeRanges are the time ranges for the shards that should be fulfilled + // by the bootstrapper. + ShardTimeRanges result.ShardTimeRanges + // RunOptions are the run options for the bootstrap run. + RunOptions RunOptions } -// ProcessResult is the result of a bootstrap process. -type ProcessResult struct { +// NamespaceDataAccumulator is the namespace data accumulator. +// TODO(r): Consider rename this to a better name. +type NamespaceDataAccumulator interface { + // CheckoutSeriesWithoutLock retrieves a series for writing to + // and when the accumulator is closed it will ensure that the + // series is released. + // If indexing is not enabled, tags is still required, simply pass + // ident.EmptyTagIterator. + // Note: Without lock variant does not perform any locking and callers + // must ensure non-parallel access themselves, this helps avoid + // overhead of the lock for the commit log bootstrapper which reads + // in a single threaded manner. + CheckoutSeriesWithoutLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, + ) (CheckoutSeriesResult, error) + + // CheckoutSeriesWithLock retrieves a series for writing to + // and when the accumulator is closed it will ensure that the + // series is released. + // If indexing is not enabled, tags is still required, simply pass + // ident.EmptyTagIterator. + // Note: With lock variant perform locking and callers do not need + // to be concerned about parallel access. + CheckoutSeriesWithLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, + ) (CheckoutSeriesResult, error) + + // Close will close the data accumulator and will release + // all series read/write refs. + Close() error +} + +// CheckoutSeriesResult is the result of a checkout series operation. +type CheckoutSeriesResult struct { + // Series is the series for the checkout operation. + Series series.DatabaseSeries + // Shard is the shard for the series. + Shard uint32 + // UniqueIndex is the unique index for the series. + UniqueIndex uint64 +} + +// NamespaceResults is the result of a bootstrap process. +type NamespaceResults struct { + // Results is the result of a bootstrap process. + Results *NamespaceResultsMap +} + +// NamespaceResult is the result of a bootstrap process for a given namespace. +type NamespaceResult struct { + Metadata namespace.Metadata + Shards []uint32 DataResult result.DataBootstrapResult IndexResult result.IndexBootstrapResult } @@ -138,16 +239,6 @@ type BootstrapperProvider interface { Provide() (Bootstrapper, error) } -// Strategy describes a bootstrap strategy. -type Strategy int - -const ( - // BootstrapSequential describes whether a bootstrap can use the sequential bootstrap strategy. - BootstrapSequential Strategy = iota - // BootstrapParallel describes whether a bootstrap can use the parallel bootstrap strategy. - BootstrapParallel -) - // Bootstrapper is the interface for different bootstrapping mechanisms. Note that a bootstrapper // can and will be reused so it is important to not rely on state stored in the bootstrapper itself // with the mindset that it will always be set to default values from the constructor. @@ -155,37 +246,19 @@ type Bootstrapper interface { // String returns the name of the bootstrapper String() string - // Can returns whether a specific bootstrapper strategy can be applied. - Can(strategy Strategy) bool - - // BootstrapData performs bootstrapping of data for the given time ranges, returning the bootstrapped - // series data and the time ranges it's unable to fulfill in parallel. A bootstrapper - // should only return an error should it want to entirely cancel the bootstrapping of the - // node, i.e. non-recoverable situation like not being able to read from the filesystem. - BootstrapData( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts RunOptions, - ) (result.DataBootstrapResult, error) - - // BootstrapIndex performs bootstrapping of index blocks for the given time ranges, returning - // the bootstrapped index blocks and the time ranges it's unable to fulfill in parallel. A bootstrapper - // should only return an error should it want to entirely cancel the bootstrapping of the - // node, i.e. non-recoverable situation like not being able to read from the filesystem. - BootstrapIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts RunOptions, - ) (result.IndexBootstrapResult, error) + // Bootstrap performs bootstrapping of series data and index metadata + // for the given time ranges, returning the bootstrapped series data + // and index blocks for the time ranges it's unable to fulfill in parallel. + // A bootstrapper should only return an error should it want to entirely + // cancel the bootstrapping of the node, i.e. non-recoverable situation + // like not being able to read from the filesystem. + Bootstrap(namespaces Namespaces) (NamespaceResults, error) } // Source represents a bootstrap source. Note that a source can and will be reused so // it is important to not rely on state stored in the source itself with the mindset // that it will always be set to default values from the constructor. type Source interface { - // Can returns whether a specific bootstrapper strategy can be applied. - Can(strategy Strategy) bool - // AvailableData returns what time ranges are available for bootstrapping a given set of shards. AvailableData( ns namespace.Metadata, @@ -193,16 +266,6 @@ type Source interface { runOpts RunOptions, ) (result.ShardTimeRanges, error) - // ReadData returns raw series for a given set of shards & specified time ranges and - // the time ranges it's unable to fulfill. A bootstrapper source should only return - // an error should it want to entirely cancel the bootstrapping of the node, - // i.e. non-recoverable situation like not being able to read from the filesystem. - ReadData( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - runOpts RunOptions, - ) (result.DataBootstrapResult, error) - // AvailableIndex returns what time ranges are available for bootstrapping. AvailableIndex( ns namespace.Metadata, @@ -210,10 +273,10 @@ type Source interface { opts RunOptions, ) (result.ShardTimeRanges, error) - // ReadIndex returns series index blocks. - ReadIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts RunOptions, - ) (result.IndexBootstrapResult, error) + // Read returns series data and index metadata for a given set of shards + // and specified time ranges and the time ranges it's unable to fulfill. + // A bootstrapper source should only return an error should it want to + // entirely cancel the bootstrapping of the node, i.e. non-recoverable + // situation like not being able to read from the filesystem. + Read(namespaces Namespaces) (NamespaceResults, error) } diff --git a/src/dbnode/storage/bootstrap/util.go b/src/dbnode/storage/bootstrap/util.go new file mode 100644 index 0000000000..13b934caea --- /dev/null +++ b/src/dbnode/storage/bootstrap/util.go @@ -0,0 +1,732 @@ +// Copyright (c) 2019 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 bootstrap + +import ( + "bytes" + "fmt" + "io" + "math" + "sort" + "sync" + "testing" + "time" + + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/encoding/m3tsz" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/storage/block" + "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" + "github.com/m3db/m3/src/dbnode/storage/series" + "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/context" + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + xtest "github.com/m3db/m3/src/x/test" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// ReadersForID is a slice of readers that share a series ID. +type ReadersForID []ReaderAtTime + +// ReaderMap is a map containing all gathered block segment readers. +type ReaderMap map[string]ReadersForID + +// Must implement NamespaceDataAccumulator. +var _ NamespaceDataAccumulator = (*TestDataAccumulator)(nil) + +// TestDataAccumulator is a NamespaceDataAccumulator that captures any +// series inserts for examination. +type TestDataAccumulator struct { + sync.Mutex + + t *testing.T + ctrl *gomock.Controller + ns string + pool encoding.MultiReaderIteratorPool + loadedBlockMap ReaderMap + schema namespace.SchemaDescr + // writeMap is a map to which values are written directly. + writeMap DecodedBlockMap + results map[string]CheckoutSeriesResult +} + +// DecodedValues is a slice of series datapoints. +type DecodedValues []series.DecodedTestValue + +// DecodedBlockMap is a map of decoded datapoints per series ID. +type DecodedBlockMap map[string]DecodedValues + +func testValuesEqual( + a series.DecodedTestValue, + b series.DecodedTestValue, +) bool { + return a.Timestamp.Equal(b.Timestamp) && + math.Abs(a.Value-b.Value) < 0.000001 && + a.Unit == b.Unit && + bytes.Equal(a.Annotation, b.Annotation) +} + +// VerifyEquals verifies that two DecodedBlockMap are equal; errors otherwise. +func (m DecodedBlockMap) VerifyEquals(other DecodedBlockMap) error { + if len(m) != len(other) { + return fmt.Errorf("block maps of length %d and %d do not match", + len(m), len(other)) + } + + seen := make(map[string]struct{}) + for k, v := range m { + otherSeries, found := other[k] + if !found { + return fmt.Errorf("series %s: values not found", k) + } + + if len(otherSeries) != len(v) { + return fmt.Errorf("series %s: length of series %d does not match other %d", + k, len(v), len(otherSeries)) + } + + // NB: make a clone here to avoid mutating base data + // just in case any tests care about order. + thisVal := append([]series.DecodedTestValue(nil), v...) + otherVal := append([]series.DecodedTestValue(nil), otherSeries...) + + sort.Sort(series.ValuesByTime(thisVal)) + sort.Sort(series.ValuesByTime(otherVal)) + for i, t := range thisVal { + o := otherVal[i] + if !testValuesEqual(t, o) { + return fmt.Errorf("series %s: value %+v does not match other value %+v", + k, t, o) + } + } + + seen[k] = struct{}{} + } + + for k := range other { + if _, beenFound := seen[k]; !beenFound { + return fmt.Errorf("series %s not found in this map", k) + } + } + + return nil +} + +// ReaderAtTime captures incoming block loads, including +// their start times and tags. +type ReaderAtTime struct { + // Start is the block start time. + Start time.Time + // Reader is the block segment reader. + Reader xio.SegmentReader + // Tags is the list of tags in a basic string map format. + Tags map[string]string +} + +// dumpLoadedBlocks decodes any accumulated values gathered from calls to +// series.LoadBlock() and returns them as raw values. +func (a *TestDataAccumulator) dumpLoadedBlocks() DecodedBlockMap { + if len(a.loadedBlockMap) == 0 { + return nil + } + + decodedMap := make(DecodedBlockMap, len(a.loadedBlockMap)) + iter := a.pool.Get() + defer iter.Close() + for k, v := range a.loadedBlockMap { + readers := make([]xio.SegmentReader, 0, len(v)) + for _, r := range v { + readers = append(readers, r.Reader) + } + + value, err := series.DecodeSegmentValues(readers, iter, a.schema) + if err != nil { + if err != io.EOF { + require.NoError(a.t, err) + } + + // NB: print out that we encountered EOF here to assist debugging tests, + // but this is not necessarily a failure. + fmt.Println("EOF: segment had no values.") + } + + sort.Sort(series.ValuesByTime(value)) + decodedMap[k] = value + } + + return decodedMap +} + +// CheckoutSeriesWithLock will retrieve a series for writing to, +// and when the accumulator is closed, it will ensure that the +// series is released (with lock). +func (a *TestDataAccumulator) CheckoutSeriesWithLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (CheckoutSeriesResult, error) { + a.Lock() + defer a.Unlock() + return a.checkoutSeriesWithLock(shardID, id, tags) +} + +// CheckoutSeriesWithoutLock will retrieve a series for writing to, +// and when the accumulator is closed, it will ensure that the +// series is released (without lock). +func (a *TestDataAccumulator) CheckoutSeriesWithoutLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (CheckoutSeriesResult, error) { + return a.checkoutSeriesWithLock(shardID, id, tags) +} + +func (a *TestDataAccumulator) checkoutSeriesWithLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (CheckoutSeriesResult, error) { + var decodedTags map[string]string + if tags != nil { + decodedTags = make(map[string]string, tags.Len()) + for tags.Next() { + tag := tags.Current() + name := tag.Name.String() + value := tag.Value.String() + if len(name) > 0 && len(value) > 0 { + decodedTags[name] = value + } + } + + if err := tags.Err(); err != nil { + return CheckoutSeriesResult{}, err + } + } else { + // Ensure the decoded tags aren't nil. + decodedTags = make(map[string]string) + } + + stringID := id.String() + if result, found := a.results[stringID]; found { + return result, nil + } + + var streamErr error + mockSeries := series.NewMockDatabaseSeries(a.ctrl) + + mockSeries.EXPECT(). + LoadBlock(gomock.Any(), gomock.Any()). + DoAndReturn(func(bl block.DatabaseBlock, _ series.WriteType) error { + reader, err := bl.Stream(context.NewContext()) + if err != nil { + streamErr = err + return err + } + + a.loadedBlockMap[stringID] = append(a.loadedBlockMap[stringID], + ReaderAtTime{ + Start: bl.StartTime(), + Reader: reader, + Tags: decodedTags, + }) + + return nil + }).AnyTimes() + + mockSeries.EXPECT().Write( + gomock.Any(), gomock.Any(), gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn( + func( + _ context.Context, + ts time.Time, + val float64, + unit xtime.Unit, + annotation []byte, + _ series.WriteOptions, + ) (bool, error) { + a.Lock() + a.writeMap[stringID] = append( + a.writeMap[stringID], series.DecodedTestValue{ + Timestamp: ts, + Value: val, + Unit: unit, + Annotation: annotation, + }) + a.Unlock() + return true, nil + }).AnyTimes() + + result := CheckoutSeriesResult{ + Shard: shardID, + Series: mockSeries, + UniqueIndex: uint64(len(a.results) + 1), + } + + a.results[stringID] = result + return result, streamErr +} + +// Release is a no-op on the test accumulator. +func (a *TestDataAccumulator) Release() {} + +// Close is a no-op on the test accumulator. +func (a *TestDataAccumulator) Close() error { return nil } + +// NamespacesTester is a utility to assist testing bootstrapping. +type NamespacesTester struct { + t *testing.T + ctrl *gomock.Controller + pool encoding.MultiReaderIteratorPool + + // Accumulators are the accumulators which incoming blocks get loaded into. + // One per namespace. + Accumulators []*TestDataAccumulator + + // Namespaces are the namespaces for this tester. + Namespaces Namespaces + // Results are the namespace results after bootstrapping. + Results NamespaceResults +} + +func buildDefaultIterPool() encoding.MultiReaderIteratorPool { + iterPool := encoding.NewMultiReaderIteratorPool(pool.NewObjectPoolOptions()) + iterPool.Init( + func(r io.Reader, _ namespace.SchemaDescr) encoding.ReaderIterator { + return m3tsz.NewReaderIterator(r, + m3tsz.DefaultIntOptimizationEnabled, + encoding.NewOptions()) + }) + return iterPool +} + +// BuildNamespacesTester builds a NamespacesTester. +func BuildNamespacesTester( + t *testing.T, + runOpts RunOptions, + ranges result.ShardTimeRanges, + mds ...namespace.Metadata, +) NamespacesTester { + return BuildNamespacesTesterWithReaderIteratorPool( + t, + runOpts, + ranges, + nil, + mds..., + ) +} + +// BuildNamespacesTesterWithReaderIteratorPool builds a NamespacesTester with a +// given MultiReaderIteratorPool. +func BuildNamespacesTesterWithReaderIteratorPool( + t *testing.T, + runOpts RunOptions, + ranges result.ShardTimeRanges, + iterPool encoding.MultiReaderIteratorPool, + mds ...namespace.Metadata, +) NamespacesTester { + shards := make([]uint32, 0, len(ranges)) + for shard := range ranges { + shards = append(shards, shard) + } + + if iterPool == nil { + iterPool = buildDefaultIterPool() + } + + ctrl := xtest.NewController(t) + namespacesMap := NewNamespacesMap(NamespacesMapOptions{}) + accumulators := make([]*TestDataAccumulator, 0, len(mds)) + for _, md := range mds { + nsCtx := namespace.NewContextFrom(md) + acc := &TestDataAccumulator{ + t: t, + ctrl: ctrl, + pool: iterPool, + ns: md.ID().String(), + results: make(map[string]CheckoutSeriesResult), + loadedBlockMap: make(ReaderMap), + writeMap: make(DecodedBlockMap), + schema: nsCtx.Schema, + } + + accumulators = append(accumulators, acc) + namespacesMap.Set(md.ID(), Namespace{ + Metadata: md, + Shards: shards, + DataAccumulator: acc, + DataRunOptions: NamespaceRunOptions{ + ShardTimeRanges: ranges.Copy(), + RunOptions: runOpts, + }, + IndexRunOptions: NamespaceRunOptions{ + ShardTimeRanges: ranges.Copy(), + RunOptions: runOpts, + }, + }) + } + + return NamespacesTester{ + t: t, + ctrl: ctrl, + pool: iterPool, + Accumulators: accumulators, + Namespaces: Namespaces{ + Namespaces: namespacesMap, + }, + } +} + +// DecodedNamespaceMap is a map of decoded blocks per namespace ID. +type DecodedNamespaceMap map[string]DecodedBlockMap + +// DumpLoadedBlocks dumps any loaded blocks as decoded series per namespace. +func (nt *NamespacesTester) DumpLoadedBlocks() DecodedNamespaceMap { + nsMap := make(DecodedNamespaceMap, len(nt.Accumulators)) + for _, acc := range nt.Accumulators { + block := acc.dumpLoadedBlocks() + + if block != nil { + nsMap[acc.ns] = block + } + } + + return nsMap +} + +// EnsureDumpLoadedBlocksForNamespace dumps all loaded blocks as decoded series, +// and fails if the namespace is not found. +func (nt *NamespacesTester) EnsureDumpLoadedBlocksForNamespace( + md namespace.Metadata, +) DecodedBlockMap { + id := md.ID().String() + for _, acc := range nt.Accumulators { + if acc.ns == id { + return acc.dumpLoadedBlocks() + } + } + + assert.FailNow(nt.t, fmt.Sprintf("namespace with id %s not found "+ + "valid namespaces are %v", id, nt.Namespaces)) + return nil +} + +// EnsureNoLoadedBlocks ensures that no blocks have been loaded into any of this +// testers accumulators. +func (nt *NamespacesTester) EnsureNoLoadedBlocks() { + require.Equal(nt.t, 0, len(nt.DumpLoadedBlocks())) +} + +// DumpWrites dumps the writes encountered for all namespaces. +func (nt *NamespacesTester) DumpWrites() DecodedNamespaceMap { + nsMap := make(DecodedNamespaceMap, len(nt.Accumulators)) + for _, acc := range nt.Accumulators { + if len(acc.writeMap) > 0 { + nsMap[acc.ns] = acc.writeMap + } + } + + return nsMap +} + +// EnsureDumpWritesForNamespace dumps the writes encountered for the +// given namespace, and fails if the namespace is not found. +func (nt *NamespacesTester) EnsureDumpWritesForNamespace( + md namespace.Metadata, +) DecodedBlockMap { + id := md.ID().String() + for _, acc := range nt.Accumulators { + if acc.ns == id { + return acc.writeMap + } + } + + assert.FailNow(nt.t, fmt.Sprintf("namespace with id %s not found "+ + "valid namespaces are %v", id, nt.Namespaces)) + return nil +} + +// EnsureNoWrites ensures that no writes have been written into any of this +// testers accumulators. +func (nt *NamespacesTester) EnsureNoWrites() { + require.Equal(nt.t, 0, len(nt.DumpWrites())) +} + +// EnsureDumpAllForNamespace dumps all results for a single namespace, and +// fails if the namespace is not found. The results are unsorted; if sorted +// order is important for verification, they should be sorted afterwards. +func (nt *NamespacesTester) EnsureDumpAllForNamespace( + md namespace.Metadata, +) (DecodedBlockMap, error) { + id := md.ID().String() + for _, acc := range nt.Accumulators { + if acc.ns != id { + continue + } + + writeMap := acc.writeMap + loadedBlockMap := acc.dumpLoadedBlocks() + merged := make(DecodedBlockMap, len(writeMap)+len(loadedBlockMap)) + for k, v := range writeMap { + merged[k] = v + } + + for k, v := range loadedBlockMap { + if vals, found := merged[k]; found { + merged[k] = append(vals, v...) + } else { + merged[k] = v + } + } + + return merged, nil + } + + return nil, fmt.Errorf("namespace with id %s not found "+ + "valid namespaces are %v", id, nt.Namespaces) +} + +// EnsureDumpReadersForNamespace dumps the readers and their start times for a +// given namespace, and fails if the namespace is not found. +func (nt *NamespacesTester) EnsureDumpReadersForNamespace( + md namespace.Metadata, +) ReaderMap { + id := md.ID().String() + for _, acc := range nt.Accumulators { + if acc.ns == id { + return acc.loadedBlockMap + } + } + + assert.FailNow(nt.t, fmt.Sprintf("namespace with id %s not found "+ + "valid namespaces are %v", id, nt.Namespaces)) + return nil +} + +// ResultForNamespace gives the result for the given namespace, and fails if +// the namespace is not found. +func (nt *NamespacesTester) ResultForNamespace(id ident.ID) NamespaceResult { + result, found := nt.Results.Results.Get(id) + require.True(nt.t, found) + return result +} + +// TestBootstrapWith bootstraps the current Namespaces with the +// provided bootstrapper. +func (nt *NamespacesTester) TestBootstrapWith(b Bootstrapper) { + res, err := b.Bootstrap(nt.Namespaces) + assert.NoError(nt.t, err) + nt.Results = res +} + +// TestReadWith reads the current Namespaces with the +// provided bootstrap source. +func (nt *NamespacesTester) TestReadWith(s Source) { + res, err := s.Read(nt.Namespaces) + require.NoError(nt.t, err) + nt.Results = res +} + +func validateRanges(ac xtime.Ranges, ex xtime.Ranges) error { + // Make range eclipses expected. + removedRange := ex.RemoveRanges(ac) + if !removedRange.IsEmpty() { + return fmt.Errorf("actual range %v does not match expected range %v "+ + "diff: %v", ac, ex, removedRange) + } + + // Now make sure no ranges outside of expected. + expectedWithAddedRanges := ex.AddRanges(ac) + if ex.Len() != expectedWithAddedRanges.Len() { + return fmt.Errorf("expected with re-added ranges not equal") + } + + iter := ex.Iter() + withAddedRangesIter := expectedWithAddedRanges.Iter() + for iter.Next() && withAddedRangesIter.Next() { + if !iter.Value().Equal(withAddedRangesIter.Value()) { + return fmt.Errorf("actual range %v does not match expected range %v", + ac, ex) + } + } + + return nil +} + +func validateShardTimeRanges( + r result.ShardTimeRanges, + ex result.ShardTimeRanges, +) error { + if len(ex) != len(r) { + return fmt.Errorf("expected %v and actual %v size mismatch", ex, r) + } + + seen := make(map[uint32]struct{}, len(r)) + for k, val := range r { + expectedVal, found := ex[k] + if !found { + return fmt.Errorf("expected shard map %v does not have shard %d; "+ + "actual: %v", ex, k, r) + } + + if err := validateRanges(val, expectedVal); err != nil { + return err + } + + seen[k] = struct{}{} + } + + for k := range ex { + if _, beenFound := seen[k]; !beenFound { + return fmt.Errorf("shard %d in actual not found in expected %v", k, ex) + } + } + + return nil +} + +// TestUnfulfilledForNamespace ensures the given namespace has the expected +// range flagged as unfulfilled. +func (nt *NamespacesTester) TestUnfulfilledForNamespace( + md namespace.Metadata, + ex result.ShardTimeRanges, + exIdx result.ShardTimeRanges, +) { + ns := nt.ResultForNamespace(md.ID()) + actual := ns.DataResult.Unfulfilled() + require.NoError(nt.t, validateShardTimeRanges(actual, ex), "data") + + if md.Options().IndexOptions().Enabled() { + actual := ns.IndexResult.Unfulfilled() + require.NoError(nt.t, validateShardTimeRanges(actual, exIdx), "index") + } +} + +// TestUnfulfilledForNamespaceIsEmpty ensures the given namespace has an empty +// unfulfilled range. +func (nt *NamespacesTester) TestUnfulfilledForNamespaceIsEmpty( + md namespace.Metadata, +) { + nt.TestUnfulfilledForIDIsEmpty(md.ID(), md.Options().IndexOptions().Enabled()) +} + +// TestUnfulfilledForIDIsEmpty ensures the given id has an empty +// unfulfilled range. +func (nt *NamespacesTester) TestUnfulfilledForIDIsEmpty( + id ident.ID, + useIndex bool, +) { + ns := nt.ResultForNamespace(id) + actual := ns.DataResult.Unfulfilled() + assert.True(nt.t, actual.IsEmpty(), fmt.Sprintf("data: not empty %v", actual)) + + if useIndex { + actual := ns.DataResult.Unfulfilled() + assert.True(nt.t, actual.IsEmpty(), + fmt.Sprintf("index: not empty %v", actual)) + } +} + +// Finish closes the namespaceTester and tests mocks for completion. +func (nt *NamespacesTester) Finish() { + nt.ctrl.Finish() +} + +// NamespaceMatcher is a matcher for namespaces. +type NamespaceMatcher struct { + // Namespaces are the expected namespaces. + Namespaces Namespaces +} + +// String describes what the matcher matches. +func (m NamespaceMatcher) String() string { return "namespace query" } + +// Matches returns whether x is a match. +func (m NamespaceMatcher) Matches(x interface{}) bool { + ns, ok := x.(Namespaces) + if !ok { + return false + } + + equalRange := func(a, b TargetRange) bool { + return a.Range.Start.Equal(b.Range.Start) && + a.Range.End.Equal(b.Range.End) + } + + for _, v := range ns.Namespaces.Iter() { + other, found := m.Namespaces.Namespaces.Get(v.Key()) + if !found { + return false + } + + val := v.Value() + if !other.Metadata.Equal(val.Metadata) { + return false + } + + if !equalRange(val.DataTargetRange, other.DataTargetRange) { + return false + } + + if !equalRange(val.IndexTargetRange, other.IndexTargetRange) { + return false + } + } + + return true +} + +// NB: assert NamespaceMatcher is a gomock.Matcher +var _ gomock.Matcher = (*NamespaceMatcher)(nil) + +// ShardTimeRangesMatcher is a matcher for ShardTimeRanges. +type ShardTimeRangesMatcher struct { + // Ranges are the expected ranges. + Ranges map[uint32]xtime.Ranges +} + +// Matches returns whether x is a match. +func (m ShardTimeRangesMatcher) Matches(x interface{}) bool { + actual, ok := x.(result.ShardTimeRanges) + if !ok { + return false + } + + if err := validateShardTimeRanges(m.Ranges, actual); err != nil { + fmt.Println("shard time ranges do not match:", err.Error()) + return false + } + + return true +} + +// String describes what the matcher matches. +func (m ShardTimeRangesMatcher) String() string { + return "shardTimeRangesMatcher" +} + +// NB: assert ShardTimeRangesMatcher is a gomock.Matcher +var _ gomock.Matcher = (*ShardTimeRangesMatcher)(nil) diff --git a/src/dbnode/storage/bootstrap_test.go b/src/dbnode/storage/bootstrap_test.go index eb0b53fea4..1943cca947 100644 --- a/src/dbnode/storage/bootstrap_test.go +++ b/src/dbnode/storage/bootstrap_test.go @@ -26,6 +26,8 @@ import ( "testing" "time" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/x/ident" "github.com/golang/mock/gomock" @@ -43,9 +45,11 @@ func TestDatabaseBootstrapWithBootstrapError(t *testing.T) { return now })) + id := ident.StringID("a") + meta, err := namespace.NewMetadata(id, namespace.NewOptions()) + require.NoError(t, err) + ns := NewMockdatabaseNamespace(ctrl) - ns.EXPECT().Bootstrap(now, gomock.Any()).Return(fmt.Errorf("an error")) - ns.EXPECT().ID().Return(ident.StringID("test")) namespaces := []databaseNamespace{ns} db := NewMockdatabase(ctrl) @@ -54,12 +58,31 @@ func TestDatabaseBootstrapWithBootstrapError(t *testing.T) { m := NewMockdatabaseMediator(ctrl) m.EXPECT().DisableFileOps() m.EXPECT().EnableFileOps().AnyTimes() - bsm := newBootstrapManager(db, m, opts).(*bootstrapManager) - err := bsm.Bootstrap() - require.NotNil(t, err) - require.Equal(t, "an error", err.Error()) - require.Equal(t, Bootstrapped, bsm.state) + bsm := newBootstrapManager(db, m, opts).(*bootstrapManager) + // Don't sleep. + bsm.sleepFn = func(time.Duration) {} + + gomock.InOrder( + ns.EXPECT().GetOwnedShards().Return([]databaseShard{}), + ns.EXPECT().Metadata().Return(meta), + ns.EXPECT().ID().Return(id), + ns.EXPECT(). + Bootstrap(gomock.Any()). + Return(fmt.Errorf("an error")). + Do(func(bootstrapResult bootstrap.NamespaceResult) { + // After returning an error, make sure we don't re-enqueue. + bsm.bootstrapFn = func() error { + return nil + } + }), + ) + + result, err := bsm.Bootstrap() + require.NoError(t, err) + + require.Equal(t, 1, len(result.ErrorsBootstrap)) + require.Equal(t, "an error", result.ErrorsBootstrap[0].Error()) } func TestDatabaseBootstrapSubsequentCallsQueued(t *testing.T) { @@ -77,21 +100,26 @@ func TestDatabaseBootstrapSubsequentCallsQueued(t *testing.T) { m.EXPECT().EnableFileOps().AnyTimes() db := NewMockdatabase(ctrl) - bsm := newBootstrapManager(db, m, opts).(*bootstrapManager) - ns := NewMockdatabaseNamespace(ctrl) + id := ident.StringID("testBootstrap") + meta, err := namespace.NewMetadata(id, namespace.NewOptions()) + require.NoError(t, err) var wg sync.WaitGroup wg.Add(1) + + ns.EXPECT().GetOwnedShards().Return([]databaseShard{}).AnyTimes() + ns.EXPECT().Metadata().Return(meta).AnyTimes() + ns.EXPECT(). - Bootstrap(now, gomock.Any()). + Bootstrap(gomock.Any()). Return(nil). - Do(func(arg0, arg1 interface{}) { + Do(func(arg0 interface{}) { defer wg.Done() // Enqueue the second bootstrap - err := bsm.Bootstrap() + _, err := bsm.Bootstrap() assert.Error(t, err) assert.Equal(t, errBootstrapEnqueued, err) assert.False(t, bsm.IsBootstrapped()) @@ -100,17 +128,17 @@ func TestDatabaseBootstrapSubsequentCallsQueued(t *testing.T) { bsm.RUnlock() // Expect the second bootstrap call - ns.EXPECT().Bootstrap(now, gomock.Any()).Return(nil) + ns.EXPECT().Bootstrap(gomock.Any()).Return(nil) }) ns.EXPECT(). ID(). - Return(ident.StringID("test")). + Return(id). Times(2) db.EXPECT(). GetOwnedNamespaces(). Return([]databaseNamespace{ns}, nil). Times(2) - err := bsm.Bootstrap() + _, err = bsm.Bootstrap() require.Nil(t, err) } diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 209f93c471..b038564ab8 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -451,8 +451,8 @@ func (d *db) queueBootstrapWithLock() { // enqueue a new bootstrap to execute before the current bootstrap // completes. go func() { - if err := d.mediator.Bootstrap(); err != nil { - d.log.Error("error while bootstrapping", zap.Error(err)) + if result, err := d.mediator.Bootstrap(); err != nil && !result.AlreadyBootstrapping { + d.log.Error("error bootstrapping", zap.Error(err)) } }() } @@ -827,7 +827,8 @@ func (d *db) Bootstrap() error { d.Lock() d.bootstraps++ d.Unlock() - return d.mediator.Bootstrap() + _, err := d.mediator.Bootstrap() + return err } func (d *db) IsBootstrapped() bool { @@ -961,8 +962,9 @@ func (d *db) GetOwnedNamespaces() ([]databaseNamespace, error) { } func (d *db) nextIndex() uint64 { - created := atomic.AddUint64(&d.created, 1) - return created - 1 + // Start with index at "1" so that a default "uniqueIndex" + // with "0" is invalid (AddUint64 will return the new value). + return atomic.AddUint64(&d.created, 1) } type tsIDs []ident.ID diff --git a/src/dbnode/storage/database_test.go b/src/dbnode/storage/database_test.go index 9130454371..40ae74197c 100644 --- a/src/dbnode/storage/database_test.go +++ b/src/dbnode/storage/database_test.go @@ -450,7 +450,7 @@ func TestDatabaseBootstrappedAssignShardSet(t *testing.T) { ns := dbAddNewMockNamespace(ctrl, d, "testns") mediator := NewMockdatabaseMediator(ctrl) - mediator.EXPECT().Bootstrap().Return(nil) + mediator.EXPECT().Bootstrap().Return(BootstrapResult{}, nil) d.mediator = mediator assert.NoError(t, d.Bootstrap()) @@ -464,7 +464,7 @@ func TestDatabaseBootstrappedAssignShardSet(t *testing.T) { var wg sync.WaitGroup wg.Add(1) - mediator.EXPECT().Bootstrap().Return(nil).Do(func() { + mediator.EXPECT().Bootstrap().Return(BootstrapResult{}, nil).Do(func() { wg.Done() }) diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 7d64d82d57..5a61680d82 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -117,6 +117,9 @@ func (m *fsMergeWithMem) ForEachRemaining( for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { seriesID := seriesElement.Value + + // TODO(r): We should really not be looking this up per series element + // and just keep it in the linked list next to the series ID. tags, ok, err := m.shard.TagsFromSeriesID(seriesID) if err != nil { return err @@ -125,6 +128,9 @@ func (m *fsMergeWithMem) ForEachRemaining( // Receiving not ok means that the series was not found, for some // reason like it falling out of retention, therefore we skip this // series and continue. + // TODO(r): This should actually be an invariant error - these should not + // be evicted until a flush otherwise the durability guarantee was not + // upheld. continue } diff --git a/src/dbnode/storage/mediator.go b/src/dbnode/storage/mediator.go index 505dd97a79..4d300c7983 100644 --- a/src/dbnode/storage/mediator.go +++ b/src/dbnode/storage/mediator.go @@ -85,6 +85,8 @@ type mediator struct { closedCh chan struct{} } +// TODO(r): Consider renaming "databaseMediator" to "databaseCoordinator" +// when we have time (now is not that time). func newMediator(database database, commitlog commitlog.CommitLog, opts Options) (databaseMediator, error) { var ( iOpts = opts.InstrumentOptions() diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 584a189a51..2eca04c4d6 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -415,6 +415,14 @@ func (n *dbNamespace) ID() ident.ID { return n.id } +func (n *dbNamespace) Metadata() namespace.Metadata { + // NB(r): metadata is updated in SetSchemaHistory so requires an RLock. + n.RLock() + result := n.metadata + n.RUnlock() + return result +} + func (n *dbNamespace) Schema() namespace.SchemaDescr { n.RLock() schema := n.schemaDescr @@ -660,6 +668,24 @@ func (n *dbNamespace) WriteTagged( return series, wasWritten, err } +func (n *dbNamespace) SeriesReadWriteRef( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (SeriesReadWriteRef, error) { + n.RLock() + shard, err := n.shardAtWithRLock(shardID) + n.RUnlock() + if err != nil { + return SeriesReadWriteRef{}, err + } + + opts := ShardSeriesReadWriteRefOptions{ + ReverseIndex: n.reverseIndex != nil, + } + return shard.SeriesReadWriteRef(id, tags, opts) +} + func (n *dbNamespace) QueryIDs( ctx context.Context, query index.Query, @@ -779,11 +805,12 @@ func (n *dbNamespace) FetchBlocksMetadataV2( return res, nextPageToken, err } -func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) error { +func (n *dbNamespace) Bootstrap( + bootstrapResult bootstrap.NamespaceResult, +) error { callStart := n.nowFn() n.Lock() - metadata := n.metadata if n.bootstrapState == Bootstrapping { n.Unlock() n.metrics.bootstrap.ReportError(n.nowFn().Sub(callStart)) @@ -812,63 +839,50 @@ func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) erro return nil } - var ( - owned = n.GetOwnedShards() - shards = make([]databaseShard, 0, len(owned)) - ) - for _, shard := range owned { - if !shard.IsBootstrapped() { - shards = append(shards, shard) - } - } - if len(shards) == 0 { - success = true - n.metrics.bootstrap.ReportSuccess(n.nowFn().Sub(callStart)) - return nil - } - - shardIDs := make([]uint32, len(shards)) - for i, shard := range shards { - shardIDs[i] = shard.ID() - } - - bootstrapResult, err := process.Run(start, metadata, shardIDs) - if err != nil { - n.log.Error("bootstrap aborted due to error", - zap.Stringer("namespace", n.id), - zap.Error(err)) - return err - } - n.metrics.bootstrap.Success.Inc(1) - // Bootstrap shards using at least half the CPUs available workers := xsync.NewWorkerPool(int(math.Ceil(float64(runtime.NumCPU()) / 2))) workers.Init() - numSeries := bootstrapResult.DataResult.ShardResults().NumSeries() - n.log.Info("bootstrap data fetched now initializing shards with series blocks", - zap.Int("numShards", len(shards)), - zap.Int64("numSeries", numSeries), - ) - var ( - multiErr = xerrors.NewMultiError() - results = bootstrapResult.DataResult.ShardResults() - mutex sync.Mutex - wg sync.WaitGroup + bootstrappedShards = bootstrapResult.Shards + multiErr = xerrors.NewMultiError() + mutex sync.Mutex + wg sync.WaitGroup ) - for _, shard := range shards { - shard := shard - wg.Add(1) - workers.Go(func() { - var bootstrapped *result.Map - if shardResult, ok := results[shard.ID()]; ok { - bootstrapped = shardResult.AllSeries() - } else { - bootstrapped = result.NewMap(result.MapOptions{}) + n.log.Info("bootstrap marking all shards as bootstrapped", + zap.Stringer("namespace", n.id), + zap.Int("numShards", len(bootstrappedShards))) + for _, shard := range n.GetOwnedShards() { + // Make sure it was bootstrapped during this bootstrap run. + shardID := shard.ID() + bootstrapped := false + for _, elem := range bootstrappedShards { + if elem == shardID { + bootstrapped = true + break } + } + if !bootstrapped { + // NB(r): Not bootstrapped in this bootstrap run. + continue + } - err := shard.Bootstrap(bootstrapped) + if shard.IsBootstrapped() { + // No concurrent bootstraps, this is an invariant since + // we only select bootstrapping the shard for a run if it's + // not already bootstrapped. + err := instrument.InvariantErrorf( + "bootstrapper already bootstrapped shard: %d", shardID) + mutex.Lock() + multiErr = multiErr.Add(err) + mutex.Unlock() + continue + } + + wg.Add(1) + shard := shard + workers.Go(func() { + err := shard.Bootstrap() mutex.Lock() multiErr = multiErr.Add(err) @@ -880,30 +894,48 @@ func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) erro wg.Wait() if n.reverseIndex != nil { - err := n.reverseIndex.Bootstrap(bootstrapResult.IndexResult.IndexResults()) + indexResults := bootstrapResult.IndexResult.IndexResults() + n.log.Info("bootstrap index with bootstrapped index segments", + zap.Int("numIndexBlocks", len(indexResults))) + err := n.reverseIndex.Bootstrap(indexResults) multiErr = multiErr.Add(err) } - markAnyUnfulfilled := func(label string, unfulfilled result.ShardTimeRanges) { + markAnyUnfulfilled := func( + bootstrapType string, + unfulfilled result.ShardTimeRanges, + ) error { shardsUnfulfilled := int64(len(unfulfilled)) n.metrics.unfulfilled.Inc(shardsUnfulfilled) - if shardsUnfulfilled > 0 { - str := unfulfilled.SummaryString() - err := fmt.Errorf("bootstrap completed with unfulfilled ranges: %s", str) + if shardsUnfulfilled == 0 { + return nil + } + + errStr := unfulfilled.SummaryString() + errFmt := "bootstrap completed with unfulfilled ranges" + n.log.Error(errFmt, + zap.Error(errors.New(errStr)), + zap.String("namespace", n.id.String()), + zap.String("bootstrapType", bootstrapType)) + return fmt.Errorf("%s: %s", errFmt, errStr) + } + + r := bootstrapResult + if err := markAnyUnfulfilled("data", r.DataResult.Unfulfilled()); err != nil { + multiErr = multiErr.Add(err) + } + if n.reverseIndex != nil { + if err := markAnyUnfulfilled("index", r.IndexResult.Unfulfilled()); err != nil { multiErr = multiErr.Add(err) - n.log.Error(err.Error(), - zap.String("namespace", n.id.String()), - zap.String("bootstrap-type", label), - ) } } - markAnyUnfulfilled("data", bootstrapResult.DataResult.Unfulfilled()) - markAnyUnfulfilled("index", bootstrapResult.IndexResult.Unfulfilled()) - err = multiErr.FinalError() + err := multiErr.FinalError() n.metrics.bootstrap.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - success = err == nil + // NB(r): "success" is read by the defer above and depending on if true/false + // will set the namespace status as bootstrapped or not. + success = err == nil return err } diff --git a/src/dbnode/storage/namespace_bootstrap_data_accumulator.go b/src/dbnode/storage/namespace_bootstrap_data_accumulator.go new file mode 100644 index 0000000000..418b268569 --- /dev/null +++ b/src/dbnode/storage/namespace_bootstrap_data_accumulator.go @@ -0,0 +1,104 @@ +// Copyright (c) 2019 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 storage + +import ( + "errors" + "sync" + + "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/m3db/m3/src/dbnode/storage/series/lookup" + "github.com/m3db/m3/src/x/ident" +) + +var ( + errAlreadyClosed = errors.New("accumulator already closed") +) + +type namespaceDataAccumulator struct { + sync.RWMutex + closed bool + namespace databaseNamespace + needsRelease []lookup.OnReleaseReadWriteRef +} + +// NewDatabaseNamespaceDataAccumulator creates a data accumulator for +// the namespace. +func NewDatabaseNamespaceDataAccumulator( + namespace databaseNamespace, +) bootstrap.NamespaceDataAccumulator { + return &namespaceDataAccumulator{ + namespace: namespace, + } +} + +func (a *namespaceDataAccumulator) CheckoutSeriesWithoutLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (bootstrap.CheckoutSeriesResult, error) { + ref, err := a.namespace.SeriesReadWriteRef(shardID, id, tags) + if err != nil { + return bootstrap.CheckoutSeriesResult{}, err + } + + a.needsRelease = append(a.needsRelease, ref.ReleaseReadWriteRef) + return bootstrap.CheckoutSeriesResult{ + Series: ref.Series, + Shard: ref.Shard, + UniqueIndex: ref.UniqueIndex, + }, nil +} + +func (a *namespaceDataAccumulator) CheckoutSeriesWithLock( + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (bootstrap.CheckoutSeriesResult, error) { + a.Lock() + result, err := a.CheckoutSeriesWithoutLock(shardID, id, tags) + a.Unlock() + return result, err +} + +func (a *namespaceDataAccumulator) Close() error { + a.Lock() + defer a.Unlock() + + if a.closed { + return errAlreadyClosed + } + + a.closed = true + + // Release all refs. + for _, elem := range a.needsRelease { + elem.OnReleaseReadWriteRef() + } + + // Memset optimization for reset. + for i := range a.needsRelease { + a.needsRelease[i] = nil + } + a.needsRelease = a.needsRelease[:0] + + return nil +} diff --git a/src/dbnode/storage/namespace_bootstrap_data_accumulator_test.go b/src/dbnode/storage/namespace_bootstrap_data_accumulator_test.go new file mode 100644 index 0000000000..1764a5ba3e --- /dev/null +++ b/src/dbnode/storage/namespace_bootstrap_data_accumulator_test.go @@ -0,0 +1,159 @@ +// Copyright (c) 2019 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 storage + +import ( + "errors" + "testing" + + "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/m3db/m3/src/dbnode/storage/series" + "github.com/m3db/m3/src/x/ident" + xtest "github.com/m3db/m3/src/x/test" + + "github.com/stretchr/testify/require" +) + +var ( + id = ident.StringID("foo") + idErr = ident.StringID("bar") + tagIter ident.TagIterator + uniqueIdx = uint64(10) +) + +type releaser struct { + calls int +} + +func (r *releaser) OnReleaseReadWriteRef() { + r.calls++ +} + +type checkoutFn func(bootstrap.NamespaceDataAccumulator, uint32, + ident.ID, ident.TagIterator) (bootstrap.CheckoutSeriesResult, error) + +func checkoutWithLock( + acc bootstrap.NamespaceDataAccumulator, + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (bootstrap.CheckoutSeriesResult, error) { + return acc.CheckoutSeriesWithLock(shardID, id, tags) +} + +func checkoutWithoutLock( + acc bootstrap.NamespaceDataAccumulator, + shardID uint32, + id ident.ID, + tags ident.TagIterator, +) (bootstrap.CheckoutSeriesResult, error) { + return acc.CheckoutSeriesWithoutLock(shardID, id, tags) +} + +func TestCheckoutSeries(t *testing.T) { + testCheckoutSeries(t, checkoutWithoutLock) +} + +func TestCheckoutSeriesWithLock(t *testing.T) { + testCheckoutSeries(t, checkoutWithLock) +} + +func testCheckoutSeries(t *testing.T, checkoutFn checkoutFn) { + ctrl := xtest.NewController(t) + defer ctrl.Finish() + var ( + ns = NewMockdatabaseNamespace(ctrl) + series = series.NewMockDatabaseSeries(ctrl) + acc = NewDatabaseNamespaceDataAccumulator(ns) + shardID = uint32(7) + + release = &releaser{} + ref = SeriesReadWriteRef{ + UniqueIndex: uniqueIdx, + Series: series, + ReleaseReadWriteRef: release, + Shard: shardID, + } + ) + + ns.EXPECT().SeriesReadWriteRef(shardID, id, tagIter).Return(ref, nil) + ns.EXPECT().SeriesReadWriteRef(shardID, idErr, tagIter). + Return(SeriesReadWriteRef{}, errors.New("err")) + + _, err := checkoutFn(acc, shardID, idErr, tagIter) + require.Error(t, err) + + seriesResult, err := checkoutFn(acc, shardID, id, tagIter) + require.NoError(t, err) + require.Equal(t, series, seriesResult.Series) + require.Equal(t, uniqueIdx, seriesResult.UniqueIndex) + require.Equal(t, shardID, seriesResult.Shard) + + cast, ok := acc.(*namespaceDataAccumulator) + require.True(t, ok) + require.Equal(t, 1, len(cast.needsRelease)) + require.Equal(t, release, cast.needsRelease[0]) + // Ensure it hasn't been released. + require.Equal(t, 0, release.calls) +} + +func TestAccumulatorRelease(t *testing.T) { + testAccumulatorRelease(t, checkoutWithoutLock) +} + +func TestAccumulatorReleaseWithLock(t *testing.T) { + testAccumulatorRelease(t, checkoutWithLock) +} + +func testAccumulatorRelease(t *testing.T, checkoutFn checkoutFn) { + ctrl := xtest.NewController(t) + defer ctrl.Finish() + + var ( + err error + ns = NewMockdatabaseNamespace(ctrl) + acc = NewDatabaseNamespaceDataAccumulator(ns) + shardID = uint32(1337) + + release = &releaser{} + ref = SeriesReadWriteRef{ + UniqueIndex: uniqueIdx, + Series: series.NewMockDatabaseSeries(ctrl), + ReleaseReadWriteRef: release, + } + ) + + ns.EXPECT().SeriesReadWriteRef(shardID, id, tagIter).Return(ref, nil) + _, err = checkoutFn(acc, shardID, id, tagIter) + require.NoError(t, err) + + cast, ok := acc.(*namespaceDataAccumulator) + require.True(t, ok) + require.Equal(t, 1, len(cast.needsRelease)) + require.Equal(t, release, cast.needsRelease[0]) + + require.NoError(t, acc.Close()) + require.Equal(t, 0, len(cast.needsRelease)) + // ensure release has been called. + require.Equal(t, 1, release.calls) + // ensure double-close errors. + require.Error(t, acc.Close()) +} diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 53479e78a2..fc69e57a8b 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -324,52 +324,48 @@ func TestNamespaceBootstrapBootstrapping(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() ns.bootstrapState = Bootstrapping - require.Equal(t, errNamespaceIsBootstrapping, ns.Bootstrap(time.Now(), nil)) + require.Equal(t, + errNamespaceIsBootstrapping, ns.Bootstrap(bootstrap.NamespaceResult{})) } func TestNamespaceBootstrapDontNeedBootstrap(t *testing.T) { ns, closer := newTestNamespaceWithIDOpts(t, defaultTestNs1ID, namespace.NewOptions().SetBootstrapEnabled(false)) defer closer() - require.NoError(t, ns.Bootstrap(time.Now(), nil)) + require.NoError(t, ns.Bootstrap(bootstrap.NamespaceResult{})) require.Equal(t, Bootstrapped, ns.bootstrapState) } func TestNamespaceBootstrapAllShards(t *testing.T) { - ctrl := gomock.NewController(xtest.Reporter{T: t}) + ctrl := xtest.NewController(t) defer ctrl.Finish() ns, closer := newTestNamespace(t) defer closer() - start := time.Now() - errs := []error{nil, errors.New("foo")} - bs := bootstrap.NewMockProcess(ctrl) - bs.EXPECT(). - Run(start, ns.metadata, sharding.IDs(testShardIDs)). - Return(bootstrap.ProcessResult{ - DataResult: result.NewDataBootstrapResult(), - IndexResult: result.NewIndexBootstrapResult(), - }, nil) - shardIDs := make([]uint32, 0, len(errs)) for i := range errs { shardID := uint32(i) shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(false) - shard.EXPECT().ID().Return(shardID).AnyTimes() - shard.EXPECT().Bootstrap(gomock.Any()).Return(errs[i]) + shard.EXPECT().ID().Return(shardID) + shard.EXPECT().Bootstrap().Return(errs[i]) ns.shards[testShardIDs[i].ID()] = shard shardIDs = append(shardIDs, shardID) } - require.Equal(t, "foo", ns.Bootstrap(start, bs).Error()) + nsResult := bootstrap.NamespaceResult{ + DataResult: result.NewDataBootstrapResult(), + Shards: shardIDs, + } + + require.Equal(t, "foo", ns.Bootstrap(nsResult).Error()) require.Equal(t, BootstrapNotStarted, ns.bootstrapState) } func TestNamespaceBootstrapOnlyNonBootstrappedShards(t *testing.T) { - ctrl := gomock.NewController(t) + ctrl := xtest.NewController(t) defer ctrl.Finish() var ( @@ -391,31 +387,31 @@ func TestNamespaceBootstrapOnlyNonBootstrappedShards(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() - start := time.Now() - - bs := bootstrap.NewMockProcess(ctrl) - bs.EXPECT(). - Run(start, ns.metadata, sharding.IDs(needsBootstrap)). - Return(bootstrap.ProcessResult{ - DataResult: result.NewDataBootstrapResult(), - IndexResult: result.NewIndexBootstrapResult(), - }, nil) - + shardIDs := make([]uint32, 0, len(needsBootstrap)) for _, testShard := range needsBootstrap { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(false) - shard.EXPECT().ID().Return(testShard.ID()).AnyTimes() - shard.EXPECT().Bootstrap(gomock.Any()).Return(nil) + shard.EXPECT().ID().Return(testShard.ID()) + shard.EXPECT().Bootstrap().Return(nil) ns.shards[testShard.ID()] = shard + shardIDs = append(shardIDs, testShard.ID()) } + for _, testShard := range alreadyBootstrapped { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(true) + shard.EXPECT().ID().Return(testShard.ID()) ns.shards[testShard.ID()] = shard + shardIDs = append(shardIDs, testShard.ID()) } - require.NoError(t, ns.Bootstrap(start, bs)) - require.Equal(t, Bootstrapped, ns.bootstrapState) + nsResult := bootstrap.NamespaceResult{ + DataResult: result.NewDataBootstrapResult(), + Shards: shardIDs, + } + + require.Error(t, ns.Bootstrap(nsResult)) + require.Equal(t, BootstrapNotStarted, ns.bootstrapState) } func TestNamespaceFlushNotBootstrapped(t *testing.T) { @@ -1187,7 +1183,7 @@ func TestNamespaceTicksIndex(t *testing.T) { defer closer() for _, s := range ns.shards { if s != nil { - s.Bootstrap(nil) + s.Bootstrap() } } diff --git a/src/dbnode/storage/repair.go b/src/dbnode/storage/repair.go index d48f0df282..37327567a9 100644 --- a/src/dbnode/storage/repair.go +++ b/src/dbnode/storage/repair.go @@ -336,7 +336,7 @@ func (r shardRepairer) loadDataIntoShard(shard databaseShard, data result.ShardR }() for { - err := shard.Load(data.AllSeries()) + err := shard.LoadBlocks(data.AllSeries()) if err == ErrDatabaseLoadLimitHit { waitedCounter.Inc(1) waitingLock.Lock() diff --git a/src/dbnode/storage/repair_test.go b/src/dbnode/storage/repair_test.go index e22d92d1ff..fce5974335 100644 --- a/src/dbnode/storage/repair_test.go +++ b/src/dbnode/storage/repair_test.go @@ -167,8 +167,8 @@ func testDatabaseShardRepairerRepair(t *testing.T, withLimit bool) { if withLimit { numIters = 2 - shard.EXPECT().Load(gomock.Any()).Return(nil) - shard.EXPECT().Load(gomock.Any()).DoAndReturn(func(*result.Map) error { + shard.EXPECT().LoadBlocks(gomock.Any()).Return(nil) + shard.EXPECT().LoadBlocks(gomock.Any()).DoAndReturn(func(*result.Map) error { // Return an error that we've hit the limit, but also start a delayed // goroutine to release the throttle repair process. go func() { @@ -177,10 +177,11 @@ func testDatabaseShardRepairerRepair(t *testing.T, withLimit bool) { }() return ErrDatabaseLoadLimitHit }) - shard.EXPECT().Load(gomock.Any()).Return(nil) + shard.EXPECT().LoadBlocks(gomock.Any()).Return(nil) } else { - shard.EXPECT().Load(gomock.Any()) + shard.EXPECT().LoadBlocks(gomock.Any()).Return(nil) } + for i := 0; i < numIters; i++ { expectedResults := block.NewFetchBlocksMetadataResults() results := block.NewFetchBlockMetadataResults() @@ -417,7 +418,7 @@ func TestDatabaseShardRepairerRepairMultiSession(t *testing.T) { FetchBlocksMetadataV2(any, start, end, any, nonNilPageToken, fetchOpts). Return(expectedResults, nil, nil) shard.EXPECT().ID().Return(shardID).AnyTimes() - shard.EXPECT().Load(gomock.Any()) + shard.EXPECT().LoadBlocks(gomock.Any()).Return(nil) inputBlocks := []block.ReplicaMetadata{ { diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 2d0870a52b..f1dba7c0b2 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -272,6 +272,11 @@ func (b *dbBuffer) Write( writeType WriteType ) switch { + case wOpts.BootstrapWrite: + writeType = WarmWrite + // Bootstrap writes are always warm writes. + // TODO(r): Validate that the block doesn't reside on disk by asking + // the shard for it's bootstrap flush states. case !pastLimit.Before(timestamp): writeType = ColdWrite if !b.coldWritesEnabled { @@ -432,6 +437,9 @@ func (b *dbBuffer) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Con } func (b *dbBuffer) Load(bl block.DatabaseBlock, writeType WriteType) { + // TODO(r): If warm write then validate that the block doesn't reside on + // disk by asking the shard for its bootstrap flush states and verifying + // that the block does not exist yet. var ( blockStart = bl.StartTime() buckets = b.bucketVersionsAtCreate(blockStart) @@ -1166,12 +1174,11 @@ func (b *BufferBucket) writeToEncoderIndex( func (b *BufferBucket) streams(ctx context.Context) []xio.BlockReader { streams := make([]xio.BlockReader, 0, len(b.loadedBlocks)+len(b.encoders)) - - for i := range b.loadedBlocks { - if b.loadedBlocks[i].Len() == 0 { + for _, bl := range b.loadedBlocks { + if bl.Len() == 0 { continue } - if s, err := b.loadedBlocks[i].Stream(ctx); err == nil && s.IsNotEmpty() { + if s, err := bl.Stream(ctx); err == nil && s.IsNotEmpty() { // NB(r): block stream method will register the stream closer already streams = append(streams, s) } diff --git a/src/dbnode/storage/series/buffer_proto_test.go b/src/dbnode/storage/series/buffer_proto_test.go index 5ae0484580..926c8bd0bb 100644 --- a/src/dbnode/storage/series/buffer_proto_test.go +++ b/src/dbnode/storage/series/buffer_proto_test.go @@ -37,7 +37,8 @@ var ( testSchemaDesc = namespace.GetTestSchemaDescr(testSchema) testProtoMessages = prototest.NewProtoTestMessages(testSchema) testProtoEqual = func(t *testing.T, expect, actual []byte) { - prototest.RequireEqual(t, testSchema, expect, actual)} + prototest.RequireEqual(t, testSchema, expect, actual) + } ) func newBufferTestProtoOptions(t *testing.T) Options { @@ -62,11 +63,11 @@ func newBufferTestProtoOptions(t *testing.T) Options { return opts } -func testSetProtoAnnotation(data []value) []value { +func testSetProtoAnnotation(data []DecodedTestValue) []DecodedTestValue { protoIter := prototest.NewProtoMessageIterator(testProtoMessages) for i := 0; i < len(data); i++ { - data[i].value = 0 - data[i].annotation = protoIter.Next() + data[i].Value = 0 + data[i].Annotation = protoIter.Next() } return data } diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index b8ccbb9c5d..26c70d45f9 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -77,9 +77,11 @@ func newBufferTestOptions() Options { } // Writes to buffer, verifying no error and that further writes should happen. -func verifyWriteToBuffer(t *testing.T, buffer databaseBuffer, v value, schema namespace.SchemaDescr) { +func verifyWriteToBuffer(t *testing.T, buffer databaseBuffer, + v DecodedTestValue, schema namespace.SchemaDescr) { ctx := context.NewContext() - wasWritten, err := buffer.Write(ctx, v.timestamp, v.value, v.unit, v.annotation, WriteOptions{SchemaDesc: schema}) + wasWritten, err := buffer.Write(ctx, v.Timestamp, v.Value, v.Unit, + v.Annotation, WriteOptions{SchemaDesc: schema}) require.NoError(t, err) require.True(t, wasWritten) ctx.Close() @@ -164,7 +166,7 @@ func testBufferWriteRead(t *testing.T, opts Options, setAnn setAnnotation) { buffer := newDatabaseBuffer().(*dbBuffer) buffer.Reset(ident.StringID("foo"), opts) - data := []value{ + data := []DecodedTestValue{ {curr.Add(secs(1)), 1, xtime.Second, nil}, {curr.Add(secs(2)), 2, xtime.Second, nil}, {curr.Add(secs(3)), 3, xtime.Second, nil}, @@ -200,13 +202,13 @@ func TestBufferReadOnlyMatchingBuckets(t *testing.T) { buffer := newDatabaseBuffer().(*dbBuffer) buffer.Reset(ident.StringID("foo"), opts) - data := []value{ + data := []DecodedTestValue{ {curr.Add(mins(1)), 1, xtime.Second, nil}, {curr.Add(mins(3)), 2, xtime.Second, nil}, } for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToBuffer(t, buffer, v, nil) } @@ -218,7 +220,7 @@ func TestBufferReadOnlyMatchingBuckets(t *testing.T) { results, err := buffer.ReadEncoded(ctx, firstBucketStart, firstBucketEnd, namespace.Context{}) assert.NoError(t, err) assert.NotNil(t, results) - requireReaderValuesEqual(t, []value{data[0]}, results, opts, namespace.Context{}) + requireReaderValuesEqual(t, []DecodedTestValue{data[0]}, results, opts, namespace.Context{}) secondBucketStart := start.Add(mins(2)).Truncate(time.Second) secondBucketEnd := start.Add(mins(4)).Truncate(time.Second) @@ -226,7 +228,7 @@ func TestBufferReadOnlyMatchingBuckets(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, results) - requireReaderValuesEqual(t, []value{data[1]}, results, opts, namespace.Context{}) + requireReaderValuesEqual(t, []DecodedTestValue{data[1]}, results, opts, namespace.Context{}) } func TestBufferWriteOutOfOrder(t *testing.T) { @@ -240,15 +242,15 @@ func TestBufferWriteOutOfOrder(t *testing.T) { buffer := newDatabaseBuffer().(*dbBuffer) buffer.Reset(ident.StringID("foo"), opts) - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(secs(10)), 2, xtime.Second, nil}, {curr.Add(secs(5)), 3, xtime.Second, nil}, } for _, v := range data { - if v.timestamp.After(curr) { - curr = v.timestamp + if v.Timestamp.After(curr) { + curr = v.Timestamp } verifyWriteToBuffer(t, buffer, v, nil) } @@ -258,11 +260,11 @@ func TestBufferWriteOutOfOrder(t *testing.T) { bucket, ok := buckets.writableBucket(WarmWrite) require.True(t, ok) assert.Equal(t, 2, len(bucket.encoders)) - assert.Equal(t, data[1].timestamp, mustGetLastEncoded(t, bucket.encoders[0]).Timestamp) - assert.Equal(t, data[2].timestamp, mustGetLastEncoded(t, bucket.encoders[1]).Timestamp) + assert.Equal(t, data[1].Timestamp, mustGetLastEncoded(t, bucket.encoders[0]).Timestamp) + assert.Equal(t, data[2].Timestamp, mustGetLastEncoded(t, bucket.encoders[1]).Timestamp) // Restore data to in order for comparison. - sort.Sort(valuesByTime(data)) + sort.Sort(ValuesByTime(data)) ctx := context.NewContext() defer ctx.Close() @@ -274,14 +276,15 @@ func TestBufferWriteOutOfOrder(t *testing.T) { requireReaderValuesEqual(t, data, results, opts, namespace.Context{}) } -func newTestBufferBucketWithData(t *testing.T, opts Options, setAnn setAnnotation) (*BufferBucket, []value) { +func newTestBufferBucketWithData(t *testing.T, + opts Options, setAnn setAnnotation) (*BufferBucket, []DecodedTestValue) { rops := opts.RetentionOptions() curr := time.Now().Truncate(rops.BlockSize()) bd := blockData{ start: curr, writeType: WarmWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {curr, 1, xtime.Second, nil}, {curr.Add(secs(10)), 2, xtime.Second, nil}, @@ -310,7 +313,7 @@ func newTestBufferBucketWithCustomData( bd blockData, opts Options, setAnn setAnnotation, -) (*BufferBucket, []value) { +) (*BufferBucket, []DecodedTestValue) { b := &BufferBucket{opts: opts} b.resetTo(bd.start, bd.writeType, opts) data := bd.data @@ -322,7 +325,7 @@ func newTestBufferBucketWithCustomData( if setAnn != nil { nsCtx = namespace.Context{Schema: testSchemaDesc} } - var expected []value + var expected []DecodedTestValue for i := 0; i < len(data); i++ { if setAnn != nil { data[i] = setAnn(data[i]) @@ -333,21 +336,22 @@ func newTestBufferBucketWithCustomData( encoder.Reset(bd.start, 0, nsCtx.Schema) for _, v := range data[i] { dp := ts.Datapoint{ - Timestamp: v.timestamp, - Value: v.value, + Timestamp: v.Timestamp, + Value: v.Value, } - err := encoder.Encode(dp, v.unit, v.annotation) + err := encoder.Encode(dp, v.Unit, v.Annotation) require.NoError(t, err) encoded++ } b.encoders = append(b.encoders, inOrderEncoder{encoder: encoder}) expected = append(expected, data[i]...) } - sort.Sort(valuesByTime(expected)) + sort.Sort(ValuesByTime(expected)) return b, expected } -func newTestBufferBucketsWithData(t *testing.T, opts Options, setAnn setAnnotation) (*BufferBucketVersions, []value) { +func newTestBufferBucketsWithData(t *testing.T, opts Options, + setAnn setAnnotation) (*BufferBucketVersions, []DecodedTestValue) { newBucket, vals := newTestBufferBucketWithData(t, opts, setAnn) return &BufferBucketVersions{ buckets: []*BufferBucket{newBucket}, @@ -361,7 +365,7 @@ func newTestBufferBucketVersionsWithCustomData( bd blockData, opts Options, setAnn setAnnotation, -) (*BufferBucketVersions, []value) { +) (*BufferBucketVersions, []DecodedTestValue) { newBucket, vals := newTestBufferBucketWithCustomData(t, bd, opts, setAnn) return &BufferBucketVersions{ buckets: []*BufferBucket{newBucket}, @@ -375,10 +379,10 @@ func newTestBufferWithCustomData( blockDatas []blockData, opts Options, setAnn setAnnotation, -) (*dbBuffer, map[xtime.UnixNano][]value) { +) (*dbBuffer, map[xtime.UnixNano][]DecodedTestValue) { buffer := newDatabaseBuffer().(*dbBuffer) buffer.Reset(ident.StringID("foo"), opts) - expectedMap := make(map[xtime.UnixNano][]value) + expectedMap := make(map[xtime.UnixNano][]DecodedTestValue) for _, bd := range blockDatas { bucketVersions, expected := newTestBufferBucketVersionsWithCustomData(t, bd, opts, setAnn) @@ -464,7 +468,7 @@ func TestBufferBucketWriteDuplicateUpserts(t *testing.T) { b := &BufferBucket{} b.resetTo(curr, WarmWrite, opts) - data := [][]value{ + data := [][]DecodedTestValue{ { {curr, 1, xtime.Second, nil}, {curr.Add(secs(10)), 2, xtime.Second, nil}, @@ -486,7 +490,7 @@ func TestBufferBucketWriteDuplicateUpserts(t *testing.T) { }, } - expected := []value{ + expected := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(secs(10)), 10, xtime.Second, nil}, {curr.Add(secs(40)), 8, xtime.Second, nil}, @@ -498,8 +502,8 @@ func TestBufferBucketWriteDuplicateUpserts(t *testing.T) { for _, values := range data { for _, value := range values { - wasWritten, err := b.write(value.timestamp, value.value, - value.unit, value.annotation, nil) + wasWritten, err := b.write(value.Timestamp, value.Value, + value.Unit, value.Annotation, nil) require.NoError(t, err) require.True(t, wasWritten) } @@ -531,34 +535,34 @@ func TestBufferBucketDuplicatePointsNotWrittenButUpserted(t *testing.T) { b.resetTo(curr, WarmWrite, opts) type dataWithShouldWrite struct { - v value + v DecodedTestValue w bool } data := [][]dataWithShouldWrite{ { - {w: true, v: value{curr, 1, xtime.Second, nil}}, - {w: false, v: value{curr, 1, xtime.Second, nil}}, - {w: false, v: value{curr, 1, xtime.Second, nil}}, - {w: false, v: value{curr, 1, xtime.Second, nil}}, - {w: true, v: value{curr.Add(secs(10)), 2, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: false, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: false, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: false, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr.Add(secs(10)), 2, xtime.Second, nil}}, }, { - {w: true, v: value{curr, 1, xtime.Second, nil}}, - {w: false, v: value{curr.Add(secs(10)), 2, xtime.Second, nil}}, - {w: true, v: value{curr.Add(secs(10)), 5, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: false, v: DecodedTestValue{curr.Add(secs(10)), 2, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr.Add(secs(10)), 5, xtime.Second, nil}}, }, { - {w: true, v: value{curr, 1, xtime.Second, nil}}, - {w: true, v: value{curr.Add(secs(20)), 8, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr, 1, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr.Add(secs(20)), 8, xtime.Second, nil}}, }, { - {w: true, v: value{curr, 10, xtime.Second, nil}}, - {w: true, v: value{curr.Add(secs(20)), 10, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr, 10, xtime.Second, nil}}, + {w: true, v: DecodedTestValue{curr.Add(secs(20)), 10, xtime.Second, nil}}, }, } - expected := []value{ + expected := []DecodedTestValue{ {curr, 10, xtime.Second, nil}, {curr.Add(secs(10)), 5, xtime.Second, nil}, {curr.Add(secs(20)), 10, xtime.Second, nil}, @@ -567,8 +571,8 @@ func TestBufferBucketDuplicatePointsNotWrittenButUpserted(t *testing.T) { for _, valuesWithMeta := range data { for _, valueWithMeta := range valuesWithMeta { value := valueWithMeta.v - wasWritten, err := b.write(value.timestamp, value.value, - value.unit, value.annotation, nil) + wasWritten, err := b.write(value.Timestamp, value.Value, + value.Unit, value.Annotation, nil) require.NoError(t, err) assert.Equal(t, valueWithMeta.w, wasWritten) } @@ -602,7 +606,7 @@ func TestIndexedBufferWriteOnlyWritesSinglePoint(t *testing.T) { buffer := newDatabaseBuffer().(*dbBuffer) buffer.Reset(ident.StringID("foo"), opts) - data := []value{ + data := []DecodedTestValue{ {curr.Add(secs(1)), 1, xtime.Second, nil}, {curr.Add(secs(2)), 2, xtime.Second, nil}, {curr.Add(secs(3)), 3, xtime.Second, nil}, @@ -618,8 +622,8 @@ func TestIndexedBufferWriteOnlyWritesSinglePoint(t *testing.T) { ForceValue: forceValue, }, } - wasWritten, err := buffer.Write(ctx, v.timestamp, v.value, v.unit, - v.annotation, writeOpts) + wasWritten, err := buffer.Write(ctx, v.Timestamp, v.Value, v.Unit, + v.Annotation, writeOpts) require.NoError(t, err) expectedWrite := i == 0 require.Equal(t, expectedWrite, wasWritten) @@ -633,7 +637,7 @@ func TestIndexedBufferWriteOnlyWritesSinglePoint(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, results) - ex := []value{ + ex := []DecodedTestValue{ {curr, forceValue, xtime.Second, nil}, } @@ -680,7 +684,7 @@ func TestBufferFetchBlocksOneResultPerBlock(t *testing.T) { coldBucket.resetTo(curr, ColdWrite, opts) coldBucket.encoders = nil buckets := []*BufferBucket{warmBucket, coldBucket} - warmEncoder := [][]value{ + warmEncoder := [][]DecodedTestValue{ { {curr, 1, xtime.Second, nil}, {curr.Add(secs(10)), 2, xtime.Second, nil}, @@ -699,14 +703,14 @@ func TestBufferFetchBlocksOneResultPerBlock(t *testing.T) { {curr.Add(secs(35)), 6, xtime.Second, nil}, }, } - coldEncoder := [][]value{ + coldEncoder := [][]DecodedTestValue{ { {curr.Add(secs(15)), 10, xtime.Second, nil}, {curr.Add(secs(25)), 20, xtime.Second, nil}, {curr.Add(secs(40)), 30, xtime.Second, nil}, }, } - data := [][][]value{warmEncoder, coldEncoder} + data := [][][]DecodedTestValue{warmEncoder, coldEncoder} for i, bucket := range data { for _, d := range bucket { @@ -715,10 +719,10 @@ func TestBufferFetchBlocksOneResultPerBlock(t *testing.T) { encoder.Reset(curr, 0, nil) for _, v := range d { dp := ts.Datapoint{ - Timestamp: v.timestamp, - Value: v.value, + Timestamp: v.Timestamp, + Value: v.Value, } - err := encoder.Encode(dp, v.unit, v.annotation) + err := encoder.Encode(dp, v.Unit, v.Annotation) require.NoError(t, err) encoded++ } @@ -808,7 +812,7 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { buffer.Reset(ident.StringID("foo"), opts) // Perform out of order writes that will create two in order encoders. - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -816,10 +820,10 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { {curr.Add(mins(1.5)), 5, xtime.Second, nil}, {curr.Add(mins(1.5)).Add(-5 * time.Second), 6, xtime.Second, nil}, } - end := data[len(data)-1].timestamp.Add(time.Nanosecond) + end := data[len(data)-1].Timestamp.Add(time.Nanosecond) for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToBuffer(t, buffer, v, nil) } @@ -856,9 +860,9 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { // Check values correct. results, err := buffer.ReadEncoded(ctx, start, end, namespace.Context{}) assert.NoError(t, err) - expected := make([]value, len(data)) + expected := make([]DecodedTestValue, len(data)) copy(expected, data) - sort.Sort(valuesByTime(expected)) + sort.Sort(ValuesByTime(expected)) requireReaderValuesEqual(t, expected, results, opts, namespace.Context{}) // Count the encoders again. @@ -896,7 +900,7 @@ func TestBufferRemoveBucket(t *testing.T) { buffer.Reset(ident.StringID("foo"), opts) // Perform out of order writes that will create two in order encoders. - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -906,7 +910,7 @@ func TestBufferRemoveBucket(t *testing.T) { } for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToBuffer(t, buffer, v, nil) } @@ -1063,7 +1067,7 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { // Create test data to perform out of order writes that will create two in-order // encoders so we can verify that Snapshot will perform a merge. - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -1082,7 +1086,7 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { // Perform the writes. for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) } @@ -1108,9 +1112,9 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { assertPersistDataFn := func(id ident.ID, tags ident.Tags, segment ts.Segment, checlsum uint32) error { // Check we got the right results. expectedData := data[:len(data)-1] // -1 because we don't expect the last datapoint. - expectedCopy := make([]value, len(expectedData)) + expectedCopy := make([]DecodedTestValue, len(expectedData)) copy(expectedCopy, expectedData) - sort.Sort(valuesByTime(expectedCopy)) + sort.Sort(ValuesByTime(expectedCopy)) actual := [][]xio.BlockReader{{ xio.BlockReader{ SegmentReader: xio.NewSegmentReader(segment), @@ -1163,7 +1167,7 @@ func TestBufferSnapshotWithColdWrites(t *testing.T) { // Create test data to perform warm writes that will create two in-order // encoders so we can verify that Snapshot will perform a merge. - warmData := []value{ + warmData := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -1179,7 +1183,7 @@ func TestBufferSnapshotWithColdWrites(t *testing.T) { // Perform warm writes. for _, v := range warmData { // Set curr so that every write is a warm write. - curr = v.timestamp + curr = v.Timestamp verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) } @@ -1193,7 +1197,7 @@ func TestBufferSnapshotWithColdWrites(t *testing.T) { // the same timestamps used in the warm writes above, otherwise these will // overwrite them. // Buffer past/future in this test case is 10 seconds. - coldData := []value{ + coldData := []DecodedTestValue{ {start.Add(secs(2)), 11, xtime.Second, nil}, {start.Add(secs(4)), 12, xtime.Second, nil}, {start.Add(secs(6)), 13, xtime.Second, nil}, @@ -1249,9 +1253,9 @@ func TestBufferSnapshotWithColdWrites(t *testing.T) { // since it's for a different block. expectedData := warmData[:len(warmData)-1] expectedData = append(expectedData, coldData...) - expectedCopy := make([]value, len(expectedData)) + expectedCopy := make([]DecodedTestValue, len(expectedData)) copy(expectedCopy, expectedData) - sort.Sort(valuesByTime(expectedCopy)) + sort.Sort(ValuesByTime(expectedCopy)) actual := [][]xio.BlockReader{{ xio.BlockReader{ SegmentReader: xio.NewSegmentReader(segment), @@ -1423,7 +1427,7 @@ func TestColdFlushBlockStarts(t *testing.T) { blockData{ start: blockStart1, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart1, 1, xtime.Second, nil}, {blockStart1.Add(secs(5)), 2, xtime.Second, nil}, @@ -1434,7 +1438,7 @@ func TestColdFlushBlockStarts(t *testing.T) { blockData{ start: blockStart2, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart2.Add(secs(2)), 4, xtime.Second, nil}, {blockStart2.Add(secs(5)), 5, xtime.Second, nil}, @@ -1447,7 +1451,7 @@ func TestColdFlushBlockStarts(t *testing.T) { blockData{ start: blockStart3, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart3.Add(secs(71)), 9, xtime.Second, nil}, }, @@ -1456,7 +1460,7 @@ func TestColdFlushBlockStarts(t *testing.T) { blockData{ start: blockStart4, writeType: WarmWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart4.Add(secs(57)), 10, xtime.Second, nil}, {blockStart4.Add(secs(66)), 11, xtime.Second, nil}, @@ -1539,7 +1543,7 @@ func TestFetchBlocksForColdFlush(t *testing.T) { blockData{ start: blockStart1, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart1, 1, xtime.Second, nil}, {blockStart1.Add(secs(5)), 2, xtime.Second, nil}, @@ -1550,7 +1554,7 @@ func TestFetchBlocksForColdFlush(t *testing.T) { blockData{ start: blockStart2, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart2.Add(secs(2)), 4, xtime.Second, nil}, {blockStart2.Add(secs(5)), 5, xtime.Second, nil}, @@ -1563,7 +1567,7 @@ func TestFetchBlocksForColdFlush(t *testing.T) { blockData{ start: blockStart3, writeType: ColdWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart3.Add(secs(71)), 9, xtime.Second, nil}, }, @@ -1572,7 +1576,7 @@ func TestFetchBlocksForColdFlush(t *testing.T) { blockData{ start: blockStart4, writeType: WarmWrite, - data: [][]value{ + data: [][]DecodedTestValue{ { {blockStart4.Add(secs(57)), 10, xtime.Second, nil}, {blockStart4.Add(secs(66)), 11, xtime.Second, nil}, @@ -1609,7 +1613,7 @@ func TestFetchBlocksForColdFlush(t *testing.T) { // but is not an error. reader, err = buffer.FetchBlocksForColdFlush(ctx, blockStart4, 1, nsCtx) assert.NoError(t, err) - requireReaderValuesEqual(t, []value{}, [][]xio.BlockReader{reader}, opts, nsCtx) + requireReaderValuesEqual(t, []DecodedTestValue{}, [][]xio.BlockReader{reader}, opts, nsCtx) } // TestBufferLoadWarmWrite tests the Load method, ensuring that blocks are successfully loaded into diff --git a/src/dbnode/storage/series/lookup/entry.go b/src/dbnode/storage/series/lookup/entry.go index 23784f7c82..1dd242d1fe 100644 --- a/src/dbnode/storage/series/lookup/entry.go +++ b/src/dbnode/storage/series/lookup/entry.go @@ -44,6 +44,15 @@ type Entry struct { reverseIndex entryIndexState } +// OnReleaseReadWriteRef is a callback that can release +// a strongly held series read/write ref. +type OnReleaseReadWriteRef interface { + OnReleaseReadWriteRef() +} + +// ensure Entry satifies the `OnReleaseReadWriteRef` interface. +var _ OnReleaseReadWriteRef = &Entry{} + // ensure Entry satisfies the `index.OnIndexSeries` interface. var _ index.OnIndexSeries = &Entry{} @@ -72,6 +81,15 @@ func (entry *Entry) DecrementReaderWriterCount() { atomic.AddInt32(&entry.curReadWriters, -1) } +// OnReleaseReadWriteRef decrements a read/write ref, it's named +// differently to decouple the concrete task needed when a ref +// is released and the intent to release the ref (simpler for +// caller readability/reasoning). +func (entry *Entry) OnReleaseReadWriteRef() { + // All we do when we release a read/write ref is decrement. + entry.DecrementReaderWriterCount() +} + // IndexedForBlockStart returns a bool to indicate if the Entry has been successfully // indexed for the given index blockstart. func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { diff --git a/src/dbnode/storage/series/lookup/lookup_mock.go b/src/dbnode/storage/series/lookup/lookup_mock.go new file mode 100644 index 0000000000..66de4c3624 --- /dev/null +++ b/src/dbnode/storage/series/lookup/lookup_mock.go @@ -0,0 +1,66 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: github.com/m3db/m3/src/dbnode/storage/series/lookup (interfaces: OnReleaseReadWriteRef) + +// Copyright (c) 2019 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 lookup is a generated GoMock package. +package lookup + +import ( + "reflect" + + "github.com/golang/mock/gomock" +) + +// MockOnReleaseReadWriteRef is a mock of OnReleaseReadWriteRef interface +type MockOnReleaseReadWriteRef struct { + ctrl *gomock.Controller + recorder *MockOnReleaseReadWriteRefMockRecorder +} + +// MockOnReleaseReadWriteRefMockRecorder is the mock recorder for MockOnReleaseReadWriteRef +type MockOnReleaseReadWriteRefMockRecorder struct { + mock *MockOnReleaseReadWriteRef +} + +// NewMockOnReleaseReadWriteRef creates a new mock instance +func NewMockOnReleaseReadWriteRef(ctrl *gomock.Controller) *MockOnReleaseReadWriteRef { + mock := &MockOnReleaseReadWriteRef{ctrl: ctrl} + mock.recorder = &MockOnReleaseReadWriteRefMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockOnReleaseReadWriteRef) EXPECT() *MockOnReleaseReadWriteRefMockRecorder { + return m.recorder +} + +// OnReleaseReadWriteRef mocks base method +func (m *MockOnReleaseReadWriteRef) OnReleaseReadWriteRef() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "OnReleaseReadWriteRef") +} + +// OnReleaseReadWriteRef indicates an expected call of OnReleaseReadWriteRef +func (mr *MockOnReleaseReadWriteRefMockRecorder) OnReleaseReadWriteRef() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnReleaseReadWriteRef", reflect.TypeOf((*MockOnReleaseReadWriteRef)(nil).OnReleaseReadWriteRef)) +} diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index fd34a4e2c3..2c063ebd5e 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -38,16 +38,16 @@ import ( "go.uber.org/zap" ) -type bootstrapState int - -const ( - bootstrapNotStarted bootstrapState = iota - bootstrapped -) - var ( // ErrSeriesAllDatapointsExpired is returned on tick when all datapoints are expired ErrSeriesAllDatapointsExpired = errors.New("series datapoints are all expired") + // errSeriesMatchUniqueIndexFailed is returned when MatchUniqueIndex is + // specified for a write but the value does not match the current series + // unique index. + errSeriesMatchUniqueIndexFailed = errors.New("series write failed due to unique index not matched") + // errSeriesMatchUniqueIndexInvalid is returned when MatchUniqueIndex is + // specified for a write but the current series unique index is invalid. + errSeriesMatchUniqueIndexInvalid = errors.New("series write failed due to unique index being invalid") errSeriesAlreadyBootstrapped = errors.New("series is already bootstrapped") errSeriesNotBootstrapped = errors.New("series is not yet bootstrapped") @@ -62,12 +62,12 @@ type dbSeries struct { // series ID before changing ownership semantics (e.g. // pooling the ID rather than releasing it to the GC on // calling series.Reset()). - id ident.ID - tags ident.Tags + id ident.ID + tags ident.Tags + uniqueIndex uint64 buffer databaseBuffer cachedBlocks block.DatabaseSeriesBlocks - bs bootstrapState blockRetriever QueryableBlockRetriever onRetrieveBlock block.OnRetrieveBlock blockOnEvictedFromWiredList block.OnEvictedFromWiredList @@ -75,9 +75,9 @@ type dbSeries struct { } // NewDatabaseSeries creates a new database series -func NewDatabaseSeries(id ident.ID, tags ident.Tags, opts Options) DatabaseSeries { +func NewDatabaseSeries(id ident.ID, tags ident.Tags, uniqueIndex uint64, opts Options) DatabaseSeries { s := newDatabaseSeries() - s.Reset(id, tags, nil, nil, nil, opts) + s.Reset(id, tags, uniqueIndex, nil, nil, nil, opts) return s } @@ -93,7 +93,6 @@ func newPooledDatabaseSeries(pool DatabaseSeriesPool) DatabaseSeries { func newDatabaseSeries() *dbSeries { series := &dbSeries{ cachedBlocks: block.NewDatabaseSeriesBlocks(0), - bs: bootstrapNotStarted, } series.buffer = newDatabaseBuffer() return series @@ -118,6 +117,13 @@ func (s *dbSeries) Tags() ident.Tags { return tags } +func (s *dbSeries) UniqueIndex() uint64 { + s.RLock() + uniqueIndex := s.uniqueIndex + s.RUnlock() + return uniqueIndex +} + func (s *dbSeries) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Context) (TickResult, error) { var r TickResult @@ -274,13 +280,6 @@ func (s *dbSeries) NumActiveBlocks() int { return value } -func (s *dbSeries) IsBootstrapped() bool { - s.RLock() - state := s.bs - s.RUnlock() - return state == bootstrapped -} - func (s *dbSeries) Write( ctx context.Context, timestamp time.Time, @@ -290,6 +289,21 @@ func (s *dbSeries) Write( wOpts WriteOptions, ) (bool, error) { s.Lock() + matchUniqueIndex := wOpts.MatchUniqueIndex + if matchUniqueIndex { + if s.uniqueIndex == 0 { + return false, errSeriesMatchUniqueIndexInvalid + } + if s.uniqueIndex != wOpts.MatchUniqueIndexValue { + // NB(r): Match unique index allows for a caller to + // reliably take a reference to a series and call Write(...) + // later while keeping a direct reference to the series + // while the shard and namespace continues to own and manage + // the lifecycle of the series. + return false, errSeriesMatchUniqueIndexFailed + } + } + wasWritten, err := s.buffer.Write(ctx, timestamp, value, unit, annotation, wOpts) s.Unlock() return wasWritten, err @@ -373,89 +387,14 @@ func (s *dbSeries) addBlockWithLock(b block.DatabaseBlock) { s.cachedBlocks.AddBlock(b) } -func (s *dbSeries) Load( - opts LoadOptions, - blocksToLoad block.DatabaseSeriesBlocks, - blockStates BootstrappedBlockStateSnapshot, -) (LoadResult, error) { - if opts.Bootstrap { - bsResult, err := s.bootstrap(blocksToLoad, blockStates) - return LoadResult{Bootstrap: bsResult}, err - } - +func (s *dbSeries) LoadBlock( + block block.DatabaseBlock, + writeType WriteType, +) error { s.Lock() - s.loadWithLock(false, blocksToLoad, blockStates) + s.buffer.Load(block, writeType) s.Unlock() - return LoadResult{}, nil -} - -func (s *dbSeries) bootstrap( - bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates BootstrappedBlockStateSnapshot, -) (BootstrapResult, error) { - s.Lock() - defer func() { - s.bs = bootstrapped - s.Unlock() - }() - - var result BootstrapResult - if s.bs == bootstrapped { - return result, errSeriesAlreadyBootstrapped - } - - if bootstrappedBlocks == nil { - return result, nil - } - - s.loadWithLock(true, bootstrappedBlocks, blockStates) - result.NumBlocksMovedToBuffer += int64(bootstrappedBlocks.Len()) - - return result, nil -} - -func (s *dbSeries) loadWithLock( - isBootstrap bool, - blocksToLoad block.DatabaseSeriesBlocks, - blockStates BootstrappedBlockStateSnapshot, -) { - for _, block := range blocksToLoad.AllBlocks() { - if !isBootstrap { - // The data being loaded is not part of the bootstrap process then it needs to be - // loaded as a cold write because the load could be happening concurrently with - // other processes like the flush (as opposed to bootstrap which cannot happen - // concurrently with a flush) and there is no way to know if this series/block - // combination has been warm flushed or not yet since updating the shard block state - // doesn't happen until the entire flush completes. - // - // As a result the only safe operation is to load the block as a cold write which - // ensures that the data will eventually be flushed and merged with the existing data - // on disk in the two scenarios where the Load() API is used (cold writes and repairs). - s.buffer.Load(block, ColdWrite) - continue - } - - blStartNano := xtime.ToUnixNano(block.StartTime()) - blState := blockStates.Snapshot[blStartNano] - if !blState.WarmRetrievable { - // If the block being bootstrapped has never been warm flushed before then the block - // can be loaded into the buffer as a WarmWrite because a subsequent warm flush will - // ensure that it gets persisted to disk. - // - // If the ColdWrites feature is disabled then this branch should always be followed. - s.buffer.Load(block, WarmWrite) - } else { - // If the block being bootstrapped has been warm flushed before then the block should - // be loaded into the buffer as a ColdWrite so that a subsequent cold flush will ensure - // that it gets persisted to disk. - // - // This branch can be executed in the situation that a cold write was received for a block - // that had already been flushed to disk. Before the cold write could be persisted to disk - // via a cold flush, the node crashed and began bootsrapping itself. The cold write would be - // read out of the commitlog and would eventually be loaded into the buffer via this branch. - s.buffer.Load(block, ColdWrite) - } - } + return nil } func (s *dbSeries) OnRetrieveBlock( @@ -562,14 +501,12 @@ func (s *dbSeries) WarmFlush( persistFn persist.DataFn, nsCtx namespace.Context, ) (FlushOutcome, error) { + // Need a write lock because the buffer WarmFlush method mutates + // state (by performing a pro-active merge). s.Lock() - defer s.Unlock() - - if s.bs != bootstrapped { - return FlushOutcomeErr, errSeriesNotBootstrapped - } - - return s.buffer.WarmFlush(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) + outcome, err := s.buffer.WarmFlush(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) + s.Unlock() + return outcome, err } func (s *dbSeries) Snapshot( @@ -583,10 +520,6 @@ func (s *dbSeries) Snapshot( s.Lock() defer s.Unlock() - if s.bs != bootstrapped { - return errSeriesNotBootstrapped - } - return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } @@ -601,9 +534,10 @@ func (s *dbSeries) Close() { s.Lock() defer s.Unlock() - // See Reset() for why these aren't finalized + // See Reset() for why these aren't finalized. s.id = nil s.tags = ident.Tags{} + s.uniqueIndex = 0 switch s.opts.CachePolicy() { case CacheLRU: @@ -629,15 +563,13 @@ func (s *dbSeries) Close() { func (s *dbSeries) Reset( id ident.ID, tags ident.Tags, + uniqueIndex uint64, blockRetriever QueryableBlockRetriever, onRetrieveBlock block.OnRetrieveBlock, onEvictedFromWiredList block.OnEvictedFromWiredList, opts Options, ) { - s.Lock() - defer s.Unlock() - - // NB(r): We explicitly do not place this ID back into an + // NB(r): We explicitly do not place the ID back into an // existing pool as high frequency users of series IDs such // as the commit log need to use the reference without the // overhead of ownership tracking. In addition, the blocks @@ -652,14 +584,17 @@ func (s *dbSeries) Reset( // Since series are purged so infrequently the overhead // of not releasing back an ID to a pool is amortized over // a long period of time. + // + // The same goes for the series tags. + s.Lock() s.id = id s.tags = tags - + s.uniqueIndex = uniqueIndex s.cachedBlocks.Reset() s.buffer.Reset(id, opts) s.opts = opts - s.bs = bootstrapNotStarted s.blockRetriever = blockRetriever s.onRetrieveBlock = onRetrieveBlock s.blockOnEvictedFromWiredList = onEvictedFromWiredList + s.Unlock() } diff --git a/src/dbnode/storage/series/series_all_test.go b/src/dbnode/storage/series/series_all_test.go index 3abbf9b280..caffcc76ba 100644 --- a/src/dbnode/storage/series/series_all_test.go +++ b/src/dbnode/storage/series/series_all_test.go @@ -25,7 +25,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/x/xio" - xtime "github.com/m3db/m3/src/x/time" "github.com/m3db/m3/src/dbnode/namespace" "github.com/stretchr/testify/require" @@ -44,45 +43,25 @@ func mins(x float64) time.Duration { type blockData struct { start time.Time writeType WriteType - data [][]value + data [][]DecodedTestValue } -type value struct { - timestamp time.Time - value float64 - unit xtime.Unit - annotation []byte -} - -type valuesByTime []value - -func (v valuesByTime) Len() int { - return len(v) -} - -func (v valuesByTime) Less(lhs, rhs int) bool { - return v[lhs].timestamp.Before(v[rhs].timestamp) -} - -func (v valuesByTime) Swap(lhs, rhs int) { - v[lhs], v[rhs] = v[rhs], v[lhs] -} - -type setAnnotation func([]value) []value +type setAnnotation func([]DecodedTestValue) []DecodedTestValue type requireAnnEqual func(*testing.T, []byte, []byte) -func decodedReaderValues(results [][]xio.BlockReader, opts Options, nsCtx namespace.Context) ([]value, error) { +func decodedReaderValues(results [][]xio.BlockReader, + opts Options, nsCtx namespace.Context) ([]DecodedTestValue, error) { slicesIter := xio.NewReaderSliceOfSlicesFromBlockReadersIterator(results) iter := opts.MultiReaderIteratorPool().Get() iter.ResetSliceOfSlices(slicesIter, nsCtx.Schema) defer iter.Close() - var all []value + var all []DecodedTestValue for iter.Next() { dp, unit, annotation := iter.Current() // Iterator reuse annotation byte slices, so make a copy. annotationCopy := append([]byte(nil), annotation...) - all = append(all, value{dp.Timestamp, dp.Value, unit, annotationCopy}) + all = append(all, DecodedTestValue{dp.Timestamp, dp.Value, unit, annotationCopy}) } if err := iter.Err(); err != nil { return nil, err @@ -91,47 +70,37 @@ func decodedReaderValues(results [][]xio.BlockReader, opts Options, nsCtx namesp return all, nil } -func requireReaderValuesEqual(t *testing.T, values []value, results [][]xio.BlockReader, opts Options, +func requireReaderValuesEqual(t *testing.T, values []DecodedTestValue, + results [][]xio.BlockReader, opts Options, nsCtx namespace.Context) { decodedValues, err := decodedReaderValues(results, opts, nsCtx) require.NoError(t, err) requireValuesEqual(t, values, decodedValues, nsCtx) } -func requireValuesEqual(t *testing.T, expected, actual []value, nsCtx namespace.Context) { +func requireValuesEqual(t *testing.T, expected, actual []DecodedTestValue, + nsCtx namespace.Context) { require.Len(t, actual, len(expected)) for i := 0; i < len(actual); i++ { - require.True(t, expected[i].timestamp.Equal(actual[i].timestamp)) - require.Equal(t, expected[i].value, actual[i].value) - require.Equal(t, expected[i].unit, actual[i].unit) + require.True(t, expected[i].Timestamp.Equal(actual[i].Timestamp)) + require.Equal(t, expected[i].Value, actual[i].Value) + require.Equal(t, expected[i].Unit, actual[i].Unit) if nsCtx.Schema == nil { - require.Equal(t, expected[i].annotation, actual[i].annotation) + require.Equal(t, expected[i].Annotation, actual[i].Annotation) } else { - testProtoEqual(t, expected[i].annotation, actual[i].annotation) + testProtoEqual(t, expected[i].Annotation, actual[i].Annotation) } } } -func decodedSegmentValues(results []xio.SegmentReader, opts Options, nsCtx namespace.Context) ([]value, error) { +func decodedSegmentValues(results []xio.SegmentReader, opts Options, + nsCtx namespace.Context) ([]DecodedTestValue, error) { iter := opts.MultiReaderIteratorPool().Get() - iter.Reset(results, time.Time{}, time.Duration(0), nsCtx.Schema) - defer iter.Close() - - var all []value - for iter.Next() { - dp, unit, annotation := iter.Current() - // Iterator reuse annotation byte slices, so make a copy. - annotationCopy := append([]byte(nil), annotation...) - all = append(all, value{dp.Timestamp, dp.Value, unit, annotationCopy}) - } - if err := iter.Err(); err != nil { - return nil, err - } - - return all, nil + return DecodeSegmentValues(results, iter, nsCtx.Schema) } -func requireSegmentValuesEqual(t *testing.T, values []value, results []xio.SegmentReader, opts Options, +func requireSegmentValuesEqual(t *testing.T, values []DecodedTestValue, + results []xio.SegmentReader, opts Options, nsCtx namespace.Context) { decodedValues, err := decodedSegmentValues(results, opts, nsCtx) diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index b845c5f905..c7ed600f7e 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -148,20 +148,6 @@ func (mr *MockDatabaseSeriesMockRecorder) ID() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ID", reflect.TypeOf((*MockDatabaseSeries)(nil).ID)) } -// IsBootstrapped mocks base method -func (m *MockDatabaseSeries) IsBootstrapped() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "IsBootstrapped") - ret0, _ := ret[0].(bool) - return ret0 -} - -// IsBootstrapped indicates an expected call of IsBootstrapped -func (mr *MockDatabaseSeriesMockRecorder) IsBootstrapped() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsBootstrapped", reflect.TypeOf((*MockDatabaseSeries)(nil).IsBootstrapped)) -} - // IsEmpty mocks base method func (m *MockDatabaseSeries) IsEmpty() bool { m.ctrl.T.Helper() @@ -176,19 +162,18 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockDatabaseSeries)(nil).IsEmpty)) } -// Load mocks base method -func (m *MockDatabaseSeries) Load(arg0 LoadOptions, arg1 block.DatabaseSeriesBlocks, arg2 BootstrappedBlockStateSnapshot) (LoadResult, error) { +// LoadBlock mocks base method +func (m *MockDatabaseSeries) LoadBlock(arg0 block.DatabaseBlock, arg1 WriteType) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Load", arg0, arg1, arg2) - ret0, _ := ret[0].(LoadResult) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret := m.ctrl.Call(m, "LoadBlock", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 } -// Load indicates an expected call of Load -func (mr *MockDatabaseSeriesMockRecorder) Load(arg0, arg1, arg2 interface{}) *gomock.Call { +// LoadBlock indicates an expected call of LoadBlock +func (mr *MockDatabaseSeriesMockRecorder) LoadBlock(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockDatabaseSeries)(nil).Load), arg0, arg1, arg2) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LoadBlock", reflect.TypeOf((*MockDatabaseSeries)(nil).LoadBlock), arg0, arg1) } // NumActiveBlocks mocks base method @@ -245,15 +230,15 @@ func (mr *MockDatabaseSeriesMockRecorder) ReadEncoded(arg0, arg1, arg2, arg3 int } // Reset mocks base method -func (m *MockDatabaseSeries) Reset(arg0 ident.ID, arg1 ident.Tags, arg2 QueryableBlockRetriever, arg3 block.OnRetrieveBlock, arg4 block.OnEvictedFromWiredList, arg5 Options) { +func (m *MockDatabaseSeries) Reset(arg0 ident.ID, arg1 ident.Tags, arg2 uint64, arg3 QueryableBlockRetriever, arg4 block.OnRetrieveBlock, arg5 block.OnEvictedFromWiredList, arg6 Options) { m.ctrl.T.Helper() - m.ctrl.Call(m, "Reset", arg0, arg1, arg2, arg3, arg4, arg5) + m.ctrl.Call(m, "Reset", arg0, arg1, arg2, arg3, arg4, arg5, arg6) } // Reset indicates an expected call of Reset -func (mr *MockDatabaseSeriesMockRecorder) Reset(arg0, arg1, arg2, arg3, arg4, arg5 interface{}) *gomock.Call { +func (mr *MockDatabaseSeriesMockRecorder) Reset(arg0, arg1, arg2, arg3, arg4, arg5, arg6 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockDatabaseSeries)(nil).Reset), arg0, arg1, arg2, arg3, arg4, arg5) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockDatabaseSeries)(nil).Reset), arg0, arg1, arg2, arg3, arg4, arg5, arg6) } // Snapshot mocks base method @@ -299,6 +284,20 @@ func (mr *MockDatabaseSeriesMockRecorder) Tick(arg0, arg1 interface{}) *gomock.C return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Tick", reflect.TypeOf((*MockDatabaseSeries)(nil).Tick), arg0, arg1) } +// UniqueIndex mocks base method +func (m *MockDatabaseSeries) UniqueIndex() uint64 { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UniqueIndex") + ret0, _ := ret[0].(uint64) + return ret0 +} + +// UniqueIndex indicates an expected call of UniqueIndex +func (mr *MockDatabaseSeriesMockRecorder) UniqueIndex() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UniqueIndex", reflect.TypeOf((*MockDatabaseSeries)(nil).UniqueIndex)) +} + // WarmFlush mocks base method func (m *MockDatabaseSeries) WarmFlush(arg0 context.Context, arg1 time.Time, arg2 persist.DataFn, arg3 namespace.Context) (FlushOutcome, error) { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/series_parallel_test.go b/src/dbnode/storage/series/series_parallel_test.go index 161ba34fcb..116c2927af 100644 --- a/src/dbnode/storage/series/series_parallel_test.go +++ b/src/dbnode/storage/series/series_parallel_test.go @@ -27,6 +27,8 @@ import ( "testing" "time" + "github.com/m3db/m3/src/dbnode/storage/block" + "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -43,10 +45,12 @@ func TestSeriesWriteReadParallel(t *testing.T) { numStepsPerWorker = numWorkers * 100 opts = newSeriesTestOptions() curr = time.Now() - series = NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series = NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + dbBlock = block.NewDatabaseBlock(time.Time{}, time.Hour*2, + ts.Segment{}, block.NewOptions(), namespace.Context{}) ) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + err := series.LoadBlock(dbBlock, WarmWrite) assert.NoError(t, err) ctx := context.NewContext() diff --git a/src/dbnode/storage/series/series_test.go b/src/dbnode/storage/series/series_test.go index 0f9a610156..ed67cae30b 100644 --- a/src/dbnode/storage/series/series_test.go +++ b/src/dbnode/storage/series/series_test.go @@ -82,16 +82,15 @@ func newSeriesTestOptions() Options { func TestSeriesEmpty(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) - assert.NoError(t, err) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) assert.True(t, series.IsEmpty()) } // Writes to series, verifying no error and that further writes should happen. -func verifyWriteToSeries(t *testing.T, series *dbSeries, v value) { +func verifyWriteToSeries(t *testing.T, series *dbSeries, v DecodedTestValue) { ctx := context.NewContext() - wasWritten, err := series.Write(ctx, v.timestamp, v.value, v.unit, v.annotation, WriteOptions{}) + wasWritten, err := series.Write(ctx, v.Timestamp, v.Value, + v.Unit, v.Annotation, WriteOptions{}) require.NoError(t, err) require.True(t, wasWritten) ctx.Close() @@ -104,11 +103,20 @@ func TestSeriesWriteFlush(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(curr) + bl.EXPECT().Stream(gomock.Any()).Return(xio.BlockReader{}, nil) + bl.EXPECT().Close() + + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(1)), 2, xtime.Second, nil}, {curr.Add(mins(2)), 3, xtime.Second, nil}, @@ -116,7 +124,7 @@ func TestSeriesWriteFlush(t *testing.T) { } for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToSeries(t, series, v) } @@ -139,11 +147,20 @@ func TestSeriesSamePointDoesNotWrite(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(curr) + bl.EXPECT().Stream(gomock.Any()).Return(xio.BlockReader{}, nil) + bl.EXPECT().Close() + + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) - data := []value{ + data := []DecodedTestValue{ {curr, 1, xtime.Second, nil}, {curr, 1, xtime.Second, nil}, {curr, 1, xtime.Second, nil}, @@ -152,9 +169,9 @@ func TestSeriesSamePointDoesNotWrite(t *testing.T) { } for i, v := range data { - curr = v.timestamp + curr = v.Timestamp ctx := context.NewContext() - wasWritten, err := series.Write(ctx, v.timestamp, v.value, v.unit, v.annotation, WriteOptions{}) + wasWritten, err := series.Write(ctx, v.Timestamp, v.Value, v.Unit, v.Annotation, WriteOptions{}) require.NoError(t, err) if i == 0 || i == len(data)-1 { require.True(t, wasWritten) @@ -182,11 +199,19 @@ func TestSeriesWriteFlushRead(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(curr) + bl.EXPECT().Len().Return(0).Times(2) + + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) - data := []value{ + data := []DecodedTestValue{ {curr.Add(mins(1)), 2, xtime.Second, nil}, {curr.Add(mins(3)), 3, xtime.Second, nil}, {curr.Add(mins(5)), 4, xtime.Second, nil}, @@ -195,7 +220,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { } for _, v := range data { - curr = v.timestamp + curr = v.Timestamp verifyWriteToSeries(t, series, v) } @@ -224,20 +249,16 @@ func TestSeriesWriteFlushRead(t *testing.T) { // cold writes and that for the load path everything is loaded as cold writes. func TestSeriesBootstrapAndLoad(t *testing.T) { testCases := []struct { - title string - loadOpts LoadOptions - f func( - series DatabaseSeries, - blocks block.DatabaseSeriesBlocks, - blockStates BootstrappedBlockStateSnapshot) + title string + bootstrapping bool }{ { - title: "load", - loadOpts: LoadOptions{}, + title: "load", + bootstrapping: false, }, { - title: "bootstrap", - loadOpts: LoadOptions{Bootstrap: true}, + title: "bootstrap", + bootstrapping: true, }, } @@ -252,21 +273,21 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) - rawWrites := []value{ + rawWrites := []DecodedTestValue{ {curr.Add(mins(1)), 2, xtime.Second, nil}, {curr.Add(mins(3)), 3, xtime.Second, nil}, {curr.Add(mins(5)), 4, xtime.Second, nil}, } for _, v := range rawWrites { - curr = v.timestamp + curr = v.Timestamp verifyWriteToSeries(t, series, v) } var ( - loadWrites = []value{ + loadWrites = []DecodedTestValue{ // Ensure each value is in a separate block so since block.DatabaseSeriesBlocks // can only store a single block per block start). {curr.Add(blockSize), 5, xtime.Second, nil}, @@ -274,7 +295,6 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { } nsCtx = namespace.Context{} blockOpts = opts.DatabaseBlockOptions() - blocks = block.NewDatabaseSeriesBlocks(len(loadWrites)) alreadyWarmFlushedBlockStart = curr.Add(blockSize).Truncate(blockSize) notYetWarmFlushedBlockStart = curr.Add(2 * blockSize).Truncate(blockSize) blockStates = BootstrappedBlockStateSnapshot{ @@ -290,19 +310,24 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { } ) for _, v := range loadWrites { - curr = v.timestamp + curr = v.Timestamp enc := opts.EncoderPool().Get() - blockStart := v.timestamp.Truncate(blockSize) + blockStart := v.Timestamp.Truncate(blockSize) enc.Reset(blockStart, 0, nil) - dp := ts.Datapoint{Timestamp: v.timestamp, Value: v.value} - require.NoError(t, enc.Encode(dp, v.unit, nil)) + dp := ts.Datapoint{Timestamp: v.Timestamp, Value: v.Value} + require.NoError(t, enc.Encode(dp, v.Unit, nil)) dbBlock := block.NewDatabaseBlock(blockStart, blockSize, enc.Discard(), blockOpts, nsCtx) - blocks.AddBlock(dbBlock) - } - _, err := series.Load(tc.loadOpts, blocks, blockStates) - require.NoError(t, err) + writeType := ColdWrite + if tc.bootstrapping { + if blockStart.Equal(notYetWarmFlushedBlockStart) { + writeType = WarmWrite + } + } + + series.LoadBlock(dbBlock, writeType) + } t.Run("Data can be read", func(t *testing.T) { ctx := context.NewContext() @@ -327,11 +352,11 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { return coldFlushBlockStarts[i] < coldFlushBlockStarts[j] }) - if tc.loadOpts.Bootstrap { + if tc.bootstrapping { // If its a bootstrap then we need to make sure that everything gets loaded as warm/cold writes // correctly based on the flush state. expectedColdFlushBlockStarts := []xtime.UnixNano{xtime.ToUnixNano(alreadyWarmFlushedBlockStart)} - require.Equal(t, expectedColdFlushBlockStarts, coldFlushBlockStarts) + assert.Equal(t, expectedColdFlushBlockStarts, coldFlushBlockStarts) } else { // If its just a regular load then everything should be loaded as cold writes for correctness // since flushes and loads can happen concurrently. @@ -339,7 +364,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { xtime.ToUnixNano(alreadyWarmFlushedBlockStart), xtime.ToUnixNano(notYetWarmFlushedBlockStart), } - require.Equal(t, expectedColdFlushBlockStarts, coldFlushBlockStarts) + assert.Equal(t, expectedColdFlushBlockStarts, coldFlushBlockStarts) } }) }) @@ -348,8 +373,15 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { func TestSeriesReadEndBeforeStart(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) ctx := context.NewContext() @@ -363,9 +395,15 @@ func TestSeriesReadEndBeforeStart(t *testing.T) { } func TestSeriesFlushNoBlock(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) flushTime := time.Unix(7200, 0) outcome, err := series.WarmFlush(nil, flushTime, nil, namespace.Context{}) @@ -377,14 +415,17 @@ func TestSeriesFlush(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + curr := time.Unix(7200, 0) opts := newSeriesTestOptions() opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) ctx := context.NewContext() @@ -410,10 +451,8 @@ func TestSeriesFlush(t *testing.T) { func TestSeriesTickEmptySeries(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) - assert.NoError(t, err) - _, err = series.Tick(NewShardBlockStateSnapshot(true, BootstrappedBlockStateSnapshot{}), namespace.Context{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + _, err := series.Tick(NewShardBlockStateSnapshot(true, BootstrappedBlockStateSnapshot{}), namespace.Context{}) require.Equal(t, ErrSeriesAllDatapointsExpired, err) } @@ -421,9 +460,12 @@ func TestSeriesTickDrainAndResetBuffer(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) buffer := NewMockdatabaseBuffer(ctrl) series.buffer = buffer @@ -440,14 +482,17 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + opts := newSeriesTestOptions() ropts := opts.RetentionOptions() curr := time.Now().Truncate(ropts.BlockSize()) opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) blockStart := curr.Add(-ropts.RetentionPeriod()).Add(-ropts.BlockSize()) b := block.NewMockDatabaseBlock(ctrl) @@ -491,6 +536,9 @@ func TestSeriesTickRecentlyRead(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + opts := newSeriesTestOptions() opts = opts. SetCachePolicy(CacheRecentlyRead). @@ -500,10 +548,10 @@ func TestSeriesTickRecentlyRead(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) // Test case where block has been read within expiry period - won't be removed @@ -566,6 +614,9 @@ func TestSeriesTickCacheLRU(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + retentionPeriod := time.Hour opts := newSeriesTestOptions() opts = opts. @@ -576,10 +627,10 @@ func TestSeriesTickCacheLRU(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) // Test case where block was not retrieved from disk - Will be removed @@ -650,6 +701,9 @@ func TestSeriesTickCacheNone(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + opts := newSeriesTestOptions() opts = opts. SetCachePolicy(CacheNone). @@ -659,10 +713,10 @@ func TestSeriesTickCacheNone(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) // Retrievable blocks should be removed @@ -714,7 +768,7 @@ func TestSeriesTickCachedBlockRemove(t *testing.T) { opts = opts.SetCachePolicy(CacheAll) ropts := opts.RetentionOptions() curr := time.Now().Truncate(ropts.BlockSize()) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) // Add current block b := block.NewMockDatabaseBlock(ctrl) @@ -779,9 +833,12 @@ func TestSeriesFetchBlocks(t *testing.T) { SegmentReader: xio.NewSegmentReader(ts.Segment{}), }, nil) blocks.EXPECT().BlockAt(starts[0]).Return(b, true) + b = block.NewMockDatabaseBlock(ctrl) + b.EXPECT().StartTime().Return(starts[1]) b.EXPECT().Stream(ctx).Return(xio.EmptyBlockReader, errors.New("bar")) blocks.EXPECT().BlockAt(starts[1]).Return(b, true) + blocks.EXPECT().BlockAt(starts[2]).Return(nil, false) // Set up the buffer @@ -791,8 +848,8 @@ func TestSeriesFetchBlocks(t *testing.T) { FetchBlocks(ctx, starts, namespace.Context{}). Return([]block.FetchBlockResult{block.NewFetchBlockResult(starts[2], nil, nil)}) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(b, WarmWrite) assert.NoError(t, err) series.cachedBlocks = blocks @@ -848,8 +905,11 @@ func TestSeriesFetchBlocksMetadata(t *testing.T) { FetchBlocksMetadata(ctx, start, end, fetchOpts). Return(expectedResults, nil) - series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(start) + + series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) series.buffer = buffer @@ -908,8 +968,8 @@ func TestSeriesOutOfOrderWritesAndRotate(t *testing.T) { expected []ts.Datapoint ) - series := NewDatabaseSeries(id, tags, opts).(*dbSeries) - series.Reset(id, tags, nil, nil, nil, opts) + series := NewDatabaseSeries(id, tags, 1, opts).(*dbSeries) + series.Reset(id, tags, 1, nil, nil, nil, opts) for iter := 0; iter < numBlocks; iter++ { start := now @@ -965,15 +1025,24 @@ func TestSeriesOutOfOrderWritesAndRotate(t *testing.T) { } func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + bl := block.NewMockDatabaseBlock(ctrl) + bl.EXPECT().StartTime().Return(time.Now()) + bl.EXPECT().Len().Return(0).AnyTimes() + opts := newSeriesTestOptions() opts = opts.SetRetentionOptions(opts.RetentionOptions(). SetRetentionPeriod(40 * 24 * time.Hour). - // A block size of 5 days is not equally as divisible as seconds from time zero and seconds from time epoch. + // A block size of 5 days is not equally as divisible as seconds from time + // zero and seconds from time epoch. // now := time.Now() // blockSize := 5 * 24 * time.Hour // fmt.Println(now) -> 2018-01-24 14:29:31.624265 -0500 EST m=+0.003810489 // fmt.Println(now.Truncate(blockSize)) -> 2018-01-21 19:00:00 -0500 EST - // fmt.Println(time.Unix(0, now.UnixNano()/int64(blockSize)*int64(blockSize))) -> 2018-01-23 19:00:00 -0500 EST + // fmt.Println(time.Unix(0, now.UnixNano()/int64(blockSize)*int64(blockSize))) + // -> 2018-01-23 19:00:00 -0500 EST SetBlockSize(5 * 24 * time.Hour). SetBufferFuture(10 * time.Minute). SetBufferPast(20 * time.Minute)) @@ -981,24 +1050,28 @@ func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) + err := series.LoadBlock(bl, WarmWrite) assert.NoError(t, err) ctx := context.NewContext() defer ctx.Close() - wasWritten, err := series.Write(ctx, curr.Add(-3*time.Minute), 1, xtime.Second, nil, WriteOptions{}) + wasWritten, err := series.Write(ctx, curr.Add(-3*time.Minute), + 1, xtime.Second, nil, WriteOptions{}) assert.NoError(t, err) assert.True(t, wasWritten) - wasWritten, err = series.Write(ctx, curr.Add(-2*time.Minute), 2, xtime.Second, nil, WriteOptions{}) + wasWritten, err = series.Write(ctx, curr.Add(-2*time.Minute), + 2, xtime.Second, nil, WriteOptions{}) assert.NoError(t, err) assert.True(t, wasWritten) - wasWritten, err = series.Write(ctx, curr.Add(-1*time.Minute), 3, xtime.Second, nil, WriteOptions{}) + wasWritten, err = series.Write(ctx, curr.Add(-1*time.Minute), + 3, xtime.Second, nil, WriteOptions{}) assert.NoError(t, err) assert.True(t, wasWritten) - results, err := series.ReadEncoded(ctx, curr.Add(-5*time.Minute), curr.Add(time.Minute), namespace.Context{}) + results, err := series.ReadEncoded(ctx, curr.Add(-5*time.Minute), + curr.Add(time.Minute), namespace.Context{}) require.NoError(t, err) values, err := decodedReaderValues(results, opts, namespace.Context{}) require.NoError(t, err) @@ -1012,7 +1085,7 @@ func TestSeriesCloseNonCacheLRUPolicy(t *testing.T) { opts := newSeriesTestOptions(). SetCachePolicy(CacheRecentlyRead) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) start := time.Now() blocks := block.NewDatabaseSeriesBlocks(0) @@ -1031,7 +1104,7 @@ func TestSeriesCloseCacheLRUPolicy(t *testing.T) { opts := newSeriesTestOptions(). SetCachePolicy(CacheLRU) - series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, 1, opts).(*dbSeries) start := time.Now() blocks := block.NewDatabaseSeriesBlocks(0) @@ -1042,7 +1115,8 @@ func TestSeriesCloseCacheLRUPolicy(t *testing.T) { // Add block that was not retrieved from disk nonDiskBlock := block.NewMockDatabaseBlock(ctrl) - nonDiskBlock.EXPECT().StartTime().Return(start.Add(opts.RetentionOptions().BlockSize())).AnyTimes() + nonDiskBlock.EXPECT().StartTime(). + Return(start.Add(opts.RetentionOptions().BlockSize())).AnyTimes() blocks.AddBlock(nonDiskBlock) series.cachedBlocks = blocks diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 90148eb2d3..520e071301 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -49,6 +49,10 @@ type DatabaseSeries interface { // Tags return the tags of the series. Tags() ident.Tags + // UniqueIndex is the unique index for the series (for this current + // process, unless the time series expires). + UniqueIndex() uint64 + // Tick executes async updates Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Context) (TickResult, error) @@ -99,15 +103,11 @@ type DatabaseSeries interface { // NumActiveBlocks returns the number of active blocks the series currently holds. NumActiveBlocks() int - // IsBootstrapped returns whether the series is bootstrapped or not. - IsBootstrapped() bool - - // Load loads data into the series. - Load( - opts LoadOptions, - blocks block.DatabaseSeriesBlocks, - blockStates BootstrappedBlockStateSnapshot, - ) (LoadResult, error) + /// LoadBlock loads a single block into the series. + LoadBlock( + block block.DatabaseBlock, + writeType WriteType, + ) error // WarmFlush flushes the WarmWrites of this series for a given start time. WarmFlush( @@ -136,6 +136,7 @@ type DatabaseSeries interface { Reset( id ident.ID, tags ident.Tags, + uniqueIndex uint64, blockRetriever QueryableBlockRetriever, onRetrieveBlock block.OnRetrieveBlock, onEvictedFromWiredList block.OnEvictedFromWiredList, @@ -403,26 +404,17 @@ type WriteOptions struct { TruncateType TruncateType // TransformOptions describes transformation options for incoming writes. TransformOptions WriteTransformOptions -} - -// LoadOptions contains the options for the Load() method. -type LoadOptions struct { - // Whether the call to Bootstrap should be considered a "true" bootstrap - // or if additional data is being loaded after the fact (as in the case - // of repairs). - Bootstrap bool -} - -// LoadResult contains the return information for the Load() method. -type LoadResult struct { - Bootstrap BootstrapResult -} - -// BootstrapResult contains information about the result of bootstrapping a series. -// It is returned from the series Bootstrap method primarily so the caller can aggregate -// and emit metrics instead of the series itself having to store additional fields (which -// would be costly because we have millions of them.) -type BootstrapResult struct { - NumBlocksMovedToBuffer int64 - NumBlocksMerged int64 + // MatchUniqueIndex specifies whether the series unique index + // must match the unique index value specified (to ensure the series + // being written is the same series as previously referenced). + MatchUniqueIndex bool + // MatchUniqueIndexValue is the series unique index value that + // must match the current series unique index value (to ensure series + // being written is the same series as previously referenced). + MatchUniqueIndexValue uint64 + // BootstrapWrite allows a warm write outside the time window as long as the + // block hasn't already been flushed to disk. This is useful for + // bootstrappers filling data that they know has not yet been flushed to + // disk. + BootstrapWrite bool } diff --git a/src/dbnode/storage/series/util.go b/src/dbnode/storage/series/util.go new file mode 100644 index 0000000000..785f3de779 --- /dev/null +++ b/src/dbnode/storage/series/util.go @@ -0,0 +1,94 @@ +// Copyright (c) 2019 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 series + +import ( + "time" + + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/x/xio" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/m3db/m3/src/dbnode/namespace" +) + +// ValuesByTime is a sortable slice of DecodedTestValue. +type ValuesByTime []DecodedTestValue + +// Len is the number of elements in the collection. +func (v ValuesByTime) Len() int { + return len(v) +} + +// Less reports whether the element with +// index i should sort before the element with index j. +func (v ValuesByTime) Less(lhs, rhs int) bool { + l := v[lhs].Timestamp + r := v[rhs].Timestamp + if l.Equal(r) { + return v[lhs].Value-v[rhs].Value < 0 + } + + return l.Before(r) +} + +// Swap swaps the elements with indexes i and j. +func (v ValuesByTime) Swap(lhs, rhs int) { + v[lhs], v[rhs] = v[rhs], v[lhs] +} + +// DecodedTestValue is a decoded datapoint. +type DecodedTestValue struct { + // Timestamp is the data point timestamp. + Timestamp time.Time + // Value is the data point value. + Value float64 + // Unit is the data point unit. + Unit xtime.Unit + // Annotation is the data point annotation. + Annotation []byte +} + +// DecodeSegmentValues is a test utility to read through a slice of +// SegmentReaders. +func DecodeSegmentValues( + results []xio.SegmentReader, + iter encoding.MultiReaderIterator, + schema namespace.SchemaDescr, +) ([]DecodedTestValue, error) { + iter.Reset(results, time.Time{}, time.Duration(0), schema) + defer iter.Close() + + var all []DecodedTestValue + for iter.Next() { + dp, unit, annotation := iter.Current() + // Iterator reuse annotation byte slices, so make a copy. + annotationCopy := append([]byte(nil), annotation...) + all = append(all, DecodedTestValue{ + dp.Timestamp, dp.Value, unit, annotationCopy}) + } + + if err := iter.Err(); err != nil { + return nil, err + } + + return all, nil +} diff --git a/src/dbnode/storage/series_wired_list_interaction_test.go b/src/dbnode/storage/series_wired_list_interaction_test.go index 86c2fcc6b1..1a21e9c8b5 100644 --- a/src/dbnode/storage/series_wired_list_interaction_test.go +++ b/src/dbnode/storage/series_wired_list_interaction_test.go @@ -80,29 +80,30 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { }) var ( - opts = DefaultTestOptions().SetDatabaseBlockOptions( + blockSize = time.Hour * 2 + start = time.Now().Truncate(blockSize) + opts = DefaultTestOptions().SetDatabaseBlockOptions( blOpts. SetWiredList(wl). SetDatabaseBlockPool(blPool), ) shard = testDatabaseShard(t, opts) id = ident.StringID("foo") - seriesEntry = series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) + seriesEntry = series.NewDatabaseSeries(id, ident.Tags{}, 1, shard.seriesOpts) + block = block.NewDatabaseBlock(start, blockSize, ts.Segment{}, blOpts, namespace.Context{}) ) - seriesEntry.Reset(id, ident.Tags{}, nil, shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) - seriesEntry.Load( - series.LoadOptions{Bootstrap: true}, - nil, - series.BootstrappedBlockStateSnapshot{}) + seriesEntry.Reset(id, ident.Tags{}, 1, nil, + shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) + + seriesEntry.LoadBlock(block, series.WarmWrite) shard.Lock() shard.insertNewShardEntryWithLock(lookup.NewEntry(seriesEntry, 0)) shard.Unlock() var ( - wg = sync.WaitGroup{} - doneCh = make(chan struct{}) - blockSize = 2 * time.Hour + wg = sync.WaitGroup{} + doneCh = make(chan struct{}) ) go func() { // Try and trigger any pooling issues @@ -118,7 +119,6 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { }() var ( - start = time.Now().Truncate(blockSize) startLock = sync.Mutex{} getAndIncStart = func() time.Time { startLock.Lock() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index b0414d2e89..6f07c2911a 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -78,6 +78,8 @@ var ( errFlushStateAlreadyBootstrapped = errors.New("flush state is already bootstrapped") errTriedToLoadNilSeries = errors.New("tried to load nil series into shard") + // ErrDatabaseLoadLimitHit is the error returned when the database load limit + // is hit or exceeded. ErrDatabaseLoadLimitHit = errors.New("error loading series, database load limit hit") ) @@ -177,7 +179,6 @@ type dbShard struct { currRuntimeOptions dbShardRuntimeOptions logger *zap.Logger metrics dbShardMetrics - newSeriesBootstrapped bool ticking bool shard uint32 } @@ -194,20 +195,16 @@ type dbShardRuntimeOptions struct { } type dbShardMetrics struct { - create tally.Counter - close tally.Counter - closeStart tally.Counter - closeLatency tally.Timer - insertAsyncInsertErrors tally.Counter - insertAsyncBootstrapErrors tally.Counter - insertAsyncWriteErrors tally.Counter - seriesBootstrapBlocksToBuffer tally.Counter - seriesBootstrapBlocksMerged tally.Counter - seriesTicked tally.Gauge + create tally.Counter + close tally.Counter + closeStart tally.Counter + closeLatency tally.Timer + insertAsyncInsertErrors tally.Counter + insertAsyncWriteErrors tally.Counter + seriesTicked tally.Gauge } func newDatabaseShardMetrics(shardID uint32, scope tally.Scope) dbShardMetrics { - seriesBootstrapScope := scope.SubScope("series-bootstrap") return dbShardMetrics{ create: scope.Counter("create"), close: scope.Counter("close"), @@ -216,14 +213,9 @@ func newDatabaseShardMetrics(shardID uint32, scope tally.Scope) dbShardMetrics { insertAsyncInsertErrors: scope.Tagged(map[string]string{ "error_type": "insert-series", }).Counter("insert-async.errors"), - insertAsyncBootstrapErrors: scope.Tagged(map[string]string{ - "error_type": "bootstrap-series", - }).Counter("insert-async.errors"), insertAsyncWriteErrors: scope.Tagged(map[string]string{ "error_type": "write-value", }).Counter("insert-async.errors"), - seriesBootstrapBlocksToBuffer: seriesBootstrapScope.Counter("blocks-to-buffer"), - seriesBootstrapBlocksMerged: seriesBootstrapScope.Counter("blocks-merged"), seriesTicked: scope.Tagged(map[string]string{ "shard": fmt.Sprintf("%d", shardID), }).Gauge("series-ticked"), @@ -305,7 +297,6 @@ func newDatabaseShard( if !needsBootstrap { s.bootstrapState = Bootstrapped - s.newSeriesBootstrapped = true } if blockRetriever != nil { @@ -788,10 +779,9 @@ func (s *dbShard) purgeExpiredSeries(expiredEntries []*lookup.Entry) { count := entry.ReaderWriterCount() // The contract requires all entries to have count >= 1. if count < 1 { - s.logger.Error("observed series with invalid ReaderWriterCount in `purgeExpiredSeries`", + s.logger.Error("purgeExpiredSeries encountered invalid series read/write count", zap.String("series", series.ID().String()), - zap.Int32("ReaderWriterCount", count), - ) + zap.Int32("readerWriterCount", count)) continue } // If this series is currently being written to or read from, we don't @@ -974,6 +964,63 @@ func (s *dbShard) writeAndIndex( return series, wasWritten, nil } +func (s *dbShard) SeriesReadWriteRef( + id ident.ID, + tags ident.TagIterator, + opts ShardSeriesReadWriteRefOptions, +) (SeriesReadWriteRef, error) { + // Try retrieve existing series. + entry, shardOpts, err := s.tryRetrieveWritableSeries(id) + if err != nil { + return SeriesReadWriteRef{}, err + } + + if entry != nil { + // The read/write ref is already incremented. + return SeriesReadWriteRef{ + Series: entry.Series, + Shard: s.shard, + UniqueIndex: entry.Index, + ReleaseReadWriteRef: entry, + }, nil + } + + entry, err = s.newShardEntry(id, newTagsIterArg(tags)) + if err != nil { + return SeriesReadWriteRef{}, err + } + + // Increment and let caller call the release when done. + entry.IncrementReaderWriterCount() + + now := s.nowFn() + wg, err := s.insertQueue.Insert(dbShardInsert{ + entry: entry, + opts: dbShardInsertAsyncOptions{ + hasPendingIndexing: opts.ReverseIndex, + pendingIndex: dbShardPendingIndex{ + timestamp: now, + enqueuedAt: now, + }, + }, + }) + if err != nil { + return SeriesReadWriteRef{}, err + } + + if !shardOpts.writeNewSeriesAsync { + // Wait if not setup to write new series async. + wg.Wait() + } + + return SeriesReadWriteRef{ + Series: entry.Series, + Shard: s.shard, + UniqueIndex: entry.Index, + ReleaseReadWriteRef: entry, + }, nil +} + func (s *dbShard) ReadEncoded( ctx context.Context, id ident.ID, @@ -1131,10 +1178,10 @@ func (s *dbShard) newShardEntry( // handle on these. seriesTags.NoFinalize() + uniqueIndex := s.increasingIndex.nextIndex() series := s.seriesPool.Get() - series.Reset(seriesID, seriesTags, s.seriesBlockRetriever, + series.Reset(seriesID, seriesTags, uniqueIndex, s.seriesBlockRetriever, s.seriesOnRetrieveBlock, s, s.seriesOpts) - uniqueIndex := s.increasingIndex.nextIndex() return lookup.NewEntry(series, uniqueIndex), nil } @@ -1263,17 +1310,6 @@ func (s *dbShard) insertSeriesSync( return nil, err } - if s.newSeriesBootstrapped { - _, err := entry.Series.Load( - series.LoadOptions{Bootstrap: true}, - nil, - series.BootstrappedBlockStateSnapshot{}) - if err != nil { - entry = nil // Don't increment the writer count for this series - return nil, err - } - } - s.insertNewShardEntryWithLock(entry) return entry, nil } @@ -1281,7 +1317,7 @@ func (s *dbShard) insertSeriesSync( func (s *dbShard) insertNewShardEntryWithLock(entry *lookup.Entry) { // Set the lookup value, we use the copied ID and since it is GC'd // we explicitly set it with options to not copy the key and not to - // finalize it + // finalize it. copiedID := entry.Series.ID() listElem := s.list.PushBack(entry) s.lookup.SetUnsafe(copiedID, listElem, shardMapSetUnsafeOptions{ @@ -1352,15 +1388,6 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Insert still pending, perform the insert entry = inserts[i].entry - if s.newSeriesBootstrapped { - _, err := entry.Series.Load( - series.LoadOptions{Bootstrap: true}, - nil, - series.BootstrappedBlockStateSnapshot{}) - if err != nil { - s.metrics.insertAsyncBootstrapErrors.Inc(1) - } - } s.insertNewShardEntryWithLock(entry) } s.Unlock() @@ -1777,9 +1804,7 @@ func (s *dbShard) FetchBlocksMetadataV2( return result, nil, nil } -func (s *dbShard) Bootstrap( - bootstrappedSeries *result.Map, -) error { +func (s *dbShard) Bootstrap() error { s.Lock() if s.bootstrapState == Bootstrapped { s.Unlock() @@ -1792,44 +1817,13 @@ func (s *dbShard) Bootstrap( s.bootstrapState = Bootstrapping s.Unlock() - // Iterate flushed time ranges to determine which blocks are retrievable. This step happens - // first because the flushState information is required for bootstrapping individual series - // (will be passed to series.Load() as BlockState). + // Iterate flushed time ranges to determine which blocks are retrievable. s.bootstrapFlushStates() - multiErr := xerrors.NewMultiError() - bootstrapResult, err := s.loadSeries(bootstrappedSeries, true) - if err != nil { - multiErr = multiErr.Add(err) - } - s.emitBootstrapResult(bootstrapResult) - - // From this point onwards, all newly created series that aren't in - // the existing map should be considered bootstrapped because they - // have no data within the retention period. - s.Lock() - s.newSeriesBootstrapped = true - s.Unlock() - - // Find the series with no data within the retention period but has - // buffered data points since server start. Any new series added - // after this will be marked as bootstrapped. - s.forEachShardEntry(func(entry *lookup.Entry) bool { - seriesEntry := entry.Series - if seriesEntry.IsBootstrapped() { - return true - } - _, err := seriesEntry.Load( - series.LoadOptions{Bootstrap: true}, - nil, - series.BootstrappedBlockStateSnapshot{}) - multiErr = multiErr.Add(err) - return true - }) - // Now that this shard has finished bootstrapping, attempt to cache all of its seekers. Cannot call // this earlier as block lease verification will fail due to the shards not being bootstrapped // (and as a result no leases can be verified since the flush state is not yet known). + multiErr := xerrors.NewMultiError() if err := s.cacheShardIndices(); err != nil { multiErr = multiErr.Add(err) } @@ -1841,7 +1835,7 @@ func (s *dbShard) Bootstrap( return multiErr.FinalError() } -func (s *dbShard) Load( +func (s *dbShard) LoadBlocks( seriesToLoad *result.Map, ) error { if seriesToLoad == nil { @@ -1857,39 +1851,14 @@ func (s *dbShard) Load( } s.Unlock() - _, err := s.loadSeries(seriesToLoad, false) - return err -} - -func (s *dbShard) loadSeries( - seriesToLoad *result.Map, - bootstrap bool, -) (dbShardBootstrapResult, error) { - if seriesToLoad == nil { - return dbShardBootstrapResult{}, nil - } - - if !bootstrap { - memTracker := s.opts.MemoryTracker() - estimatedSize := result.EstimateMapBytesSize(seriesToLoad) - ok := memTracker.IncNumLoadedBytes(estimatedSize) - if !ok { - return dbShardBootstrapResult{}, ErrDatabaseLoadLimitHit - } + memTracker := s.opts.MemoryTracker() + estimatedSize := result.EstimateMapBytesSize(seriesToLoad) + ok := memTracker.IncNumLoadedBytes(estimatedSize) + if !ok { + return ErrDatabaseLoadLimitHit } - var ( - // Only used for the bootstrap path. - shardBootstrapResult = dbShardBootstrapResult{} - multiErr = xerrors.NewMultiError() - ) - // Safe to use the same snapshot for all the series since the block states can't change while - // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. - blockStates := s.BlockStatesSnapshot() - blockStatesSnapshot, bootstrapped := blockStates.UnwrapValue() - if !bootstrapped { - return dbShardBootstrapResult{}, errFlushStateIsNotBootstrapped - } + multiErr := xerrors.NewMultiError() for _, elem := range seriesToLoad.Iter() { dbBlocks := elem.Value() @@ -1919,24 +1888,29 @@ func (s *dbShard) loadSeries( dbBlocks.Tags.Finalize() } - loadOpts := series.LoadOptions{Bootstrap: bootstrap} - loadResult, err := entry.Series.Load( - loadOpts, - dbBlocks.Blocks, - blockStatesSnapshot) - if err != nil { - multiErr = multiErr.Add(err) - } - if bootstrap { - shardBootstrapResult.update(loadResult.Bootstrap) + for _, block := range dbBlocks.Blocks.AllBlocks() { + // NB(rartoul): The data being loaded is not part of the bootstrap process then it needs to be + // loaded as a cold write because the load could be happening concurrently with + // other processes like the flush (as opposed to bootstrap which cannot happen + // concurrently with a flush) and there is no way to know if this series/block + // combination has been warm flushed or not yet since updating the shard block state + // doesn't happen until the entire flush completes. + // + // As a result the only safe operation is to load the block as a cold write which + // ensures that the data will eventually be flushed and merged with the existing data + // on disk in the two scenarios where the Load() API is used (cold writes and repairs). + err = entry.Series.LoadBlock(block, series.ColdWrite) + if err != nil { + multiErr = multiErr.Add(err) + } + // Cannot close blocks once done as series takes ref to them. } - // Cannot close blocks once done as series takes ref to them. // Always decrement the writer count, avoid continue on bootstrap error entry.DecrementReaderWriterCount() } - return shardBootstrapResult, multiErr.FinalError() + return multiErr.FinalError() } func (s *dbShard) bootstrapFlushStates() error { @@ -2449,11 +2423,6 @@ func (s *dbShard) BootstrapState() BootstrapState { return bs } -func (s *dbShard) emitBootstrapResult(r dbShardBootstrapResult) { - s.metrics.seriesBootstrapBlocksToBuffer.Inc(r.numBlocksMovedToBuffer) - s.metrics.seriesBootstrapBlocksMerged.Inc(r.numBlocksMerged) -} - func (s *dbShard) logFlushResult(r dbShardFlushResult) { s.logger.Debug("shard flush outcome", zap.Uint32("shard", s.ID()), @@ -2461,18 +2430,6 @@ func (s *dbShard) logFlushResult(r dbShardFlushResult) { ) } -// dbShardBootstrapResult is a helper struct for keeping track of the result of bootstrapping all the -// series in the shard. -type dbShardBootstrapResult struct { - numBlocksMovedToBuffer int64 - numBlocksMerged int64 -} - -func (r *dbShardBootstrapResult) update(u series.BootstrapResult) { - r.numBlocksMovedToBuffer += u.NumBlocksMovedToBuffer - r.numBlocksMerged += u.NumBlocksMerged -} - // dbShardFlushResult is a helper struct for keeping track of the result of flushing all the // series in the shard. type dbShardFlushResult struct { diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 88bcf2c589..0805d172d1 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -300,7 +300,7 @@ func (q *dbShardInsertQueue) Insert(insert dbShardInsert) (*sync.WaitGroup, erro } if limit := q.insertPerSecondLimit; limit > 0 { if q.insertPerSecondLimitWindowNanos != windowNanos { - // Rolled into to a new window + // Rolled into a new window q.insertPerSecondLimitWindowNanos = windowNanos q.insertPerSecondLimitWindowValues = 0 } diff --git a/src/dbnode/storage/shard_race_prop_test.go b/src/dbnode/storage/shard_race_prop_test.go index e0a2de401e..ab228cec74 100644 --- a/src/dbnode/storage/shard_race_prop_test.go +++ b/src/dbnode/storage/shard_race_prop_test.go @@ -215,7 +215,7 @@ func TestShardTickBootstrapWriteRace(t *testing.T) { id := ident.StringID(fmt.Sprintf("foo.%d", i)) // existing ids if i < 20 { - addTestSeriesWithCountAndBootstrap(shard, id, 0, false) + addTestSeriesWithCount(shard, id, 0) } // write ids if i >= 10 { @@ -246,6 +246,7 @@ func TestShardTickBootstrapWriteRace(t *testing.T) { wg.Done() } + assert.NoError(t, shard.Bootstrap()) for _, id := range writeIDs { id := id go func() { @@ -263,7 +264,7 @@ func TestShardTickBootstrapWriteRace(t *testing.T) { go func() { defer doneFn() <-barrier - err := shard.Bootstrap(bootstrapResult) + err := shard.LoadBlocks(bootstrapResult) assert.NoError(t, err) }() diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 4026338566..11c1ffbb85 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -107,7 +107,7 @@ func TestShardDontNeedBootstrap(t *testing.T) { defer shard.Close() require.Equal(t, Bootstrapped, shard.bootstrapState) - require.True(t, shard.newSeriesBootstrapped) + require.True(t, shard.IsBootstrapped()) } func TestShardErrorIfDoubleBootstrap(t *testing.T) { @@ -120,15 +120,15 @@ func TestShardErrorIfDoubleBootstrap(t *testing.T) { defer shard.Close() require.Equal(t, Bootstrapped, shard.bootstrapState) - require.True(t, shard.newSeriesBootstrapped) + require.True(t, shard.IsBootstrapped()) } func TestShardBootstrapState(t *testing.T) { opts := DefaultTestOptions() s := testDatabaseShard(t, opts) defer s.Close() - require.NoError(t, s.Bootstrap(nil)) - require.Error(t, s.Bootstrap(nil)) + require.NoError(t, s.Bootstrap()) + require.Error(t, s.Bootstrap()) } func TestShardFlushStateNotStarted(t *testing.T) { @@ -154,7 +154,7 @@ func TestShardFlushStateNotStarted(t *testing.T) { s := testDatabaseShard(t, opts) defer s.Close() - s.Bootstrap(nil) + s.Bootstrap() notStarted := fileOpState{WarmStatus: fileOpNotStarted} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { @@ -164,46 +164,6 @@ func TestShardFlushStateNotStarted(t *testing.T) { } } -func TestShardBootstrapWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - opts := DefaultTestOptions() - s := testDatabaseShard(t, opts) - defer s.Close() - - fooSeries := series.NewMockDatabaseSeries(ctrl) - fooSeries.EXPECT().ID().Return(ident.StringID("foo")).AnyTimes() - fooSeries.EXPECT().IsEmpty().Return(false).AnyTimes() - barSeries := series.NewMockDatabaseSeries(ctrl) - barSeries.EXPECT().ID().Return(ident.StringID("bar")).AnyTimes() - barSeries.EXPECT().IsEmpty().Return(false).AnyTimes() - s.Lock() - s.insertNewShardEntryWithLock(lookup.NewEntry(fooSeries, 0)) - s.insertNewShardEntryWithLock(lookup.NewEntry(barSeries, 0)) - s.Unlock() - - fooBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) - barBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) - fooSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, fooBlocks, gomock.Any()).Return(series.LoadResult{}, nil) - fooSeries.EXPECT().IsBootstrapped().Return(true) - barSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, barBlocks, gomock.Any()).Return(series.LoadResult{}, errors.New("series error")) - barSeries.EXPECT().IsBootstrapped().Return(true) - - fooID := ident.StringID("foo") - barID := ident.StringID("bar") - - bootstrappedSeries := result.NewMap(result.MapOptions{}) - bootstrappedSeries.Set(fooID, result.DatabaseSeriesBlocks{ID: fooID, Blocks: fooBlocks}) - bootstrappedSeries.Set(barID, result.DatabaseSeriesBlocks{ID: barID, Blocks: barBlocks}) - - err := s.Bootstrap(bootstrappedSeries) - - require.NotNil(t, err) - require.Equal(t, "series error", err.Error()) - require.Equal(t, Bootstrapped, s.bootstrapState) -} - // TestShardBootstrapWithFlushVersion ensures that the shard is able to bootstrap // the cold flush version from the info files. func TestShardBootstrapWithFlushVersion(t *testing.T) { @@ -232,7 +192,6 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { mockSeries := series.NewMockDatabaseSeries(ctrl) mockSeries.EXPECT().ID().Return(mockSeriesID).AnyTimes() mockSeries.EXPECT().IsEmpty().Return(false).AnyTimes() - mockSeries.EXPECT().IsBootstrapped().Return(true).AnyTimes() // Load the mock into the shard as an expected series so that we can assert // on the call to its Bootstrap() method below. @@ -262,30 +221,9 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { require.NoError(t, writer.Close()) } - bootstrappedSeries := result.NewMap(result.MapOptions{}) - blocks := block.NewMockDatabaseSeriesBlocks(ctrl) - blocks.EXPECT().AllBlocks().AnyTimes() - blocks.EXPECT().Len().AnyTimes() - bootstrappedSeries.Set(mockSeriesID, result.DatabaseSeriesBlocks{ - ID: mockSeriesID, - Blocks: blocks, - }) - - // Ensure that the bootstrapped flush/block states get passed to the series.Load() - // method properly. - blockStateSnapshot := series.BootstrappedBlockStateSnapshot{ - Snapshot: map[xtime.UnixNano]series.BlockState{}, - } - for i, blockStart := range blockStarts { - blockStateSnapshot.Snapshot[xtime.ToUnixNano(blockStart)] = series.BlockState{ - WarmRetrievable: true, - ColdVersion: i, - } - } - mockSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, blocks, blockStateSnapshot) - - err = s.Bootstrap(bootstrappedSeries) + err = s.Bootstrap() require.NoError(t, err) + require.Equal(t, Bootstrapped, s.bootstrapState) for i, blockStart := range blockStarts { @@ -344,9 +282,7 @@ func TestShardBootstrapWithFlushVersionNoCleanUp(t *testing.T) { require.NoError(t, writer.Close()) } - bootstrappedSeries := result.NewMap(result.MapOptions{}) - - err = s.Bootstrap(bootstrappedSeries) + err = s.Bootstrap() require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) @@ -385,9 +321,7 @@ func TestShardBootstrapWithCacheShardIndices(t *testing.T) { mockRetriever.EXPECT().CacheShardIndices([]uint32{s.ID()}).Return(nil) mockRetrieverMgr.EXPECT().Retriever(s.namespace).Return(mockRetriever, nil) - bootstrappedSeries := result.NewMap(result.MapOptions{}) - - err = s.Bootstrap(bootstrappedSeries) + err = s.Bootstrap() require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) } @@ -434,15 +368,15 @@ func testShardLoadLimit(t *testing.T, limit int64, shouldReturnError bool) { sr.AddBlock(ident.StringID("bar"), barTags, blocks[1]) seriesMap := sr.AllSeries() - require.NoError(t, s.Bootstrap(nil)) + require.NoError(t, s.Bootstrap()) // First load will never trigger the limit. - require.NoError(t, s.Load(seriesMap)) + require.NoError(t, s.LoadBlocks(seriesMap)) if shouldReturnError { - require.Error(t, s.Load(seriesMap)) + require.Error(t, s.LoadBlocks(seriesMap)) } else { - require.NoError(t, s.Load(seriesMap)) + require.NoError(t, s.LoadBlocks(seriesMap)) } } @@ -454,7 +388,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { s := testDatabaseShard(t, DefaultTestOptions()) defer s.Close() - s.Bootstrap(nil) + s.Bootstrap() s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, NumFailures: 1, @@ -525,7 +459,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) s := testDatabaseShard(t, opts) defer s.Close() - s.Bootstrap(nil) + s.Bootstrap() s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, NumFailures: 1, @@ -614,7 +548,7 @@ func TestShardColdFlush(t *testing.T) { blockSize := opts.SeriesOptions().RetentionOptions().BlockSize() shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + require.NoError(t, shard.Bootstrap()) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -694,7 +628,7 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) blockSize := opts.SeriesOptions().RetentionOptions().BlockSize() shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + require.NoError(t, shard.Bootstrap()) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -877,14 +811,7 @@ func addTestSeries(shard *dbShard, id ident.ID) series.DatabaseSeries { } func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.DatabaseSeries { - return addTestSeriesWithCountAndBootstrap(shard, id, count, true) -} - -func addTestSeriesWithCountAndBootstrap(shard *dbShard, id ident.ID, count int32, bootstrap bool) series.DatabaseSeries { - seriesEntry := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) - if bootstrap { - seriesEntry.Load(series.LoadOptions{Bootstrap: true}, nil, series.BootstrappedBlockStateSnapshot{}) - } + seriesEntry := series.NewDatabaseSeries(id, ident.Tags{}, 1, shard.seriesOpts) shard.Lock() entry := lookup.NewEntry(seriesEntry, 0) for i := int32(0); i < count; i++ { @@ -951,7 +878,7 @@ func TestShardTick(t *testing.T) { sleepPerSeries := time.Microsecond shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + shard.Bootstrap() shard.SetRuntimeOptions(runtime.NewOptions(). SetTickPerSeriesSleepDuration(sleepPerSeries). SetTickSeriesBatchSize(1)) @@ -1120,7 +1047,7 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { sleepPerSeries := time.Microsecond shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + shard.Bootstrap() shard.SetRuntimeOptions(runtime.NewOptions(). SetWriteNewSeriesAsync(true). SetTickPerSeriesSleepDuration(sleepPerSeries). @@ -1177,7 +1104,7 @@ func TestShardTickRace(t *testing.T) { opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) defer shard.Close() - shard.Bootstrap(nil) + shard.Bootstrap() addTestSeries(shard, ident.StringID("foo")) var wg sync.WaitGroup @@ -1206,7 +1133,7 @@ func TestShardTickRace(t *testing.T) { func TestShardTickCleanupSmallBatchSize(t *testing.T) { opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + shard.Bootstrap() addTestSeries(shard, ident.StringID("foo")) shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) require.Equal(t, 0, shard.lookup.Len()) @@ -1229,7 +1156,7 @@ func TestShardReturnsErrorForConcurrentTicks(t *testing.T) { SetFilesystemOptions(fsOpts)) shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) + shard.Bootstrap() shard.currRuntimeOptions.tickSleepSeriesBatchSize = 1 shard.currRuntimeOptions.tickSleepPerSeries = time.Millisecond @@ -1580,7 +1507,7 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { shard := testDatabaseShard(t, opts) defer shard.Close() - require.NoError(t, shard.Bootstrap(nil)) + require.NoError(t, shard.Bootstrap()) ropts := shard.seriesOpts.RetentionOptions() end := opts.ClockOptions().NowFn()().Truncate(ropts.BlockSize()) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 58da3fa0d7..8af24576dc 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -961,6 +961,20 @@ func (mr *MockNamespaceMockRecorder) ID() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ID", reflect.TypeOf((*MockNamespace)(nil).ID)) } +// Metadata mocks base method +func (m *MockNamespace) Metadata() namespace.Metadata { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Metadata") + ret0, _ := ret[0].(namespace.Metadata) + return ret0 +} + +// Metadata indicates an expected call of Metadata +func (mr *MockNamespaceMockRecorder) Metadata() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Metadata", reflect.TypeOf((*MockNamespace)(nil).Metadata)) +} + // Schema mocks base method func (m *MockNamespace) Schema() namespace.SchemaDescr { m.ctrl.T.Helper() @@ -1054,6 +1068,20 @@ func (mr *MockdatabaseNamespaceMockRecorder) ID() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ID", reflect.TypeOf((*MockdatabaseNamespace)(nil).ID)) } +// Metadata mocks base method +func (m *MockdatabaseNamespace) Metadata() namespace.Metadata { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Metadata") + ret0, _ := ret[0].(namespace.Metadata) + return ret0 +} + +// Metadata indicates an expected call of Metadata +func (mr *MockdatabaseNamespaceMockRecorder) Metadata() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Metadata", reflect.TypeOf((*MockdatabaseNamespace)(nil).Metadata)) +} + // Schema mocks base method func (m *MockdatabaseNamespace) Schema() namespace.SchemaDescr { m.ctrl.T.Helper() @@ -1274,17 +1302,17 @@ func (mr *MockdatabaseNamespaceMockRecorder) FetchBlocksMetadataV2(ctx, shardID, } // Bootstrap mocks base method -func (m *MockdatabaseNamespace) Bootstrap(start time.Time, process bootstrap.Process) error { +func (m *MockdatabaseNamespace) Bootstrap(bootstrapResult bootstrap.NamespaceResult) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Bootstrap", start, process) + ret := m.ctrl.Call(m, "Bootstrap", bootstrapResult) ret0, _ := ret[0].(error) return ret0 } // Bootstrap indicates an expected call of Bootstrap -func (mr *MockdatabaseNamespaceMockRecorder) Bootstrap(start, process interface{}) *gomock.Call { +func (mr *MockdatabaseNamespaceMockRecorder) Bootstrap(bootstrapResult interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseNamespace)(nil).Bootstrap), start, process) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseNamespace)(nil).Bootstrap), bootstrapResult) } // WarmFlush mocks base method @@ -1416,6 +1444,21 @@ func (mr *MockdatabaseNamespaceMockRecorder) FlushState(shardID, blockStart inte return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FlushState", reflect.TypeOf((*MockdatabaseNamespace)(nil).FlushState), shardID, blockStart) } +// SeriesReadWriteRef mocks base method +func (m *MockdatabaseNamespace) SeriesReadWriteRef(shardID uint32, id ident.ID, tags ident.TagIterator) (SeriesReadWriteRef, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SeriesReadWriteRef", shardID, id, tags) + ret0, _ := ret[0].(SeriesReadWriteRef) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SeriesReadWriteRef indicates an expected call of SeriesReadWriteRef +func (mr *MockdatabaseNamespaceMockRecorder) SeriesReadWriteRef(shardID, id, tags interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeriesReadWriteRef", reflect.TypeOf((*MockdatabaseNamespace)(nil).SeriesReadWriteRef), shardID, id, tags) +} + // MockShard is a mock of Shard interface type MockShard struct { ctrl *gomock.Controller @@ -1709,31 +1752,31 @@ func (mr *MockdatabaseShardMockRecorder) FetchBlocksMetadataV2(ctx, start, end, } // Bootstrap mocks base method -func (m *MockdatabaseShard) Bootstrap(bootstrappedSeries *result.Map) error { +func (m *MockdatabaseShard) Bootstrap() error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Bootstrap", bootstrappedSeries) + ret := m.ctrl.Call(m, "Bootstrap") ret0, _ := ret[0].(error) return ret0 } // Bootstrap indicates an expected call of Bootstrap -func (mr *MockdatabaseShardMockRecorder) Bootstrap(bootstrappedSeries interface{}) *gomock.Call { +func (mr *MockdatabaseShardMockRecorder) Bootstrap() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap), bootstrappedSeries) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap)) } -// Load mocks base method -func (m *MockdatabaseShard) Load(series *result.Map) error { +// LoadBlocks mocks base method +func (m *MockdatabaseShard) LoadBlocks(series *result.Map) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Load", series) + ret := m.ctrl.Call(m, "LoadBlocks", series) ret0, _ := ret[0].(error) return ret0 } -// Load indicates an expected call of Load -func (mr *MockdatabaseShardMockRecorder) Load(series interface{}) *gomock.Call { +// LoadBlocks indicates an expected call of LoadBlocks +func (mr *MockdatabaseShardMockRecorder) LoadBlocks(series interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockdatabaseShard)(nil).Load), series) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LoadBlocks", reflect.TypeOf((*MockdatabaseShard)(nil).LoadBlocks), series) } // WarmFlush mocks base method @@ -1852,6 +1895,21 @@ func (mr *MockdatabaseShardMockRecorder) TagsFromSeriesID(seriesID interface{}) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TagsFromSeriesID", reflect.TypeOf((*MockdatabaseShard)(nil).TagsFromSeriesID), seriesID) } +// SeriesReadWriteRef mocks base method +func (m *MockdatabaseShard) SeriesReadWriteRef(id ident.ID, tags ident.TagIterator, opts ShardSeriesReadWriteRefOptions) (SeriesReadWriteRef, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SeriesReadWriteRef", id, tags, opts) + ret0, _ := ret[0].(SeriesReadWriteRef) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SeriesReadWriteRef indicates an expected call of SeriesReadWriteRef +func (mr *MockdatabaseShardMockRecorder) SeriesReadWriteRef(id, tags, opts interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeriesReadWriteRef", reflect.TypeOf((*MockdatabaseShard)(nil).SeriesReadWriteRef), id, tags, opts) +} + // MocknamespaceIndex is a mock of namespaceIndex interface type MocknamespaceIndex struct { ctrl *gomock.Controller @@ -2149,11 +2207,12 @@ func (mr *MockdatabaseBootstrapManagerMockRecorder) LastBootstrapCompletionTime( } // Bootstrap mocks base method -func (m *MockdatabaseBootstrapManager) Bootstrap() error { +func (m *MockdatabaseBootstrapManager) Bootstrap() (BootstrapResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Bootstrap") - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(BootstrapResult) + ret1, _ := ret[1].(error) + return ret0, ret1 } // Bootstrap indicates an expected call of Bootstrap @@ -2650,11 +2709,12 @@ func (mr *MockdatabaseMediatorMockRecorder) LastBootstrapCompletionTime() *gomoc } // Bootstrap mocks base method -func (m *MockdatabaseMediator) Bootstrap() error { +func (m *MockdatabaseMediator) Bootstrap() (BootstrapResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Bootstrap") - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(BootstrapResult) + ret1, _ := ret[1].(error) + return ret0, ret1 } // Bootstrap indicates an expected call of Bootstrap diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index f6b167f8db..256c8d0220 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -39,6 +39,7 @@ import ( "github.com/m3db/m3/src/dbnode/storage/index" "github.com/m3db/m3/src/dbnode/storage/repair" "github.com/m3db/m3/src/dbnode/storage/series" + "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/dbnode/x/xpool" @@ -217,7 +218,7 @@ type Database interface { FlushState(namespace ident.ID, shardID uint32, blockStart time.Time) (fileOpState, error) } -// database is the internal database interface +// database is the internal database interface. type database interface { Database @@ -228,25 +229,28 @@ type database interface { UpdateOwnedNamespaces(namespaces namespace.Map) error } -// Namespace is a time series database namespace +// Namespace is a time series database namespace. type Namespace interface { - // Options returns the namespace options + // Options returns the namespace options. Options() namespace.Options - // ID returns the ID of the namespace + // ID returns the ID of the namespace. ID() ident.ID + // Metadata returns the metadata of the namespace. + Metadata() namespace.Metadata + // Schema returns the schema of the namespace. Schema() namespace.SchemaDescr - // NumSeries returns the number of series in the namespace + // NumSeries returns the number of series in the namespace. NumSeries() int64 - // Shards returns the shard description + // Shards returns the shard description. Shards() []Shard } -// NamespacesByID is a sortable slice of namespaces by ID +// NamespacesByID is a sortable slice of namespaces by ID. type NamespacesByID []Namespace func (n NamespacesByID) Len() int { return len(n) } @@ -334,8 +338,8 @@ type databaseNamespace interface { opts block.FetchBlocksMetadataOptions, ) (block.FetchBlocksMetadataResults, PageToken, error) - // Bootstrap performs bootstrapping. - Bootstrap(start time.Time, process bootstrap.Process) error + // Bootstrap marks shards as bootstrapped for the namespace. + Bootstrap(bootstrapResult bootstrap.NamespaceResult) error // WarmFlush flushes in-memory WarmWrites. WarmFlush(blockStart time.Time, flush persist.FlushPreparer) error @@ -370,6 +374,30 @@ type databaseNamespace interface { // FlushState returns the flush state for the specified shard and block start. FlushState(shardID uint32, blockStart time.Time) (fileOpState, error) + + // SeriesReadWriteRef returns a read/write ref to a series, callers + // must make sure to call the release callback once finished + // with the reference. + SeriesReadWriteRef( + shardID uint32, + id ident.ID, + tags ident.TagIterator, + ) (SeriesReadWriteRef, error) +} + +// SeriesReadWriteRef is a read/write reference for a series, +// must make sure to release +type SeriesReadWriteRef struct { + // Series reference for read/writing. + Series series.DatabaseSeries + // UniqueIndex is the unique index of the series (as applicable). + UniqueIndex uint64 + // Shard is the shard of the series. + Shard uint32 + // ReleaseReadWriteRef must be called after using the series ref + // to release the reference count to the series so it can + // be expired by the owning shard eventually. + ReleaseReadWriteRef lookup.OnReleaseReadWriteRef } // Shard is a time series database shard. @@ -401,6 +429,7 @@ type databaseShard interface { // Tick performs all async updates Tick(c context.Cancellable, startTime time.Time, nsCtx namespace.Context) (tickResult, error) + // Write writes a value to the shard for an ID. Write( ctx context.Context, id ident.ID, @@ -411,7 +440,7 @@ type databaseShard interface { wOpts series.WriteOptions, ) (ts.Series, bool, error) - // WriteTagged values to the shard for an ID. + // WriteTagged writes a value to the shard for an ID with tags. WriteTagged( ctx context.Context, id ident.ID, @@ -459,14 +488,14 @@ type databaseShard interface { opts block.FetchBlocksMetadataOptions, ) (block.FetchBlocksMetadataResults, PageToken, error) - // Bootstrap bootstraps the shard with provided data. - Bootstrap( - bootstrappedSeries *result.Map, - ) error + // Bootstrap bootstraps the shard after all provided data + // has been loaded using LoadBootstrapBlocks. + Bootstrap() error - // Load does the same thing as Bootstrap, except it can be called more than once - // and after a shard is bootstrapped already. - Load(series *result.Map) error + // LoadBlocks does the same thing as LoadBootstrapBlocks, + // except it can be called more than once and after a shard is + // bootstrapped already. + LoadBlocks(series *result.Map) error // WarmFlush flushes the WarmWrites in this shard. WarmFlush( @@ -511,7 +540,24 @@ type databaseShard interface { ) (repair.MetadataComparisonResult, error) // TagsFromSeriesID returns the series tags from a series ID. + // TODO(r): Seems like this is a work around that shouldn't be + // necessary given the callsites that current exist? TagsFromSeriesID(seriesID ident.ID) (ident.Tags, bool, error) + + // SeriesReadWriteRef returns a read/write ref to a series, callers + // must make sure to call the release callback once finished + // with the reference. + SeriesReadWriteRef( + id ident.ID, + tags ident.TagIterator, + opts ShardSeriesReadWriteRefOptions, + ) (SeriesReadWriteRef, error) +} + +// ShardSeriesReadWriteRefOptions are options for SeriesReadWriteRef +// for the shard. +type ShardSeriesReadWriteRefOptions struct { + ReverseIndex bool } // namespaceIndex indexes namespace writes. @@ -607,12 +653,18 @@ type databaseBootstrapManager interface { LastBootstrapCompletionTime() (time.Time, bool) // Bootstrap performs bootstrapping for all namespaces and shards owned. - Bootstrap() error + Bootstrap() (BootstrapResult, error) // Report reports runtime information. Report() } +// BootstrapResult is a bootstrap result. +type BootstrapResult struct { + ErrorsBootstrap []error + AlreadyBootstrapping bool +} + // databaseFlushManager manages flushing in-memory data to persistent storage. type databaseFlushManager interface { // Flush flushes in-memory data to persistent storage. @@ -720,7 +772,7 @@ type databaseMediator interface { LastBootstrapCompletionTime() (time.Time, bool) // Bootstrap bootstraps the database with file operations performed at the end. - Bootstrap() error + Bootstrap() (BootstrapResult, error) // DisableFileOps disables file operations. DisableFileOps() diff --git a/src/x/generated-source-files.mk b/src/x/generated-source-files.mk index 109be81cbd..324c224308 100644 --- a/src/x/generated-source-files.mk +++ b/src/x/generated-source-files.mk @@ -8,8 +8,8 @@ gorename_package_version := 52c7307cddd221bb98f0a3215216789f3c821b10 # Tests that all currently generated types match their contents if they were regenerated .PHONY: test-genny-all test-genny-all: genny-all - @test "$(shell git diff --shortstat 2>/dev/null)" = "" || (git diff --no-color && echo "Check git status, there are dirty files" && exit 1) - @test "$(shell git status --porcelain 2>/dev/null | grep "^??")" = "" || (git status --porcelain && echo "Check git status, there are untracked files" && exit 1) + @test "$(shell git --no-pager diff --shortstat 2>/dev/null)" = "" || (git --no-pager diff --no-color && echo "Check git status, there are dirty files" && exit 1) + @test "$(shell git --no-pager status --porcelain 2>/dev/null | grep "^??")" = "" || (git status --porcelain && echo "Check git status, there are untracked files" && exit 1) .PHONY: genny-all genny-all: genny-map-all genny-list-all genny-arraypool-all @@ -79,25 +79,25 @@ hashmap-gen-rename: install-gorename $(eval out_dir=$(gopath_prefix)/$(target_package)) $(eval temp_outdir=$(out_dir)$(temp_suffix)) echo $(temp_outdir) - @if [ -d $(temp_outdir) ] ; then echo "temp directory $(temp_outdir) exists, failing" ; exit 1 ; fi + @if test -d $(temp_outdir); then echo "temp directory $(temp_outdir) exists, failing" ; exit 1 ; fi mkdir -p $(temp_outdir) - [ ! -f $(out_dir)/new_map_gen.go ] || mv $(out_dir)/new_map_gen.go $(temp_outdir)/new_map_gen.go + ! test -f $(out_dir)/new_map_gen.go || mv $(out_dir)/new_map_gen.go $(temp_outdir)/new_map_gen.go ifeq ($(rename_nogen_key),) # allow users to short circuit the generation of key.go if they don't need it. echo 'package $(pkg)' > $(temp_outdir)/key.go echo '' >> $(temp_outdir)/key.go - [ "$(key_type_alias)" == "" ] || echo "type $(key_type_alias) interface{}" >> $(temp_outdir)/key.go + test "$(key_type_alias)" == "" || echo "type $(key_type_alias) interface{}" >> $(temp_outdir)/key.go endif ifeq ($(rename_nogen_value),) # Allow users to short circuit the generation of value.go if they don't need it. echo 'package $(pkg)' > $(temp_outdir)/value.go echo '' >> $(temp_outdir)/value.go - [[ $(value_type_alias) = struct* ]] || echo 'type $(value_type_alias) interface{}' >> $(temp_outdir)/value.go + test "$(value_type_alias)" = "struct*" || echo 'type $(value_type_alias) interface{}' >> $(temp_outdir)/value.go endif mv $(out_dir)/map_gen.go $(temp_outdir)/map_gen.go make hashmap-gen-rename-helper mv $(temp_outdir)/map_gen.go $(out_dir)/map_gen.go - [ ! -f $(temp_outdir)/new_map_gen.go ] || mv $(temp_outdir)/new_map_gen.go $(out_dir)/new_map_gen.go + ! test -f $(temp_outdir)/new_map_gen.go || mv $(temp_outdir)/new_map_gen.go $(out_dir)/new_map_gen.go rm -f $(temp_outdir)/key.go rm -f $(temp_outdir)/value.go rmdir $(temp_outdir) diff --git a/src/x/ident/bytes_id.go b/src/x/ident/bytes_id.go index 5a7cfb48b6..3f7990266c 100644 --- a/src/x/ident/bytes_id.go +++ b/src/x/ident/bytes_id.go @@ -58,3 +58,56 @@ func (v BytesID) IsNoFinalize() bool { // Finalize is a no-op for a bytes ID as it has no associated pool. func (v BytesID) Finalize() { } + +var _ ID = (*ReuseableBytesID)(nil) + +// ReuseableBytesID is a reuseable bytes ID, use with extreme care in +// places where the lifecycle is known (there is no checking with this +// ID). +type ReuseableBytesID struct { + bytes []byte +} + +// NewReuseableBytesID returns a new reuseable bytes ID, use with extreme +// care in places where the lifecycle is known (there is no checking with +// this ID). +func NewReuseableBytesID() *ReuseableBytesID { + return &ReuseableBytesID{} +} + +// Reset resets the bytes ID for reuse, make sure there are zero references +// to this ID from any other data structure at this point. +func (i *ReuseableBytesID) Reset(bytes []byte) { + i.bytes = bytes +} + +// Bytes implements ID. +func (i *ReuseableBytesID) Bytes() []byte { + return i.bytes +} + +// Equal implements ID. +func (i *ReuseableBytesID) Equal(value ID) bool { + return bytes.Equal(i.bytes, value.Bytes()) +} + +// NoFinalize implements ID. +func (i *ReuseableBytesID) NoFinalize() { +} + +// IsNoFinalize implements ID. +func (i *ReuseableBytesID) IsNoFinalize() bool { + // Reuseable bytes ID are always not able to not be finalized + // as this ID is reused with reset. + return false +} + +// Finalize implements ID. +func (i *ReuseableBytesID) Finalize() { + // Noop as it will be re-used. +} + +// String returns the bytes ID as a string. +func (i *ReuseableBytesID) String() string { + return string(i.bytes) +} diff --git a/src/x/test/reporter.go b/src/x/test/reporter.go index 807e509ce9..61196a0a56 100644 --- a/src/x/test/reporter.go +++ b/src/x/test/reporter.go @@ -69,3 +69,10 @@ func (r Reporter) Errorf(format string, args ...interface{}) { func (r Reporter) Fatalf(format string, args ...interface{}) { panic(fmt.Sprintf(format, args...)) } + +// NewController provides a gomock.Controller wrapped with a xtest.Reporter, +// which gives more useful error modes on unexpected mock calls. +// See xtest.Reporter for more context. +func NewController(t *testing.T) *gomock.Controller { + return gomock.NewController(Reporter{T: t}) +} From 6f2ecf52afccbb30f3ac605874417d264d5fdd76 Mon Sep 17 00:00:00 2001 From: Becca Powell Date: Tue, 26 Nov 2019 11:51:57 -0800 Subject: [PATCH 06/11] update namespace_configuration docs to use the right keys (#2045) --- docs/operational_guide/namespace_configuration.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/operational_guide/namespace_configuration.md b/docs/operational_guide/namespace_configuration.md index 17235b710b..69ffc87e16 100644 --- a/docs/operational_guide/namespace_configuration.md +++ b/docs/operational_guide/namespace_configuration.md @@ -53,16 +53,16 @@ curl -X POST :/api/v1/ "snapshotEnabled": true, "repairEnabled": false, "retentionOptions": { - "retentionPeriodDuration": "2d", - "blockSizeDuration": "2h", - "bufferFutureDuration": "10m", - "bufferPastDuration": "10m", + "retentionPeriod": "2d", + "blockSize": "2h", + "bufferFuture": "10m", + "bufferPast": "10m", "blockDataExpiry": true, - "blockDataExpiryAfterNotAccessPeriodDuration": "5m" + "blockDataExpiryAfterNotAccessedPeriod": "5m" }, "indexOptions": { "enabled": true, - "blockSizeDuration": "2h" + "blockSize": "2h" } } }' @@ -178,7 +178,7 @@ Whether to use the built-in indexing. Must be `true`. Can be modified without creating a new namespace: `no` -#### blockSizeDuration +#### blockSize The size of blocks (in duration) that the index uses. Should match the databases [blocksize](#blocksize) for optimal memory usage. From de4bc64ebf2d15fc705bafac754f78164ea10a79 Mon Sep 17 00:00:00 2001 From: Benjamin Raskin Date: Tue, 26 Nov 2019 22:18:29 -0500 Subject: [PATCH 07/11] Add monitoring to docs tree (#2050) --- mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/mkdocs.yml b/mkdocs.yml index 316f95edf0..2bc74e0bc7 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -105,6 +105,7 @@ pages: - "Bootstrapping & Crash Recovery": "operational_guide/bootstrapping_crash_recovery.md" - "Docker & Kernel Configuration": "operational_guide/kernel_configuration.md" - "etcd": "operational_guide/etcd.md" + - "Monitoring": "operational_guide/monitoring.md" - "Integrations": - "Prometheus": "integrations/prometheus.md" - "Graphite": "integrations/graphite.md" From 65b7f92cf3668f024830a96612a52d49ec1cf97f Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 27 Nov 2019 11:28:47 -0500 Subject: [PATCH 08/11] [dbnode] Fix bootstrapping data for an unowned shard with commit log (#2052) --- .../bootstrapper/commitlog/source.go | 13 ++++++++++- src/dbnode/storage/bootstrap/process.go | 22 +++++++++---------- 2 files changed, 22 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 358644ea70..afafd4770f 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -289,6 +289,7 @@ func (s *commitLogSource) Read( ReturnMetadataAsRef: true, } datapointsSkippedNotBootstrappingNamespace = 0 + datapointsSkippedNotBootstrappingShard = 0 startCommitLogsRead = s.nowFn() ) s.log.Info("read commit logs start") @@ -303,7 +304,8 @@ func (s *commitLogSource) Read( zap.Stringer("took", s.nowFn().Sub(startCommitLogsRead)), zap.Int("datapointsRead", datapointsRead), zap.Int("datapointsSkippedNotInRange", datapointsSkippedNotInRange), - zap.Int("datapointsSkippedNotBootstrappingNamespace", datapointsSkippedNotBootstrappingNamespace)) + zap.Int("datapointsSkippedNotBootstrappingNamespace", datapointsSkippedNotBootstrappingNamespace), + zap.Int("datapointsSkippedNotBootstrappingShard", datapointsSkippedNotBootstrappingShard)) }() iter, corruptFiles, err := s.newIteratorFn(iterOpts) @@ -476,6 +478,15 @@ func (s *commitLogSource) Read( datapointsSkippedNotBootstrappingNamespace++ continue } + // If not bootstrapping shard for this series then also skip. + // NB(r): This can occur when a topology change happens then we + // bootstrap from the commit log data that the node no longer owns. + shard := seriesEntry.series.Shard + _, bootstrapping := seriesEntry.namespace.dataAndIndexShardRanges[shard] + if !bootstrapping { + datapointsSkippedNotBootstrappingShard++ + continue + } // Distribute work. // NB(r): In future we could batch a few points together before sending diff --git a/src/dbnode/storage/bootstrap/process.go b/src/dbnode/storage/bootstrap/process.go index 77d1e9f0ae..87805ed7b0 100644 --- a/src/dbnode/storage/bootstrap/process.go +++ b/src/dbnode/storage/bootstrap/process.go @@ -217,9 +217,17 @@ func (b bootstrapProcess) Run( begin := b.nowFn() res, err := b.bootstrapper.Bootstrap(namespaces) took := b.nowFn().Sub(begin) + if err != nil { + b.log.Error("bootstrap process error", + zap.Duration("took", took), + zap.Error(err)) + return NamespaceResults{}, err + } + for _, entry := range namespaces.Namespaces.Iter() { namespace := entry.Value() nsID := namespace.Metadata.ID() + result, ok := res.Results.Get(nsID) if !ok { return NamespaceResults{}, @@ -228,11 +236,7 @@ func (b bootstrapProcess) Run( logFields := b.logFields(namespace.Metadata, namespace.Shards, namespace.DataTargetRange.Range, namespace.IndexTargetRange.Range) - b.logBootstrapResult(result, logFields, err, took) - } - - if err != nil { - return NamespaceResults{}, err + b.logBootstrapResult(result, logFields, took) } bootstrapResult = MergeNamespaceResults(bootstrapResult, res) @@ -286,7 +290,6 @@ func (b bootstrapProcess) logBootstrapRun( func (b bootstrapProcess) logBootstrapResult( result NamespaceResult, logFields []zapcore.Field, - err error, took time.Duration, ) { logFields = append(logFields, @@ -295,13 +298,8 @@ func (b bootstrapProcess) logBootstrapResult( logFields = append(logFields, zap.Int("numIndexBlocks", len(result.IndexResult.IndexResults()))) } - if err != nil { - logFields = append(logFields, zap.Error(err)) - b.log.Info("bootstrap range completed with error", logFields...) - return - } - b.log.Info("bootstrap range completed successfully", logFields...) + b.log.Info("bootstrap range completed", logFields...) } func (b bootstrapProcess) targetRangesForData( From 4c44f1d0f80b5f9e3073b3eaf8b4b84e230e962e Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 2 Dec 2019 19:27:48 -0500 Subject: [PATCH 09/11] [query] Restrict query by header tag (#2053) --- .../query_fanout/restrict.go | 225 +++++ .../query_fanout/restrict.sh | 19 + .../query_fanout/test.sh | 10 +- .../query_fanout/warning.sh | 2 + src/query/api/v1/handler/fetch_options.go | 52 +- .../api/v1/handler/fetch_options_test.go | 74 +- src/query/api/v1/handler/headers.go | 4 + src/query/api/v1/handler/prometheus/common.go | 47 +- .../api/v1/handler/prometheus/native/read.go | 4 +- .../handler/prometheus/native/read_common.go | 2 + .../prometheus/native/read_instantaneous.go | 4 +- .../api/v1/handler/prometheus/remote/read.go | 4 + .../prometheus/validator/handler_test.go | 2 - src/query/api/v1/handler/tag_options.go | 113 +++ src/query/api/v1/handler/tag_options_test.go | 144 ++++ src/query/api/v1/handler/types.go | 8 +- src/query/executor/engine.go | 3 +- src/query/functions/fetch_test.go | 7 +- src/query/generated/proto/rpcpb/query.pb.go | 776 +++++++++++++----- src/query/generated/proto/rpcpb/query.proto | 14 +- src/query/models/matcher.go | 9 + src/query/models/matcher_test.go | 16 +- src/query/storage/fanout/storage.go | 36 + src/query/storage/index.go | 4 +- src/query/storage/index_test.go | 2 +- src/query/storage/m3/cluster_resolver.go | 18 +- src/query/storage/m3/cluster_resolver_test.go | 42 +- src/query/storage/m3/storage.go | 8 +- src/query/storage/restrict_query_options.go | 163 ++++ .../storage/restrict_query_options_test.go | 53 ++ src/query/storage/types.go | 139 +--- src/query/storage/types_test.go | 214 ----- src/query/tsdb/remote/codecs.go | 155 +++- src/query/tsdb/remote/codecs_test.go | 310 ++++++- 34 files changed, 2065 insertions(+), 618 deletions(-) create mode 100644 scripts/docker-integration-tests/query_fanout/restrict.go create mode 100755 scripts/docker-integration-tests/query_fanout/restrict.sh create mode 100644 src/query/api/v1/handler/tag_options.go create mode 100644 src/query/api/v1/handler/tag_options_test.go create mode 100644 src/query/storage/restrict_query_options.go create mode 100644 src/query/storage/restrict_query_options_test.go delete mode 100644 src/query/storage/types_test.go diff --git a/scripts/docker-integration-tests/query_fanout/restrict.go b/scripts/docker-integration-tests/query_fanout/restrict.go new file mode 100644 index 0000000000..39527839a9 --- /dev/null +++ b/scripts/docker-integration-tests/query_fanout/restrict.go @@ -0,0 +1,225 @@ +// Copyright (c) 2019 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 main + +import ( + "encoding/json" + "flag" + "fmt" + "io/ioutil" + "net/http" + "os" + "runtime" + + "github.com/m3db/m3/src/query/api/v1/handler" + "github.com/m3db/m3/src/query/api/v1/handler/prometheus" + "github.com/m3db/m3/src/query/models" + + "github.com/stretchr/testify/require" +) + +func main() { + var ts int + flag.IntVar(&ts, "t", -1, "metric name to search") + flag.Parse() + + require.True(t, ts > 0, "no timestamp supplied") + name = fmt.Sprintf("foo_%d", ts) + instant := fmt.Sprintf("http://0.0.0.0:7201/api/v1/query?query=%s", name) + rnge := fmt.Sprintf("http://0.0.0.0:7201/api/v1/query_range?query=%s"+ + "&start=%d&end=%d&step=100", name, ts/100*100, (ts/100+1)*100) + + for _, url := range []string{instant, rnge} { + singleClusterDefaultStrip(url) + bothClusterCustomStrip(url) + bothClusterDefaultStrip(url) + bothClusterNoStrip(url) + bothClusterMultiStrip(url) + } +} + +func queryWithHeader(url string, h string) (prometheus.Response, error) { + var result prometheus.Response + req, err := http.NewRequest("GET", url, nil) + if err != nil { + return result, err + } + + req.Header.Add(handler.RestrictByTagsJSONHeader, h) + client := http.DefaultClient + resp, err := client.Do(req) + if err != nil { + return result, err + } + + if resp.StatusCode != http.StatusOK { + return result, fmt.Errorf("response failed with code %s", resp.Status) + } + + defer resp.Body.Close() + data, err := ioutil.ReadAll(resp.Body) + if err != nil { + return result, err + } + + json.Unmarshal(data, &result) + return result, err +} + +func mustMatcher(t models.MatchType, n string, v string) models.Matcher { + m, err := models.NewMatcher(models.MatchEqual, []byte("val"), []byte("1")) + if err != nil { + panic(err) + } + + return m +} + +type tester struct{} + +// Ensure tester is a TestingT and set a global `t`. +var t require.TestingT = &tester{} + +// name is global and set on startup. +var name string + +func (t *tester) Errorf(format string, args ...interface{}) { + _, fn, line, _ := runtime.Caller(4) + args[2] = fmt.Sprintf(" at %s:%d:\n%v", fn, line, args[2]) + fmt.Printf(format, args...) +} + +func (t *tester) FailNow() { + os.Exit(1) +} + +func mustParseOpts(o handler.StringTagOptions) string { + m, err := json.Marshal(o) + require.NoError(t, err, "cannot marshal to json") + return string(m) +} + +func bothClusterDefaultStrip(url string) { + m := mustParseOpts(handler.StringTagOptions{ + Restrict: []handler.StringMatch{ + handler.StringMatch{Name: "val", Type: "EQUAL", Value: "1"}, + }, + }) + + resp, err := queryWithHeader(url, string(m)) + require.NoError(t, err, "failed to query") + + data := resp.Data.Result + data.Sort() + require.Equal(t, len(data), 2) + clusters := []string{"coordinator-cluster-a", "coordinator-cluster-b"} + for i, d := range data { + require.Equal(t, 2, len(d.Metric)) + require.Equal(t, name, d.Metric["__name__"]) + require.Equal(t, clusters[i], d.Metric["cluster"]) + } +} + +func bothClusterCustomStrip(url string) { + m := mustParseOpts(handler.StringTagOptions{ + Restrict: []handler.StringMatch{ + handler.StringMatch{Name: "val", Type: "EQUAL", Value: "1"}, + }, + Strip: []string{"__name__"}, + }) + + resp, err := queryWithHeader(url, string(m)) + require.NoError(t, err, "failed to query") + + data := resp.Data.Result + data.Sort() + require.Equal(t, len(data), 2) + clusters := []string{"coordinator-cluster-a", "coordinator-cluster-b"} + for i, d := range data { + require.Equal(t, 2, len(d.Metric)) + require.Equal(t, clusters[i], d.Metric["cluster"]) + require.Equal(t, "1", d.Metric["val"]) + } +} + +func bothClusterNoStrip(url string) { + m := mustParseOpts(handler.StringTagOptions{ + Restrict: []handler.StringMatch{ + handler.StringMatch{Name: "val", Type: "EQUAL", Value: "1"}, + }, + Strip: []string{}, + }) + + resp, err := queryWithHeader(url, string(m)) + require.NoError(t, err, "failed to query") + + data := resp.Data.Result + data.Sort() + require.Equal(t, len(data), 2) + clusters := []string{"coordinator-cluster-a", "coordinator-cluster-b"} + for i, d := range data { + require.Equal(t, 3, len(d.Metric)) + require.Equal(t, name, d.Metric["__name__"]) + require.Equal(t, clusters[i], d.Metric["cluster"]) + require.Equal(t, "1", d.Metric["val"]) + } +} + +func bothClusterMultiStrip(url string) { + m := mustParseOpts(handler.StringTagOptions{ + Restrict: []handler.StringMatch{ + handler.StringMatch{Name: "val", Type: "EQUAL", Value: "1"}, + }, + Strip: []string{"val", "__name__"}, + }) + + resp, err := queryWithHeader(url, string(m)) + require.NoError(t, err, "failed to query") + + data := resp.Data.Result + data.Sort() + require.Equal(t, len(data), 2) + clusters := []string{"coordinator-cluster-a", "coordinator-cluster-b"} + for i, d := range data { + require.Equal(t, 1, len(d.Metric)) + require.Equal(t, clusters[i], d.Metric["cluster"]) + } +} + +// NB: cluster 1 is expected to have metrics with vals in range: [1,5] +// and cluster 2 is expected to have metrics with vals in range: [1,10] +// so setting the value to be in (5..10] should hit only a single metric. +func singleClusterDefaultStrip(url string) { + m := mustParseOpts(handler.StringTagOptions{ + Restrict: []handler.StringMatch{ + handler.StringMatch{Name: "val", Type: "EQUAL", Value: "9"}, + }, + }) + + resp, err := queryWithHeader(url, string(m)) + require.NoError(t, err, "failed to query") + + data := resp.Data.Result + require.Equal(t, len(data), 1, url) + require.Equal(t, 2, len(data[0].Metric)) + require.Equal(t, name, data[0].Metric["__name__"], "single") + require.Equal(t, "coordinator-cluster-b", data[0].Metric["cluster"]) +} diff --git a/scripts/docker-integration-tests/query_fanout/restrict.sh b/scripts/docker-integration-tests/query_fanout/restrict.sh new file mode 100755 index 0000000000..e648c572c7 --- /dev/null +++ b/scripts/docker-integration-tests/query_fanout/restrict.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +set -ex +TEST_PATH=$GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests +FANOUT_PATH=$TEST_PATH/query_fanout +source $TEST_PATH/common.sh +source $FANOUT_PATH/warning.sh + +function test_restrictions { + t=$(date +%s) + METRIC_NAME="foo_$t" + # # write 5 metrics to cluster a + write_metrics coordinator-cluster-a 5 + # write 10 metrics to cluster b + write_metrics coordinator-cluster-b 10 + + # unlimited query against cluster a has no header + ATTEMPTS=3 TIMEOUT=1 retry_with_backoff go run $FANOUT_PATH/restrict.go -t $t +} diff --git a/scripts/docker-integration-tests/query_fanout/test.sh b/scripts/docker-integration-tests/query_fanout/test.sh index 5d1f3ef905..5a4321e2dc 100755 --- a/scripts/docker-integration-tests/query_fanout/test.sh +++ b/scripts/docker-integration-tests/query_fanout/test.sh @@ -2,8 +2,11 @@ set -xe -source $GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/common.sh -source $GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/query_fanout/warning.sh +TEST_PATH=$GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests +FANOUT_PATH=$TEST_PATH/query_fanout +source $TEST_PATH/common.sh +source $FANOUT_PATH/warning.sh +source $FANOUT_PATH/restrict.sh REVISION=$(git rev-parse HEAD) COMPOSE_FILE=$GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/query_fanout/docker-compose.yml @@ -216,3 +219,6 @@ ATTEMPTS=5 TIMEOUT=1 retry_with_backoff complete_tags echo "running fanout warning tests" test_fanout_warnings + +echo "running restrict tests" +test_restrictions diff --git a/scripts/docker-integration-tests/query_fanout/warning.sh b/scripts/docker-integration-tests/query_fanout/warning.sh index c2ed7dc358..dc12a5a398 100755 --- a/scripts/docker-integration-tests/query_fanout/warning.sh +++ b/scripts/docker-integration-tests/query_fanout/warning.sh @@ -337,6 +337,8 @@ function test_fanout_warning_missing_zone { ATTEMPTS=3 TIMEOUT=1 retry_with_backoff find_carbon 16 remote_store_cluster-c_complete_tags_warning ATTEMPTS=3 TIMEOUT=1 retry_with_backoff find_carbon 9 max_fetch_series_limit_applied,remote_store_cluster-c_complete_tags_warning + + docker-compose -f ${COMPOSE_FILE} start coordinator-cluster-c } function test_fanout_warnings { diff --git a/src/query/api/v1/handler/fetch_options.go b/src/query/api/v1/handler/fetch_options.go index 2248381320..d36ab9df01 100644 --- a/src/query/api/v1/handler/fetch_options.go +++ b/src/query/api/v1/handler/fetch_options.go @@ -21,6 +21,7 @@ package handler import ( + "encoding/json" "fmt" "math" "net/http" @@ -103,6 +104,7 @@ func (b fetchOptionsBuilder) NewFetchOptions( if err != nil { return nil, xhttp.NewParseError(err, http.StatusBadRequest) } + fetchOpts.Limit = limit if str := req.Header.Get(MetricsTypeHeader); str != "" { mt, err := storage.ParseMetricsType(str) @@ -111,9 +113,13 @@ func (b fetchOptionsBuilder) NewFetchOptions( "could not parse metrics type: input=%s, err=%v", str, err) return nil, xhttp.NewParseError(err, http.StatusBadRequest) } - fetchOpts.RestrictFetchOptions = newOrExistingRestrictFetchOptions(fetchOpts) - fetchOpts.RestrictFetchOptions.MetricsType = mt + + fetchOpts.RestrictQueryOptions = newOrExistingRestrictQueryOptions(fetchOpts) + fetchOpts.RestrictQueryOptions.RestrictByType = + newOrExistingRestrictQueryOptionsRestrictByType(fetchOpts) + fetchOpts.RestrictQueryOptions.RestrictByType.MetricsType = mt } + if str := req.Header.Get(MetricsStoragePolicyHeader); str != "" { sp, err := policy.ParseStoragePolicy(str) if err != nil { @@ -121,10 +127,29 @@ func (b fetchOptionsBuilder) NewFetchOptions( "could not parse storage policy: input=%s, err=%v", str, err) return nil, xhttp.NewParseError(err, http.StatusBadRequest) } - fetchOpts.RestrictFetchOptions = newOrExistingRestrictFetchOptions(fetchOpts) - fetchOpts.RestrictFetchOptions.StoragePolicy = sp + + fetchOpts.RestrictQueryOptions = newOrExistingRestrictQueryOptions(fetchOpts) + fetchOpts.RestrictQueryOptions.RestrictByType = + newOrExistingRestrictQueryOptionsRestrictByType(fetchOpts) + fetchOpts.RestrictQueryOptions.RestrictByType.StoragePolicy = sp } - if restrict := fetchOpts.RestrictFetchOptions; restrict != nil { + + if str := req.Header.Get(RestrictByTagsJSONHeader); str != "" { + var opts StringTagOptions + if err := json.Unmarshal([]byte(str), &opts); err != nil { + return nil, xhttp.NewParseError(err, http.StatusBadRequest) + } + + tagOpts, err := opts.toOptions() + if err != nil { + return nil, xhttp.NewParseError(err, http.StatusBadRequest) + } + + fetchOpts.RestrictQueryOptions = newOrExistingRestrictQueryOptions(fetchOpts) + fetchOpts.RestrictQueryOptions.RestrictByTag = tagOpts + } + + if restrict := fetchOpts.RestrictQueryOptions; restrict != nil { if err := restrict.Validate(); err != nil { err = fmt.Errorf( "could not validate restrict options: err=%v", err) @@ -152,13 +177,22 @@ func (b fetchOptionsBuilder) NewFetchOptions( return fetchOpts, nil } -func newOrExistingRestrictFetchOptions( +func newOrExistingRestrictQueryOptions( + fetchOpts *storage.FetchOptions, +) *storage.RestrictQueryOptions { + if v := fetchOpts.RestrictQueryOptions; v != nil { + return v + } + return &storage.RestrictQueryOptions{} +} + +func newOrExistingRestrictQueryOptionsRestrictByType( fetchOpts *storage.FetchOptions, -) *storage.RestrictFetchOptions { - if v := fetchOpts.RestrictFetchOptions; v != nil { +) *storage.RestrictByType { + if v := fetchOpts.RestrictQueryOptions.RestrictByType; v != nil { return v } - return &storage.RestrictFetchOptions{} + return &storage.RestrictByType{} } // ParseStep parses the step duration for an HTTP request. diff --git a/src/query/api/v1/handler/fetch_options_test.go b/src/query/api/v1/handler/fetch_options_test.go index d3acd770ae..175c284782 100644 --- a/src/query/api/v1/handler/fetch_options_test.go +++ b/src/query/api/v1/handler/fetch_options_test.go @@ -30,6 +30,7 @@ import ( "time" "github.com/m3db/m3/src/metrics/policy" + "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/storage" "github.com/stretchr/testify/assert" @@ -47,7 +48,7 @@ func TestFetchOptionsBuilder(t *testing.T) { headers map[string]string query string expectedLimit int - expectedRestrict *storage.RestrictFetchOptions + expectedRestrict *storage.RestrictQueryOptions expectedLookback *expectedLookback expectedErr bool }{ @@ -78,8 +79,10 @@ func TestFetchOptionsBuilder(t *testing.T) { headers: map[string]string{ MetricsTypeHeader: storage.UnaggregatedMetricsType.String(), }, - expectedRestrict: &storage.RestrictFetchOptions{ - MetricsType: storage.UnaggregatedMetricsType, + expectedRestrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnaggregatedMetricsType, + }, }, }, { @@ -88,9 +91,11 @@ func TestFetchOptionsBuilder(t *testing.T) { MetricsTypeHeader: storage.AggregatedMetricsType.String(), MetricsStoragePolicyHeader: "1m:14d", }, - expectedRestrict: &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + expectedRestrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + }, }, }, { @@ -167,10 +172,10 @@ func TestFetchOptionsBuilder(t *testing.T) { require.NoError(t, err) require.Equal(t, test.expectedLimit, opts.Limit) if test.expectedRestrict == nil { - require.Nil(t, opts.RestrictFetchOptions) + require.Nil(t, opts.RestrictQueryOptions) } else { - require.NotNil(t, opts.RestrictFetchOptions) - require.Equal(t, *test.expectedRestrict, *opts.RestrictFetchOptions) + require.NotNil(t, opts.RestrictQueryOptions) + require.Equal(t, *test.expectedRestrict, *opts.RestrictQueryOptions) } if test.expectedLookback == nil { require.Nil(t, opts.LookbackDuration) @@ -269,3 +274,54 @@ func TestParseDurationOverflowError(t *testing.T) { _, err = ParseDuration(r, StepParam) assert.Error(t, err) } + +func TestFetchOptionsWithHeader(t *testing.T) { + type expectedLookback struct { + value time.Duration + } + + headers := map[string]string{ + MetricsTypeHeader: storage.AggregatedMetricsType.String(), + MetricsStoragePolicyHeader: "1m:14d", + RestrictByTagsJSONHeader: `{ + "match":[ + {"name":"a", "value":"b", "type":"EQUAL"}, + {"name":"c", "value":"d", "type":"NOTEQUAL"}, + {"name":"e", "value":"f", "type":"REGEXP"}, + {"name":"g", "value":"h", "type":"NOTREGEXP"}, + {"name":"i", "value":"j", "type":"EXISTS"}, + {"name":"k", "value":"l", "type":"NOTEXISTS"} + ], + "strip":["foo"] + }`, + } + + builder := NewFetchOptionsBuilder(FetchOptionsBuilderOptions{Limit: 5}) + req := httptest.NewRequest("GET", "/", nil) + for k, v := range headers { + req.Header.Add(k, v) + } + + opts, err := builder.NewFetchOptions(req) + require.NoError(t, err) + require.NotNil(t, opts.RestrictQueryOptions) + ex := &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + }, + RestrictByTag: &storage.RestrictByTag{ + Restrict: models.Matchers{ + mustMatcher("a", "b", models.MatchEqual), + mustMatcher("c", "d", models.MatchNotEqual), + mustMatcher("e", "f", models.MatchRegexp), + mustMatcher("g", "h", models.MatchNotRegexp), + mustMatcher("i", "j", models.MatchField), + mustMatcher("k", "l", models.MatchNotField), + }, + Strip: toStrip("foo"), + }, + } + + require.Equal(t, ex, opts.RestrictQueryOptions) +} diff --git a/src/query/api/v1/handler/headers.go b/src/query/api/v1/handler/headers.go index 7af42d0508..d0829e23f0 100644 --- a/src/query/api/v1/handler/headers.go +++ b/src/query/api/v1/handler/headers.go @@ -56,6 +56,10 @@ const ( // metrics type. MetricsStoragePolicyHeader = "M3-Storage-Policy" + // RestrictByTagsJSONHeader provides tag options to enforces on queries, + // in JSON format. See `handler.stringTagOptions` for definitions.` + RestrictByTagsJSONHeader = "M3-Restrict-By-Tags-JSON" + // UnaggregatedStoragePolicy specifies the unaggregated storage policy. UnaggregatedStoragePolicy = "unaggregated" diff --git a/src/query/api/v1/handler/prometheus/common.go b/src/query/api/v1/handler/prometheus/common.go index 59b17d3ba8..a93a33c065 100644 --- a/src/query/api/v1/handler/prometheus/common.go +++ b/src/query/api/v1/handler/prometheus/common.go @@ -36,6 +36,7 @@ import ( "github.com/m3db/m3/src/query/models" xpromql "github.com/m3db/m3/src/query/parser/promql" "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/query/ts" "github.com/m3db/m3/src/query/util" "github.com/m3db/m3/src/query/util/json" xhttp "github.com/m3db/m3/src/x/net/http" @@ -451,9 +452,10 @@ func (r results) Less(i, j int) bool { // Swap swaps the elements with indexes i and j. func (r results) Swap(i, j int) { r[i], r[j] = r[j], r[i] } -func (r results) sort() { - for _, result := range r { - result.genID() +// Sort sorts the results. +func (r results) Sort() { + for i, result := range r { + r[i] = result.genID() } sort.Sort(r) @@ -477,18 +479,22 @@ type Values []Value // Value is a single value for Prometheus result. type Value []interface{} -func (r *Result) genID() { +func (r *Result) genID() Result { + tags := make(sort.StringSlice, len(r.Metric)) + for k, v := range r.Metric { + tags = append(tags, fmt.Sprintf("%s:%s,", k, v)) + } + + sort.Sort(tags) var sb strings.Builder // NB: this may clash but exact tag values are also checked, and this is a // validation endpoint so there's less concern over correctness. - for k, v := range r.Metric { - sb.WriteString(k) - sb.WriteString(`:"`) - sb.WriteString(v) - sb.WriteString(`",`) + for _, t := range tags { + sb.WriteString(t) } r.id = sb.String() + return *r } // MatchInformation describes how well two responses match. @@ -533,8 +539,8 @@ func (r results) matches(other results) (MatchInformation, error) { }, err } - r.sort() - other.sort() + r.Sort() + other.Sort() for i, result := range r { if err := result.matches(other[i]); err != nil { return MatchInformation{ @@ -632,3 +638,22 @@ type PromDebug struct { Input Response `json:"input"` Results Response `json:"results"` } + +// FilterSeriesByOptions removes series tags based on options. +func FilterSeriesByOptions( + series []*ts.Series, + opts *storage.FetchOptions, +) []*ts.Series { + if opts == nil { + return series + } + + keys := opts.RestrictQueryOptions.GetRestrictByTag().GetFilterByNames() + if len(keys) > 0 { + for i, s := range series { + series[i].Tags = s.Tags.TagsWithoutKeys(keys) + } + } + + return series +} diff --git a/src/query/api/v1/handler/prometheus/native/read.go b/src/query/api/v1/handler/prometheus/native/read.go index a4aa942a65..aabd3b78d8 100644 --- a/src/query/api/v1/handler/prometheus/native/read.go +++ b/src/query/api/v1/handler/prometheus/native/read.go @@ -146,7 +146,9 @@ func (h *PromReadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { QueryContextOptions: models.QueryContextOptions{ LimitMaxTimeseries: fetchOpts.Limit, }} - if restrictOpts := fetchOpts.RestrictFetchOptions; restrictOpts != nil { + + restrictOpts := fetchOpts.RestrictQueryOptions.GetRestrictByType() + if restrictOpts != nil { restrict := &models.RestrictFetchTypeQueryContextOptions{ MetricsType: uint(restrictOpts.MetricsType), StoragePolicy: restrictOpts.StoragePolicy, diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index b58ab51861..3638f658f8 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -28,6 +28,7 @@ import ( "sort" "github.com/m3db/m3/src/query/api/v1/handler" + "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/executor" "github.com/m3db/m3/src/query/models" @@ -150,6 +151,7 @@ func read( return emptyResult, err } + series = prometheus.FilterSeriesByOptions(series, fetchOpts) return readResult{ series: series, meta: meta, diff --git a/src/query/api/v1/handler/prometheus/native/read_instantaneous.go b/src/query/api/v1/handler/prometheus/native/read_instantaneous.go index a8a13094f8..154b249256 100644 --- a/src/query/api/v1/handler/prometheus/native/read_instantaneous.go +++ b/src/query/api/v1/handler/prometheus/native/read_instantaneous.go @@ -96,7 +96,9 @@ func (h *PromReadInstantHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques QueryContextOptions: models.QueryContextOptions{ LimitMaxTimeseries: fetchOpts.Limit, }} - if restrictOpts := fetchOpts.RestrictFetchOptions; restrictOpts != nil { + + restrictOpts := fetchOpts.RestrictQueryOptions.GetRestrictByType() + if restrictOpts != nil { restrict := &models.RestrictFetchTypeQueryContextOptions{ MetricsType: uint(restrictOpts.MetricsType), StoragePolicy: restrictOpts.StoragePolicy, diff --git a/src/query/api/v1/handler/prometheus/remote/read.go b/src/query/api/v1/handler/prometheus/remote/read.go index a4280bd4ff..b8121c9fd9 100644 --- a/src/query/api/v1/handler/prometheus/remote/read.go +++ b/src/query/api/v1/handler/prometheus/remote/read.go @@ -232,6 +232,10 @@ func (h *PromReadHandler) read( mu.Lock() meta = meta.CombineMetadata(result.Metadata) mu.Unlock() + result.SeriesList = prometheus.FilterSeriesByOptions( + result.SeriesList, + fetchOpts, + ) promRes := storage.FetchResultToPromResult(result, h.keepEmpty) promResults[i] = promRes }() diff --git a/src/query/api/v1/handler/prometheus/validator/handler_test.go b/src/query/api/v1/handler/prometheus/validator/handler_test.go index 4c5254dc72..13e6494df6 100644 --- a/src/query/api/v1/handler/prometheus/validator/handler_test.go +++ b/src/query/api/v1/handler/prometheus/validator/handler_test.go @@ -22,7 +22,6 @@ package validator import ( "encoding/json" - "fmt" "io" "net/http" "net/http/httptest" @@ -335,7 +334,6 @@ func TestValidateEndpoint(t *testing.T) { recorder := httptest.NewRecorder() debugHandler.ServeHTTP(recorder, req) - fmt.Println(recorder.Body.String()) var mismatches MismatchesJSON require.NoError(t, json.Unmarshal(recorder.Body.Bytes(), &mismatches)) assert.False(t, mismatches.Correct) diff --git a/src/query/api/v1/handler/tag_options.go b/src/query/api/v1/handler/tag_options.go new file mode 100644 index 0000000000..664376aeb7 --- /dev/null +++ b/src/query/api/v1/handler/tag_options.go @@ -0,0 +1,113 @@ +// Copyright (c) 2019 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 handler + +import ( + "errors" + "fmt" + + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/storage" +) + +func matchByName(name string) (models.MatchType, error) { + t := models.MatchEqual + switch name { + case "EQUAL": + // noop + case "NOTEQUAL": + t = models.MatchNotEqual + case "REGEXP": + t = models.MatchRegexp + case "NOTREGEXP": + t = models.MatchNotRegexp + case "EXISTS": + t = models.MatchField + case "NOTEXISTS": + t = models.MatchNotField + case "ALL": + return t, errors.New("ALL type not supported as a tag matcher restriction") + default: + return t, fmt.Errorf("matcher type %s not recognized", name) + } + + return t, nil +} + +func (m StringMatch) toMatcher() (models.Matcher, error) { + t, err := matchByName(m.Type) + if err != nil { + return models.Matcher{}, err + } + + return models.NewMatcher(t, []byte(m.Name), []byte(m.Value)) +} + +func (o *StringTagOptions) toOptions() (*storage.RestrictByTag, error) { + if len(o.Restrict) == 0 && len(o.Strip) == 0 { + return nil, nil + } + + opts := &storage.RestrictByTag{} + if len(o.Restrict) > 0 { + opts.Restrict = make(models.Matchers, 0, len(o.Restrict)) + for _, m := range o.Restrict { + r, err := m.toMatcher() + if err != nil { + return nil, err + } + + opts.Restrict = append(opts.Restrict, r) + } + } + + if o.Strip != nil { + opts.Strip = make([][]byte, 0, len(o.Strip)) + for _, s := range o.Strip { + opts.Strip = append(opts.Strip, []byte(s)) + } + } else { + // If strip not explicitly set, strip tag names from + // the restricted matchers. + opts.Strip = make([][]byte, 0, len(opts.Restrict)) + for _, s := range opts.Restrict { + opts.Strip = append(opts.Strip, s.Name) + } + } + + return opts, nil +} + +// StringMatch is an easy to use JSON representation of models.Matcher that +// allows plaintext fields rather than forcing base64 encoded values. +type StringMatch struct { + Name string `json:"name"` + Type string `json:"type"` + Value string `json:"value"` +} + +// StringTagOptions is an easy to use JSON representation of +// storage.RestrictByTag that allows plaintext string fields rather than +// forcing base64 encoded values. +type StringTagOptions struct { + Restrict []StringMatch `json:"match"` + Strip []string `json:"strip"` +} diff --git a/src/query/api/v1/handler/tag_options_test.go b/src/query/api/v1/handler/tag_options_test.go new file mode 100644 index 0000000000..71204704a4 --- /dev/null +++ b/src/query/api/v1/handler/tag_options_test.go @@ -0,0 +1,144 @@ +// Copyright (c) 2019 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 handler + +import ( + "encoding/json" + "testing" + + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/storage" + + "github.com/stretchr/testify/require" +) + +func toStrip(strs ...string) [][]byte { + b := make([][]byte, 0, len(strs)) + for _, s := range strs { + b = append(b, []byte(s)) + } + + return b +} + +func mustMatcher(n string, v string, t models.MatchType) models.Matcher { + m, err := models.NewMatcher(t, []byte(n), []byte(v)) + if err != nil { + panic(err) + } + + return m +} + +func TestParse(t *testing.T) { + tests := []struct { + json string + expected *storage.RestrictByTag + expectedError bool + }{ + {"{}", nil, false}, + { + `{ + "match":[ + {"name":"a", "value":"b", "type":"EQUAL"}, + {"name":"c", "value":"d", "type":"NOTEQUAL"}, + {"name":"e", "value":"f", "type":"REGEXP"}, + {"name":"g", "value":"h", "type":"NOTREGEXP"}, + {"name":"i", "type":"EXISTS"}, + {"name":"j", "type":"NOTEXISTS"} + ] + }`, + &storage.RestrictByTag{ + Restrict: models.Matchers{ + mustMatcher("a", "b", models.MatchEqual), + mustMatcher("c", "d", models.MatchNotEqual), + mustMatcher("e", "f", models.MatchRegexp), + mustMatcher("g", "h", models.MatchNotRegexp), + mustMatcher("i", "", models.MatchField), + mustMatcher("j", "", models.MatchNotField), + }, + Strip: toStrip("a", "c", "e", "g", "i", "j"), + }, + false, + }, + { + `{ + "match":[ + {"name":"a", "value":"b", "type":"EQUAL"}, + {"name":"c", "value":"d", "type":"NOTEQUAL"}, + {"name":"e", "value":"f", "type":"REGEXP"}, + {"name":"g", "value":"h", "type":"NOTREGEXP"}, + {"name":"i", "value":"j", "type":"EXISTS"}, + {"name":"k", "value":"l", "type":"NOTEXISTS"} + ], + "strip":["foo"] + }`, + &storage.RestrictByTag{ + Restrict: models.Matchers{ + mustMatcher("a", "b", models.MatchEqual), + mustMatcher("c", "d", models.MatchNotEqual), + mustMatcher("e", "f", models.MatchRegexp), + mustMatcher("g", "h", models.MatchNotRegexp), + mustMatcher("i", "j", models.MatchField), + mustMatcher("k", "l", models.MatchNotField), + }, + Strip: toStrip("foo"), + }, + false, + }, + { + `{"match":[{"name":"i", "value":"j", "type":"EXISTS"}],"strip":[]}`, + &storage.RestrictByTag{ + Restrict: models.Matchers{ + mustMatcher("i", "j", models.MatchField), + }, + Strip: [][]byte{}, + }, + false, + }, + { + `{"strip":["foo"]}`, + &storage.RestrictByTag{ + Strip: toStrip("foo"), + }, + false, + }, + {`{"match":[{}]}`, nil, true}, + {`{"match":[{"type":"ALL"}]}`, nil, true}, + {`{"match":[{"type":"invalid"}]}`, nil, true}, + {`{"match":[{"name":"a","type":"EQUAL"}]}`, nil, true}, + } + + for _, tt := range tests { + var opts StringTagOptions + err := json.Unmarshal([]byte(tt.json), &opts) + require.NoError(t, err) + + a, err := opts.toOptions() + if tt.expectedError { + require.Error(t, err) + require.Nil(t, a) + } else { + require.NoError(t, err) + require.Equal(t, tt.expected, a) + } + } +} diff --git a/src/query/api/v1/handler/types.go b/src/query/api/v1/handler/types.go index c1dab5d06a..54a6a605ae 100644 --- a/src/query/api/v1/handler/types.go +++ b/src/query/api/v1/handler/types.go @@ -20,16 +20,16 @@ package handler -// HeaderKeyType is the type for the header key +// HeaderKeyType is the type for the header key. type HeaderKeyType int const ( - // HeaderKey is the key which headers will be added to in the request context + // HeaderKey is the key which headers will be added to in the request context. HeaderKey HeaderKeyType = iota - // RoutePrefixV1 is the v1 prefix for all coordinator routes + // RoutePrefixV1 is the v1 prefix for all coordinator routes. RoutePrefixV1 = "/api/v1" - // RoutePrefixExperimenta is the experimental prefix for all coordinator routes + // RoutePrefixExperimental is the experimental prefix for all coordinator routes. RoutePrefixExperimental = "/api/experimental" ) diff --git a/src/query/executor/engine.go b/src/query/executor/engine.go index 079638bbba..d4e1e1c0c1 100644 --- a/src/query/executor/engine.go +++ b/src/query/executor/engine.go @@ -115,7 +115,8 @@ func (e *engine) Execute( opts *QueryOptions, fetchOpts *storage.FetchOptions, ) (*storage.FetchResult, error) { - return e.opts.Store().Fetch(ctx, query, fetchOpts) + result, err := e.opts.Store().Fetch(ctx, query, fetchOpts) + return result, err } func (e *engine) ExecuteExpr( diff --git a/src/query/functions/fetch_test.go b/src/query/functions/fetch_test.go index e1832950bd..e7783aa74d 100644 --- a/src/query/functions/fetch_test.go +++ b/src/query/functions/fetch_test.go @@ -146,7 +146,8 @@ func TestFetchWithRestrictFetch(t *testing.T) { assert.Equal(t, expected, sink.Values) fetchOpts := mockStorage.LastFetchOptions() - require.NotNil(t, fetchOpts.RestrictFetchOptions) - assert.Equal(t, storage.AggregatedMetricsType, storage.MetricsType(fetchOpts.RestrictFetchOptions.MetricsType)) - assert.Equal(t, "10s:42d", fetchOpts.RestrictFetchOptions.StoragePolicy.String()) + restrictByType := fetchOpts.RestrictQueryOptions.GetRestrictByType() + require.NotNil(t, restrictByType) + assert.Equal(t, storage.AggregatedMetricsType, storage.MetricsType(restrictByType.MetricsType)) + assert.Equal(t, "10s:42d", restrictByType.StoragePolicy.String()) } diff --git a/src/query/generated/proto/rpcpb/query.pb.go b/src/query/generated/proto/rpcpb/query.pb.go index cac72df3d1..ae904e0ef8 100644 --- a/src/query/generated/proto/rpcpb/query.pb.go +++ b/src/query/generated/proto/rpcpb/query.pb.go @@ -34,7 +34,9 @@ TagMatchers TagMatcher FetchOptions - RestrictFetchOptions + RestrictQueryOptions + RestrictFetchType + RestrictFetchTags FetchResponse Series SeriesMetadata @@ -394,7 +396,7 @@ func (m *TagMatcher) GetType() MatcherType { type FetchOptions struct { Limit int64 `protobuf:"varint,1,opt,name=limit,proto3" json:"limit,omitempty"` - Restrict *RestrictFetchOptions `protobuf:"bytes,2,opt,name=restrict" json:"restrict,omitempty"` + Restrict *RestrictQueryOptions `protobuf:"bytes,2,opt,name=restrict" json:"restrict,omitempty"` LookbackDuration int64 `protobuf:"varint,3,opt,name=lookbackDuration,proto3" json:"lookbackDuration,omitempty"` Unaggregated FanoutOption `protobuf:"varint,4,opt,name=unaggregated,proto3,enum=rpc.FanoutOption" json:"unaggregated,omitempty"` Aggregated FanoutOption `protobuf:"varint,5,opt,name=aggregated,proto3,enum=rpc.FanoutOption" json:"aggregated,omitempty"` @@ -414,7 +416,7 @@ func (m *FetchOptions) GetLimit() int64 { return 0 } -func (m *FetchOptions) GetRestrict() *RestrictFetchOptions { +func (m *FetchOptions) GetRestrict() *RestrictQueryOptions { if m != nil { return m.Restrict } @@ -456,30 +458,78 @@ func (m *FetchOptions) GetIncludeResolution() bool { return false } -type RestrictFetchOptions struct { +type RestrictQueryOptions struct { + RestrictFetchType *RestrictFetchType `protobuf:"bytes,3,opt,name=RestrictFetchType" json:"RestrictFetchType,omitempty"` + RestrictFetchTags *RestrictFetchTags `protobuf:"bytes,4,opt,name=RestrictFetchTags" json:"RestrictFetchTags,omitempty"` +} + +func (m *RestrictQueryOptions) Reset() { *m = RestrictQueryOptions{} } +func (m *RestrictQueryOptions) String() string { return proto.CompactTextString(m) } +func (*RestrictQueryOptions) ProtoMessage() {} +func (*RestrictQueryOptions) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{6} } + +func (m *RestrictQueryOptions) GetRestrictFetchType() *RestrictFetchType { + if m != nil { + return m.RestrictFetchType + } + return nil +} + +func (m *RestrictQueryOptions) GetRestrictFetchTags() *RestrictFetchTags { + if m != nil { + return m.RestrictFetchTags + } + return nil +} + +type RestrictFetchType struct { MetricsType MetricsType `protobuf:"varint,1,opt,name=metricsType,proto3,enum=rpc.MetricsType" json:"metricsType,omitempty"` MetricsStoragePolicy *policypb.StoragePolicy `protobuf:"bytes,2,opt,name=metricsStoragePolicy" json:"metricsStoragePolicy,omitempty"` } -func (m *RestrictFetchOptions) Reset() { *m = RestrictFetchOptions{} } -func (m *RestrictFetchOptions) String() string { return proto.CompactTextString(m) } -func (*RestrictFetchOptions) ProtoMessage() {} -func (*RestrictFetchOptions) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{6} } +func (m *RestrictFetchType) Reset() { *m = RestrictFetchType{} } +func (m *RestrictFetchType) String() string { return proto.CompactTextString(m) } +func (*RestrictFetchType) ProtoMessage() {} +func (*RestrictFetchType) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{7} } -func (m *RestrictFetchOptions) GetMetricsType() MetricsType { +func (m *RestrictFetchType) GetMetricsType() MetricsType { if m != nil { return m.MetricsType } return MetricsType_UNKNOWN_METRICS_TYPE } -func (m *RestrictFetchOptions) GetMetricsStoragePolicy() *policypb.StoragePolicy { +func (m *RestrictFetchType) GetMetricsStoragePolicy() *policypb.StoragePolicy { if m != nil { return m.MetricsStoragePolicy } return nil } +type RestrictFetchTags struct { + Restrict *TagMatchers `protobuf:"bytes,1,opt,name=restrict" json:"restrict,omitempty"` + Strip [][]byte `protobuf:"bytes,2,rep,name=strip" json:"strip,omitempty"` +} + +func (m *RestrictFetchTags) Reset() { *m = RestrictFetchTags{} } +func (m *RestrictFetchTags) String() string { return proto.CompactTextString(m) } +func (*RestrictFetchTags) ProtoMessage() {} +func (*RestrictFetchTags) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{8} } + +func (m *RestrictFetchTags) GetRestrict() *TagMatchers { + if m != nil { + return m.Restrict + } + return nil +} + +func (m *RestrictFetchTags) GetStrip() [][]byte { + if m != nil { + return m.Strip + } + return nil +} + type FetchResponse struct { Series []*Series `protobuf:"bytes,1,rep,name=series" json:"series,omitempty"` Meta *ResultMetadata `protobuf:"bytes,2,opt,name=meta" json:"meta,omitempty"` @@ -488,7 +538,7 @@ type FetchResponse struct { func (m *FetchResponse) Reset() { *m = FetchResponse{} } func (m *FetchResponse) String() string { return proto.CompactTextString(m) } func (*FetchResponse) ProtoMessage() {} -func (*FetchResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{7} } +func (*FetchResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{9} } func (m *FetchResponse) GetSeries() []*Series { if m != nil { @@ -515,7 +565,7 @@ type Series struct { func (m *Series) Reset() { *m = Series{} } func (m *Series) String() string { return proto.CompactTextString(m) } func (*Series) ProtoMessage() {} -func (*Series) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{8} } +func (*Series) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{10} } type isSeries_Value interface { isSeries_Value() @@ -644,7 +694,7 @@ type SeriesMetadata struct { func (m *SeriesMetadata) Reset() { *m = SeriesMetadata{} } func (m *SeriesMetadata) String() string { return proto.CompactTextString(m) } func (*SeriesMetadata) ProtoMessage() {} -func (*SeriesMetadata) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{9} } +func (*SeriesMetadata) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{11} } func (m *SeriesMetadata) GetId() []byte { if m != nil { @@ -675,7 +725,7 @@ type DecompressedSeries struct { func (m *DecompressedSeries) Reset() { *m = DecompressedSeries{} } func (m *DecompressedSeries) String() string { return proto.CompactTextString(m) } func (*DecompressedSeries) ProtoMessage() {} -func (*DecompressedSeries) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{10} } +func (*DecompressedSeries) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{12} } func (m *DecompressedSeries) GetDatapoints() []*Datapoint { if m != nil { @@ -699,7 +749,7 @@ type Datapoint struct { func (m *Datapoint) Reset() { *m = Datapoint{} } func (m *Datapoint) String() string { return proto.CompactTextString(m) } func (*Datapoint) ProtoMessage() {} -func (*Datapoint) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{11} } +func (*Datapoint) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{13} } func (m *Datapoint) GetTimestamp() int64 { if m != nil { @@ -723,7 +773,7 @@ type Tag struct { func (m *Tag) Reset() { *m = Tag{} } func (m *Tag) String() string { return proto.CompactTextString(m) } func (*Tag) ProtoMessage() {} -func (*Tag) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{12} } +func (*Tag) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{14} } func (m *Tag) GetName() []byte { if m != nil { @@ -747,7 +797,7 @@ type M3CompressedSeries struct { func (m *M3CompressedSeries) Reset() { *m = M3CompressedSeries{} } func (m *M3CompressedSeries) String() string { return proto.CompactTextString(m) } func (*M3CompressedSeries) ProtoMessage() {} -func (*M3CompressedSeries) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{13} } +func (*M3CompressedSeries) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{15} } func (m *M3CompressedSeries) GetCompressedTags() []byte { if m != nil { @@ -770,7 +820,7 @@ type M3CompressedValuesReplica struct { func (m *M3CompressedValuesReplica) Reset() { *m = M3CompressedValuesReplica{} } func (m *M3CompressedValuesReplica) String() string { return proto.CompactTextString(m) } func (*M3CompressedValuesReplica) ProtoMessage() {} -func (*M3CompressedValuesReplica) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{14} } +func (*M3CompressedValuesReplica) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{16} } func (m *M3CompressedValuesReplica) GetSegments() []*M3Segments { if m != nil { @@ -787,7 +837,7 @@ type M3Segments struct { func (m *M3Segments) Reset() { *m = M3Segments{} } func (m *M3Segments) String() string { return proto.CompactTextString(m) } func (*M3Segments) ProtoMessage() {} -func (*M3Segments) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{15} } +func (*M3Segments) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{17} } func (m *M3Segments) GetMerged() *M3Segment { if m != nil { @@ -813,7 +863,7 @@ type M3Segment struct { func (m *M3Segment) Reset() { *m = M3Segment{} } func (m *M3Segment) String() string { return proto.CompactTextString(m) } func (*M3Segment) ProtoMessage() {} -func (*M3Segment) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{16} } +func (*M3Segment) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{18} } func (m *M3Segment) GetHead() []byte { if m != nil { @@ -855,7 +905,7 @@ type SearchRequest struct { func (m *SearchRequest) Reset() { *m = SearchRequest{} } func (m *SearchRequest) String() string { return proto.CompactTextString(m) } func (*SearchRequest) ProtoMessage() {} -func (*SearchRequest) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{17} } +func (*SearchRequest) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{19} } type isSearchRequest_Matchers interface { isSearchRequest_Matchers() @@ -967,7 +1017,7 @@ type M3TagProperty struct { func (m *M3TagProperty) Reset() { *m = M3TagProperty{} } func (m *M3TagProperty) String() string { return proto.CompactTextString(m) } func (*M3TagProperty) ProtoMessage() {} -func (*M3TagProperty) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{18} } +func (*M3TagProperty) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{20} } func (m *M3TagProperty) GetId() []byte { if m != nil { @@ -990,7 +1040,7 @@ type M3TagProperties struct { func (m *M3TagProperties) Reset() { *m = M3TagProperties{} } func (m *M3TagProperties) String() string { return proto.CompactTextString(m) } func (*M3TagProperties) ProtoMessage() {} -func (*M3TagProperties) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{19} } +func (*M3TagProperties) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{21} } func (m *M3TagProperties) GetProperties() []*M3TagProperty { if m != nil { @@ -1007,7 +1057,7 @@ type TagProperty struct { func (m *TagProperty) Reset() { *m = TagProperty{} } func (m *TagProperty) String() string { return proto.CompactTextString(m) } func (*TagProperty) ProtoMessage() {} -func (*TagProperty) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{20} } +func (*TagProperty) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{22} } func (m *TagProperty) GetKey() []byte { if m != nil { @@ -1030,7 +1080,7 @@ type TagProperties struct { func (m *TagProperties) Reset() { *m = TagProperties{} } func (m *TagProperties) String() string { return proto.CompactTextString(m) } func (*TagProperties) ProtoMessage() {} -func (*TagProperties) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{21} } +func (*TagProperties) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{23} } func (m *TagProperties) GetProperties() []*TagProperty { if m != nil { @@ -1050,7 +1100,7 @@ type SearchResponse struct { func (m *SearchResponse) Reset() { *m = SearchResponse{} } func (m *SearchResponse) String() string { return proto.CompactTextString(m) } func (*SearchResponse) ProtoMessage() {} -func (*SearchResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{22} } +func (*SearchResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{24} } type isSearchResponse_Value interface { isSearchResponse_Value() @@ -1181,7 +1231,7 @@ type CompleteTagsRequestOptions struct { func (m *CompleteTagsRequestOptions) Reset() { *m = CompleteTagsRequestOptions{} } func (m *CompleteTagsRequestOptions) String() string { return proto.CompactTextString(m) } func (*CompleteTagsRequestOptions) ProtoMessage() {} -func (*CompleteTagsRequestOptions) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{23} } +func (*CompleteTagsRequestOptions) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{25} } func (m *CompleteTagsRequestOptions) GetType() CompleteTagsType { if m != nil { @@ -1228,7 +1278,7 @@ type CompleteTagsRequest struct { func (m *CompleteTagsRequest) Reset() { *m = CompleteTagsRequest{} } func (m *CompleteTagsRequest) String() string { return proto.CompactTextString(m) } func (*CompleteTagsRequest) ProtoMessage() {} -func (*CompleteTagsRequest) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{24} } +func (*CompleteTagsRequest) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{26} } type isCompleteTagsRequest_Matchers interface { isCompleteTagsRequest_Matchers() @@ -1325,7 +1375,7 @@ type TagNames struct { func (m *TagNames) Reset() { *m = TagNames{} } func (m *TagNames) String() string { return proto.CompactTextString(m) } func (*TagNames) ProtoMessage() {} -func (*TagNames) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{25} } +func (*TagNames) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{27} } func (m *TagNames) GetNames() [][]byte { if m != nil { @@ -1342,7 +1392,7 @@ type TagValue struct { func (m *TagValue) Reset() { *m = TagValue{} } func (m *TagValue) String() string { return proto.CompactTextString(m) } func (*TagValue) ProtoMessage() {} -func (*TagValue) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{26} } +func (*TagValue) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{28} } func (m *TagValue) GetKey() []byte { if m != nil { @@ -1365,7 +1415,7 @@ type TagValues struct { func (m *TagValues) Reset() { *m = TagValues{} } func (m *TagValues) String() string { return proto.CompactTextString(m) } func (*TagValues) ProtoMessage() {} -func (*TagValues) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{27} } +func (*TagValues) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{29} } func (m *TagValues) GetValues() []*TagValue { if m != nil { @@ -1385,7 +1435,7 @@ type CompleteTagsResponse struct { func (m *CompleteTagsResponse) Reset() { *m = CompleteTagsResponse{} } func (m *CompleteTagsResponse) String() string { return proto.CompactTextString(m) } func (*CompleteTagsResponse) ProtoMessage() {} -func (*CompleteTagsResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{28} } +func (*CompleteTagsResponse) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{30} } type isCompleteTagsResponse_Value interface { isCompleteTagsResponse_Value() @@ -1514,7 +1564,7 @@ type ResultMetadata struct { func (m *ResultMetadata) Reset() { *m = ResultMetadata{} } func (m *ResultMetadata) String() string { return proto.CompactTextString(m) } func (*ResultMetadata) ProtoMessage() {} -func (*ResultMetadata) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{29} } +func (*ResultMetadata) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{31} } func (m *ResultMetadata) GetExhaustive() bool { if m != nil { @@ -1545,7 +1595,7 @@ type Warning struct { func (m *Warning) Reset() { *m = Warning{} } func (m *Warning) String() string { return proto.CompactTextString(m) } func (*Warning) ProtoMessage() {} -func (*Warning) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{30} } +func (*Warning) Descriptor() ([]byte, []int) { return fileDescriptorQuery, []int{32} } func (m *Warning) GetName() []byte { if m != nil { @@ -1568,7 +1618,9 @@ func init() { proto.RegisterType((*TagMatchers)(nil), "rpc.TagMatchers") proto.RegisterType((*TagMatcher)(nil), "rpc.TagMatcher") proto.RegisterType((*FetchOptions)(nil), "rpc.FetchOptions") - proto.RegisterType((*RestrictFetchOptions)(nil), "rpc.RestrictFetchOptions") + proto.RegisterType((*RestrictQueryOptions)(nil), "rpc.RestrictQueryOptions") + proto.RegisterType((*RestrictFetchType)(nil), "rpc.RestrictFetchType") + proto.RegisterType((*RestrictFetchTags)(nil), "rpc.RestrictFetchTags") proto.RegisterType((*FetchResponse)(nil), "rpc.FetchResponse") proto.RegisterType((*Series)(nil), "rpc.Series") proto.RegisterType((*SeriesMetadata)(nil), "rpc.SeriesMetadata") @@ -2086,7 +2138,45 @@ func (m *FetchOptions) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *RestrictFetchOptions) Marshal() (dAtA []byte, err error) { +func (m *RestrictQueryOptions) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RestrictQueryOptions) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.RestrictFetchType != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintQuery(dAtA, i, uint64(m.RestrictFetchType.Size())) + n5, err := m.RestrictFetchType.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + } + if m.RestrictFetchTags != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintQuery(dAtA, i, uint64(m.RestrictFetchTags.Size())) + n6, err := m.RestrictFetchTags.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } + return i, nil +} + +func (m *RestrictFetchType) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -2096,7 +2186,7 @@ func (m *RestrictFetchOptions) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RestrictFetchOptions) MarshalTo(dAtA []byte) (int, error) { +func (m *RestrictFetchType) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -2110,11 +2200,47 @@ func (m *RestrictFetchOptions) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.MetricsStoragePolicy.Size())) - n5, err := m.MetricsStoragePolicy.MarshalTo(dAtA[i:]) + n7, err := m.MetricsStoragePolicy.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n5 + i += n7 + } + return i, nil +} + +func (m *RestrictFetchTags) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RestrictFetchTags) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Restrict != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintQuery(dAtA, i, uint64(m.Restrict.Size())) + n8, err := m.Restrict.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + } + if len(m.Strip) > 0 { + for _, b := range m.Strip { + dAtA[i] = 0x12 + i++ + i = encodeVarintQuery(dAtA, i, uint64(len(b))) + i += copy(dAtA[i:], b) + } } return i, nil } @@ -2150,11 +2276,11 @@ func (m *FetchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.Meta.Size())) - n6, err := m.Meta.MarshalTo(dAtA[i:]) + n9, err := m.Meta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n9 } return i, nil } @@ -2178,18 +2304,18 @@ func (m *Series) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.Meta.Size())) - n7, err := m.Meta.MarshalTo(dAtA[i:]) + n10, err := m.Meta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n10 } if m.Value != nil { - nn8, err := m.Value.MarshalTo(dAtA[i:]) + nn11, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn8 + i += nn11 } return i, nil } @@ -2200,11 +2326,11 @@ func (m *Series_Decompressed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.Decompressed.Size())) - n9, err := m.Decompressed.MarshalTo(dAtA[i:]) + n12, err := m.Decompressed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n12 } return i, nil } @@ -2214,11 +2340,11 @@ func (m *Series_Compressed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintQuery(dAtA, i, uint64(m.Compressed.Size())) - n10, err := m.Compressed.MarshalTo(dAtA[i:]) + n13, err := m.Compressed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n13 } return i, nil } @@ -2442,11 +2568,11 @@ func (m *M3Segments) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.Merged.Size())) - n11, err := m.Merged.MarshalTo(dAtA[i:]) + n14, err := m.Merged.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n14 } if len(m.Unmerged) > 0 { for _, msg := range m.Unmerged { @@ -2519,11 +2645,11 @@ func (m *SearchRequest) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Matchers != nil { - nn12, err := m.Matchers.MarshalTo(dAtA[i:]) + nn15, err := m.Matchers.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn12 + i += nn15 } if m.Start != 0 { dAtA[i] = 0x10 @@ -2539,11 +2665,11 @@ func (m *SearchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintQuery(dAtA, i, uint64(m.Options.Size())) - n13, err := m.Options.MarshalTo(dAtA[i:]) + n16, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n16 } return i, nil } @@ -2554,11 +2680,11 @@ func (m *SearchRequest_TagMatchers) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.TagMatchers.Size())) - n14, err := m.TagMatchers.MarshalTo(dAtA[i:]) + n17, err := m.TagMatchers.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n17 } return i, nil } @@ -2698,21 +2824,21 @@ func (m *SearchResponse) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn15, err := m.Value.MarshalTo(dAtA[i:]) + nn18, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn15 + i += nn18 } if m.Meta != nil { dAtA[i] = 0x1a i++ i = encodeVarintQuery(dAtA, i, uint64(m.Meta.Size())) - n16, err := m.Meta.MarshalTo(dAtA[i:]) + n19, err := m.Meta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n19 } return i, nil } @@ -2723,11 +2849,11 @@ func (m *SearchResponse_Decompressed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.Decompressed.Size())) - n17, err := m.Decompressed.MarshalTo(dAtA[i:]) + n20, err := m.Decompressed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n20 } return i, nil } @@ -2737,11 +2863,11 @@ func (m *SearchResponse_Compressed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.Compressed.Size())) - n18, err := m.Compressed.MarshalTo(dAtA[i:]) + n21, err := m.Compressed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n21 } return i, nil } @@ -2787,11 +2913,11 @@ func (m *CompleteTagsRequestOptions) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintQuery(dAtA, i, uint64(m.Options.Size())) - n19, err := m.Options.MarshalTo(dAtA[i:]) + n22, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n22 } return i, nil } @@ -2812,21 +2938,21 @@ func (m *CompleteTagsRequest) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Matchers != nil { - nn20, err := m.Matchers.MarshalTo(dAtA[i:]) + nn23, err := m.Matchers.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn20 + i += nn23 } if m.Options != nil { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.Options.Size())) - n21, err := m.Options.MarshalTo(dAtA[i:]) + n24, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n24 } return i, nil } @@ -2837,11 +2963,11 @@ func (m *CompleteTagsRequest_TagMatchers) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.TagMatchers.Size())) - n22, err := m.TagMatchers.MarshalTo(dAtA[i:]) + n25, err := m.TagMatchers.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n25 } return i, nil } @@ -2949,21 +3075,21 @@ func (m *CompleteTagsResponse) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn23, err := m.Value.MarshalTo(dAtA[i:]) + nn26, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn23 + i += nn26 } if m.Meta != nil { dAtA[i] = 0x1a i++ i = encodeVarintQuery(dAtA, i, uint64(m.Meta.Size())) - n24, err := m.Meta.MarshalTo(dAtA[i:]) + n27, err := m.Meta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n27 } return i, nil } @@ -2974,11 +3100,11 @@ func (m *CompleteTagsResponse_Default) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintQuery(dAtA, i, uint64(m.Default.Size())) - n25, err := m.Default.MarshalTo(dAtA[i:]) + n28, err := m.Default.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n28 } return i, nil } @@ -2988,11 +3114,11 @@ func (m *CompleteTagsResponse_NamesOnly) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintQuery(dAtA, i, uint64(m.NamesOnly.Size())) - n26, err := m.NamesOnly.MarshalTo(dAtA[i:]) + n29, err := m.NamesOnly.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n29 } return i, nil } @@ -3034,22 +3160,22 @@ func (m *ResultMetadata) MarshalTo(dAtA []byte) (int, error) { } } if len(m.Resolutions) > 0 { - dAtA28 := make([]byte, len(m.Resolutions)*10) - var j27 int + dAtA31 := make([]byte, len(m.Resolutions)*10) + var j30 int for _, num1 := range m.Resolutions { num := uint64(num1) for num >= 1<<7 { - dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) + dAtA31[j30] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j27++ + j30++ } - dAtA28[j27] = uint8(num) - j27++ + dAtA31[j30] = uint8(num) + j30++ } dAtA[i] = 0x1a i++ - i = encodeVarintQuery(dAtA, i, uint64(j27)) - i += copy(dAtA[i:], dAtA28[:j27]) + i = encodeVarintQuery(dAtA, i, uint64(j30)) + i += copy(dAtA[i:], dAtA31[:j30]) } return i, nil } @@ -3197,7 +3323,21 @@ func (m *FetchOptions) Size() (n int) { return n } -func (m *RestrictFetchOptions) Size() (n int) { +func (m *RestrictQueryOptions) Size() (n int) { + var l int + _ = l + if m.RestrictFetchType != nil { + l = m.RestrictFetchType.Size() + n += 1 + l + sovQuery(uint64(l)) + } + if m.RestrictFetchTags != nil { + l = m.RestrictFetchTags.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} + +func (m *RestrictFetchType) Size() (n int) { var l int _ = l if m.MetricsType != 0 { @@ -3210,6 +3350,22 @@ func (m *RestrictFetchOptions) Size() (n int) { return n } +func (m *RestrictFetchTags) Size() (n int) { + var l int + _ = l + if m.Restrict != nil { + l = m.Restrict.Size() + n += 1 + l + sovQuery(uint64(l)) + } + if len(m.Strip) > 0 { + for _, b := range m.Strip { + l = len(b) + n += 1 + l + sovQuery(uint64(l)) + } + } + return n +} + func (m *FetchResponse) Size() (n int) { var l int _ = l @@ -4247,7 +4403,7 @@ func (m *FetchOptions) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Restrict == nil { - m.Restrict = &RestrictFetchOptions{} + m.Restrict = &RestrictQueryOptions{} } if err := m.Restrict.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4370,7 +4526,7 @@ func (m *FetchOptions) Unmarshal(dAtA []byte) error { } return nil } -func (m *RestrictFetchOptions) Unmarshal(dAtA []byte) error { +func (m *RestrictQueryOptions) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4393,10 +4549,126 @@ func (m *RestrictFetchOptions) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RestrictFetchOptions: wiretype end group for non-group") + return fmt.Errorf("proto: RestrictQueryOptions: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RestrictFetchOptions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RestrictQueryOptions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestrictFetchType", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RestrictFetchType == nil { + m.RestrictFetchType = &RestrictFetchType{} + } + if err := m.RestrictFetchType.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestrictFetchTags", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RestrictFetchTags == nil { + m.RestrictFetchTags = &RestrictFetchTags{} + } + if err := m.RestrictFetchTags.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RestrictFetchType) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RestrictFetchType: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RestrictFetchType: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -4472,6 +4744,118 @@ func (m *RestrictFetchOptions) Unmarshal(dAtA []byte) error { } return nil } +func (m *RestrictFetchTags) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RestrictFetchTags: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RestrictFetchTags: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Restrict", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Restrict == nil { + m.Restrict = &TagMatchers{} + } + if err := m.Restrict.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Strip", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Strip = append(m.Strip, make([]byte, postIndex-iNdEx)) + copy(m.Strip[len(m.Strip)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *FetchResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -7385,104 +7769,108 @@ func init() { } var fileDescriptorQuery = []byte{ - // 1579 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x57, 0xdb, 0x52, 0x1b, 0x47, - 0x1a, 0xd6, 0x68, 0xd0, 0xe9, 0xd7, 0x01, 0xb9, 0x61, 0xd7, 0x82, 0xf5, 0xb2, 0xd4, 0xec, 0xae, - 0x97, 0xc5, 0x5e, 0x64, 0x83, 0xbd, 0x8e, 0x53, 0x95, 0x83, 0x00, 0x19, 0x28, 0x83, 0x84, 0x5b, - 0x43, 0xec, 0xa4, 0x92, 0x22, 0xad, 0x51, 0x5b, 0x4c, 0xa1, 0x39, 0x78, 0xa6, 0xe5, 0x18, 0x57, - 0x1e, 0x22, 0x95, 0x9b, 0x3c, 0x40, 0x2a, 0x79, 0x02, 0x3f, 0x42, 0x2e, 0x72, 0x99, 0x47, 0x48, - 0x39, 0xb9, 0xc8, 0x63, 0xa4, 0xba, 0xa7, 0xe7, 0x24, 0x89, 0xb2, 0xcb, 0x77, 0xd3, 0xff, 0xb1, - 0xff, 0xbf, 0xbf, 0xfe, 0xfe, 0x1e, 0xf8, 0x70, 0x68, 0xb2, 0xb3, 0x71, 0x7f, 0xc3, 0x70, 0xac, - 0xa6, 0xb5, 0x35, 0xe8, 0x37, 0xad, 0xad, 0xa6, 0xef, 0x19, 0xcd, 0x67, 0x63, 0xea, 0x5d, 0x34, - 0x87, 0xd4, 0xa6, 0x1e, 0x61, 0x74, 0xd0, 0x74, 0x3d, 0x87, 0x39, 0x4d, 0xcf, 0x35, 0xdc, 0x7e, - 0xa0, 0xdb, 0x10, 0x12, 0xa4, 0x7a, 0xae, 0xb1, 0xbc, 0x7b, 0x49, 0x10, 0x8b, 0x32, 0xcf, 0x34, - 0xfc, 0xa9, 0x30, 0xae, 0x33, 0x32, 0x8d, 0x0b, 0xb7, 0x2f, 0x3f, 0x82, 0x50, 0xda, 0x3c, 0x54, - 0xf7, 0x29, 0x19, 0xb1, 0x33, 0x4c, 0x9f, 0x8d, 0xa9, 0xcf, 0xb4, 0xa7, 0x50, 0x0b, 0x05, 0xbe, - 0xeb, 0xd8, 0x3e, 0x45, 0xd7, 0xa1, 0x36, 0x76, 0x99, 0x69, 0xd1, 0xdd, 0xb1, 0x47, 0x98, 0xe9, - 0xd8, 0x0d, 0x65, 0x55, 0x59, 0x2b, 0xe1, 0x09, 0x29, 0xba, 0x09, 0x57, 0x02, 0x49, 0x87, 0xd8, - 0x8e, 0x4f, 0x0d, 0xc7, 0x1e, 0xf8, 0x8d, 0xec, 0xaa, 0xb2, 0xa6, 0xe2, 0x69, 0x85, 0xf6, 0x83, - 0x02, 0x95, 0x07, 0x94, 0x19, 0x61, 0x62, 0xb4, 0x08, 0x39, 0x9f, 0x11, 0x8f, 0x89, 0xe8, 0x2a, - 0x0e, 0x16, 0xa8, 0x0e, 0x2a, 0xb5, 0x07, 0x32, 0x0c, 0xff, 0x44, 0x77, 0xa0, 0xcc, 0xc8, 0xf0, - 0x88, 0x30, 0xe3, 0x8c, 0x7a, 0x7e, 0x43, 0x5d, 0x55, 0xd6, 0xca, 0x9b, 0xf5, 0x0d, 0xcf, 0x35, - 0x36, 0xf4, 0x58, 0xbe, 0x9f, 0xc1, 0x49, 0x33, 0x74, 0x03, 0x0a, 0x8e, 0xcb, 0xb7, 0xe9, 0x37, - 0xe6, 0x84, 0xc7, 0x15, 0xe1, 0x21, 0x76, 0xd0, 0x0d, 0x14, 0x38, 0xb4, 0xd8, 0x06, 0x28, 0x5a, - 0xd2, 0x51, 0xfb, 0x18, 0xca, 0x89, 0xb0, 0xe8, 0x76, 0x3a, 0xbb, 0xb2, 0xaa, 0xae, 0x95, 0x37, - 0xe7, 0x27, 0xb2, 0xa7, 0x52, 0x6b, 0x9f, 0x03, 0xc4, 0x2a, 0x84, 0x60, 0xce, 0x26, 0x16, 0x15, - 0x55, 0x56, 0xb0, 0xf8, 0xe6, 0xa5, 0x3f, 0x27, 0xa3, 0x31, 0x15, 0x65, 0x56, 0x70, 0xb0, 0x40, - 0xff, 0x82, 0x39, 0x76, 0xe1, 0x52, 0x51, 0x61, 0x4d, 0x56, 0x28, 0xa3, 0xe8, 0x17, 0x2e, 0xc5, - 0x42, 0xab, 0xfd, 0x9e, 0x95, 0x7d, 0x94, 0x55, 0xf0, 0x60, 0x23, 0xd3, 0x32, 0xa3, 0x3e, 0x8a, - 0x05, 0xba, 0x0b, 0x45, 0x8f, 0xfa, 0x1c, 0x19, 0x4c, 0x64, 0x29, 0x6f, 0x2e, 0x89, 0x80, 0x58, - 0x0a, 0x53, 0x8d, 0x88, 0x4c, 0xd1, 0x3a, 0xd4, 0x47, 0x8e, 0x73, 0xde, 0x27, 0xc6, 0x79, 0x74, - 0xfa, 0xaa, 0x88, 0x3b, 0x25, 0x47, 0x77, 0xa1, 0x32, 0xb6, 0xc9, 0x70, 0xe8, 0xd1, 0x21, 0x87, - 0x9d, 0xe8, 0x73, 0x2d, 0xec, 0x33, 0xb1, 0x9d, 0x31, 0x0b, 0xe2, 0xe3, 0x94, 0x19, 0xba, 0x0d, - 0x90, 0x70, 0xca, 0x5d, 0xe6, 0x94, 0x30, 0x42, 0x3b, 0xb0, 0x10, 0xaf, 0xb8, 0xde, 0x32, 0x5f, - 0xd2, 0x41, 0x23, 0x7f, 0x99, 0xef, 0x2c, 0x6b, 0x0e, 0x57, 0xd3, 0x36, 0x46, 0xe3, 0x01, 0xc5, - 0xd4, 0x77, 0x46, 0x63, 0x51, 0x5b, 0x61, 0x55, 0x59, 0x2b, 0xe2, 0x69, 0x85, 0xf6, 0x9d, 0x02, - 0x8b, 0xb3, 0x7a, 0x85, 0x36, 0xa1, 0x2c, 0x6f, 0x1c, 0x3f, 0x14, 0xd1, 0xf4, 0xe8, 0xb0, 0x62, - 0x39, 0x4e, 0x1a, 0xa1, 0x87, 0xb0, 0x28, 0x97, 0x3d, 0xe6, 0x78, 0x64, 0x48, 0x8f, 0xc5, 0x95, - 0x94, 0x07, 0x73, 0x75, 0x23, 0xbc, 0xaa, 0x1b, 0x29, 0x35, 0x9e, 0xe9, 0xa4, 0x7d, 0x01, 0x55, - 0x79, 0x8f, 0xe4, 0x7d, 0xfd, 0x27, 0xe4, 0x7d, 0xea, 0x99, 0x34, 0x44, 0x67, 0x59, 0x6c, 0xa6, - 0x27, 0x44, 0x58, 0xaa, 0xd0, 0x7f, 0x60, 0xce, 0xa2, 0x8c, 0xc8, 0x94, 0x0b, 0x21, 0x16, 0xc6, - 0x23, 0x76, 0x44, 0x19, 0x19, 0x10, 0x46, 0xb0, 0x30, 0xd0, 0x5e, 0x29, 0x90, 0xef, 0xa5, 0x7d, - 0x94, 0x84, 0x4f, 0xa0, 0x4a, 0xfb, 0xa0, 0x0f, 0xa0, 0x32, 0xa0, 0x86, 0x63, 0xb9, 0x1e, 0xf5, - 0x7d, 0x3a, 0x88, 0xea, 0xe2, 0x0e, 0xbb, 0x09, 0x45, 0xe0, 0xbc, 0x9f, 0xc1, 0x29, 0x73, 0x74, - 0x1f, 0x20, 0xe1, 0xac, 0x26, 0x9c, 0x8f, 0xb6, 0x76, 0xa6, 0x9d, 0x13, 0xc6, 0xdb, 0x05, 0x79, - 0x93, 0xb4, 0x27, 0x50, 0x4b, 0x6f, 0x0d, 0xd5, 0x20, 0x6b, 0x0e, 0xe4, 0xb5, 0xcb, 0x9a, 0x03, - 0x74, 0x0d, 0x4a, 0x82, 0x62, 0x74, 0xd3, 0xa2, 0x92, 0x5f, 0x62, 0x01, 0x6a, 0x40, 0x81, 0xda, - 0x03, 0xa1, 0x0b, 0xf0, 0x1e, 0x2e, 0xb5, 0x3e, 0xa0, 0xe9, 0x1a, 0xd0, 0x06, 0x00, 0xcf, 0xe2, - 0x3a, 0xa6, 0xcd, 0xc2, 0xc6, 0xd7, 0x82, 0x82, 0x43, 0x31, 0x4e, 0x58, 0xa0, 0x6b, 0x30, 0xc7, - 0xc8, 0x90, 0xf3, 0x23, 0xb7, 0x2c, 0x86, 0x04, 0x82, 0x85, 0x54, 0xfb, 0x08, 0x4a, 0x91, 0x1b, - 0xdf, 0x28, 0x27, 0x4f, 0x9f, 0x11, 0xcb, 0x95, 0x97, 0x3a, 0x16, 0xa4, 0xb9, 0x43, 0x91, 0xdc, - 0xa1, 0x35, 0x41, 0xd5, 0xc9, 0xf0, 0xed, 0xc9, 0x46, 0x7b, 0x01, 0x68, 0xba, 0xb9, 0x9c, 0xfa, - 0xe3, 0x4a, 0x75, 0xbe, 0xdf, 0x20, 0xd2, 0x84, 0x14, 0xbd, 0xcf, 0xd9, 0xc5, 0x1d, 0x99, 0x06, - 0x09, 0x2b, 0x5a, 0x99, 0x3a, 0xaf, 0x4f, 0x78, 0x1e, 0x1f, 0x07, 0x66, 0x38, 0xb2, 0xd7, 0xf6, - 0x61, 0xe9, 0x52, 0x33, 0x74, 0x03, 0x8a, 0x3e, 0x1d, 0x5a, 0x34, 0x6e, 0xea, 0xbc, 0x0c, 0xdc, - 0x93, 0x62, 0x1c, 0x19, 0x68, 0x5f, 0x02, 0xc4, 0x72, 0x74, 0x1d, 0xf2, 0x16, 0xf5, 0x86, 0x74, - 0x20, 0xf1, 0x5a, 0x4b, 0x3b, 0x62, 0xa9, 0x45, 0xeb, 0x50, 0x1c, 0xdb, 0xd2, 0x32, 0x9b, 0x38, - 0xb7, 0xd8, 0x32, 0xd2, 0x6b, 0x0e, 0x94, 0x22, 0x31, 0x6f, 0xee, 0x19, 0x25, 0x21, 0xa4, 0xc4, - 0x37, 0x97, 0x31, 0x62, 0x8e, 0x64, 0x6f, 0xc5, 0x77, 0x1a, 0x68, 0xea, 0x24, 0xd0, 0xae, 0x41, - 0xa9, 0x3f, 0x72, 0x8c, 0xf3, 0x9e, 0xf9, 0x92, 0x0a, 0xca, 0x54, 0x71, 0x2c, 0xd0, 0x7e, 0x54, - 0xa0, 0xda, 0xa3, 0xc4, 0x8b, 0xc7, 0xe4, 0x9d, 0xc9, 0x01, 0xf4, 0x56, 0xe3, 0x2f, 0x1a, 0xae, - 0xd9, 0x19, 0xc3, 0x55, 0x8d, 0x87, 0xeb, 0x3b, 0x8f, 0xc9, 0x3d, 0xa8, 0x1e, 0x6d, 0xe9, 0x64, - 0x78, 0xec, 0x39, 0x2e, 0xf5, 0xd8, 0xc5, 0xd4, 0x75, 0x9b, 0x86, 0x52, 0x76, 0x16, 0x94, 0xb4, - 0x36, 0xcc, 0x27, 0x03, 0x71, 0x14, 0x6e, 0x02, 0xb8, 0xd1, 0x4a, 0xc2, 0x00, 0xc9, 0x33, 0x4a, - 0xa4, 0xc4, 0x09, 0x2b, 0xed, 0x9e, 0x18, 0xdb, 0xd1, 0x6e, 0xea, 0xa0, 0x9e, 0xd3, 0x0b, 0xb9, - 0x1d, 0xfe, 0x89, 0xfe, 0x0a, 0x79, 0x81, 0xfc, 0x70, 0x1f, 0x72, 0xa5, 0xb5, 0xa0, 0x9a, 0xce, - 0x7e, 0x6b, 0x46, 0xf6, 0xa8, 0xdf, 0x33, 0x73, 0xbf, 0x52, 0x38, 0xf9, 0x04, 0x87, 0x26, 0x39, - 0xf9, 0xbd, 0x09, 0x46, 0x0c, 0x8e, 0x0d, 0x4d, 0x84, 0x99, 0x45, 0x86, 0xff, 0x4f, 0x91, 0x61, - 0xc0, 0xa4, 0x8b, 0x53, 0xc5, 0x4f, 0x31, 0x61, 0x44, 0xd6, 0xea, 0x1b, 0x08, 0x3e, 0xa6, 0xcc, - 0x9f, 0x14, 0x58, 0xe6, 0xf7, 0x70, 0x44, 0x19, 0xe5, 0x47, 0x21, 0x11, 0x17, 0x0e, 0xba, 0xff, - 0xca, 0xe7, 0x48, 0x30, 0xe1, 0xfe, 0x22, 0x02, 0x26, 0xcd, 0xe3, 0x37, 0x09, 0x3f, 0xeb, 0xa7, - 0xe6, 0x88, 0x51, 0xaf, 0x43, 0x2c, 0xaa, 0x87, 0x34, 0x57, 0xc1, 0x13, 0xd2, 0x18, 0x95, 0xea, - 0x0c, 0x54, 0xce, 0xcd, 0x44, 0x65, 0xee, 0x4d, 0xa8, 0xd4, 0xbe, 0x55, 0x60, 0x61, 0x46, 0x19, - 0xef, 0x78, 0x71, 0xee, 0xc7, 0xa9, 0x83, 0xde, 0xff, 0x63, 0xaa, 0xf0, 0x74, 0x9f, 0x66, 0x5f, - 0x8f, 0x55, 0x28, 0xea, 0x64, 0xc8, 0x0b, 0x17, 0x55, 0x73, 0x22, 0x0e, 0xb0, 0x54, 0xc1, 0xc1, - 0x42, 0xbb, 0x23, 0x2c, 0x04, 0xfb, 0xbd, 0x01, 0xad, 0x6a, 0x02, 0xad, 0x9b, 0x50, 0x0a, 0xbd, - 0x7c, 0xf4, 0xef, 0xc8, 0x28, 0x40, 0x69, 0x35, 0x2c, 0x4e, 0xe8, 0x23, 0x9f, 0xef, 0x15, 0x58, - 0x4c, 0xef, 0x5f, 0x82, 0x74, 0x1d, 0x0a, 0x03, 0xfa, 0x94, 0x8c, 0x47, 0x2c, 0x45, 0x99, 0x51, - 0x82, 0xfd, 0x0c, 0x0e, 0x0d, 0xd0, 0xff, 0xa0, 0x24, 0xf6, 0xdd, 0xb5, 0x47, 0xe1, 0xbb, 0x25, - 0x4a, 0x27, 0xca, 0xdc, 0xcf, 0xe0, 0xd8, 0xe2, 0x1d, 0xd0, 0xf8, 0x35, 0xd4, 0xd2, 0x06, 0x68, - 0x05, 0x80, 0xbe, 0x38, 0x23, 0x63, 0x9f, 0x99, 0xcf, 0x03, 0x18, 0x16, 0x71, 0x42, 0x82, 0xd6, - 0xa0, 0xf8, 0x15, 0xf1, 0x6c, 0xd3, 0x8e, 0xc6, 0x6a, 0x45, 0xe4, 0x79, 0x1c, 0x08, 0x71, 0xa4, - 0x45, 0xab, 0x50, 0xf6, 0xa2, 0xa7, 0x1d, 0xff, 0x85, 0x50, 0xd7, 0x54, 0x9c, 0x14, 0x69, 0xf7, - 0xa0, 0x20, 0xdd, 0x66, 0xce, 0xd0, 0x06, 0x14, 0x2c, 0xea, 0xfb, 0x64, 0x18, 0x4e, 0xd1, 0x70, - 0xb9, 0x4e, 0xa1, 0x9c, 0x78, 0xa3, 0xa3, 0x12, 0xe4, 0xda, 0x8f, 0x4e, 0x5a, 0x87, 0xf5, 0x0c, - 0xaa, 0x40, 0xb1, 0xd3, 0xd5, 0x83, 0x95, 0x82, 0x00, 0xf2, 0xb8, 0xbd, 0xd7, 0x7e, 0x72, 0x5c, - 0xcf, 0xa2, 0x2a, 0x94, 0x3a, 0x5d, 0x5d, 0x2e, 0x55, 0xae, 0x6a, 0x3f, 0x39, 0xe8, 0xe9, 0xbd, - 0xfa, 0x9c, 0x54, 0xc9, 0x65, 0x0e, 0x15, 0x40, 0x6d, 0x1d, 0x1e, 0xd6, 0xf3, 0xeb, 0x06, 0x94, - 0x13, 0xaf, 0x4b, 0xd4, 0x80, 0xc5, 0x93, 0xce, 0xc3, 0x4e, 0xf7, 0x71, 0xe7, 0xf4, 0xa8, 0xad, - 0xe3, 0x83, 0x9d, 0xde, 0xa9, 0xfe, 0xe9, 0x71, 0xbb, 0x9e, 0x41, 0x7f, 0x87, 0xa5, 0x93, 0x4e, - 0x6b, 0x6f, 0x0f, 0xb7, 0xf7, 0x5a, 0x7a, 0x7b, 0x37, 0xad, 0x56, 0xd0, 0xdf, 0xe0, 0xea, 0x65, - 0xca, 0xec, 0xfa, 0x01, 0x54, 0x92, 0xcf, 0x68, 0x84, 0xa0, 0xb6, 0xdb, 0x7e, 0xd0, 0x3a, 0x39, - 0xd4, 0x4f, 0xbb, 0xc7, 0xfa, 0x41, 0xb7, 0x53, 0xcf, 0xa0, 0x2b, 0x50, 0x7d, 0xd0, 0xc5, 0x3b, - 0xed, 0xd3, 0x76, 0xa7, 0xb5, 0x7d, 0xd8, 0xde, 0xad, 0x2b, 0xdc, 0x2c, 0x10, 0xed, 0x1e, 0xf4, - 0x02, 0x59, 0x76, 0xfd, 0x26, 0xd4, 0x27, 0xb9, 0x02, 0x95, 0xa1, 0x20, 0xc3, 0xd5, 0x33, 0x7c, - 0xa1, 0xb7, 0xf6, 0x3a, 0xad, 0xa3, 0x76, 0x5d, 0xd9, 0xfc, 0x43, 0x81, 0xdc, 0x23, 0xfe, 0xbf, - 0x8b, 0x6e, 0x43, 0x3e, 0xf8, 0x1b, 0x45, 0x01, 0x57, 0xa6, 0xfe, 0x55, 0x97, 0x17, 0x52, 0x32, - 0x89, 0xe2, 0x5b, 0x90, 0x13, 0xc4, 0x80, 0x12, 0x24, 0x11, 0x3a, 0xa0, 0xa4, 0x28, 0xb0, 0xbf, - 0xa5, 0xa0, 0x2d, 0xfe, 0xc2, 0xe5, 0x74, 0x2d, 0x93, 0xa4, 0x06, 0xee, 0xf2, 0x42, 0x4a, 0x16, - 0x39, 0xb5, 0xa1, 0x92, 0xac, 0x08, 0x35, 0x2e, 0xe3, 0x85, 0xe5, 0xa5, 0x19, 0x9a, 0x30, 0xcc, - 0xf6, 0xd5, 0x9f, 0x5f, 0xaf, 0x28, 0xbf, 0xbc, 0x5e, 0x51, 0x7e, 0x7d, 0xbd, 0xa2, 0x7c, 0xf3, - 0xdb, 0x4a, 0xe6, 0xb3, 0x9c, 0xf8, 0xdf, 0xef, 0xe7, 0xc5, 0xff, 0xf9, 0xd6, 0x9f, 0x01, 0x00, - 0x00, 0xff, 0xff, 0xf3, 0x1f, 0xfe, 0x09, 0x2c, 0x10, 0x00, 0x00, + // 1640 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xdb, 0x72, 0xdb, 0x4e, + 0x19, 0xb7, 0xac, 0xf8, 0xf4, 0xf9, 0x10, 0x67, 0x13, 0xfe, 0x75, 0x42, 0x09, 0x1e, 0x01, 0x7f, + 0x42, 0x5a, 0xe2, 0xd6, 0x69, 0x29, 0x65, 0x86, 0x83, 0x13, 0xbb, 0x49, 0xa6, 0x89, 0x9d, 0xae, + 0x15, 0x5a, 0x18, 0x98, 0xb0, 0x96, 0xb7, 0x8e, 0x26, 0xd6, 0xa1, 0x92, 0x5c, 0x9a, 0x0e, 0x0f, + 0xc1, 0x30, 0x3c, 0x01, 0x0c, 0x3c, 0x41, 0x1f, 0x81, 0x0b, 0x2e, 0x79, 0x04, 0xa6, 0x70, 0xc1, + 0x63, 0x30, 0xbb, 0x5a, 0x9d, 0x2c, 0x65, 0xda, 0xe9, 0x9d, 0xbf, 0xf3, 0x7e, 0xdf, 0xfe, 0xf6, + 0xb7, 0x2b, 0xc3, 0xcf, 0x66, 0xba, 0x77, 0xb5, 0x98, 0xec, 0x69, 0x96, 0xd1, 0x31, 0xf6, 0xa7, + 0x93, 0x8e, 0xb1, 0xdf, 0x71, 0x1d, 0xad, 0xf3, 0x66, 0x41, 0x9d, 0x9b, 0xce, 0x8c, 0x9a, 0xd4, + 0x21, 0x1e, 0x9d, 0x76, 0x6c, 0xc7, 0xf2, 0xac, 0x8e, 0x63, 0x6b, 0xf6, 0xc4, 0xb7, 0xed, 0x71, + 0x0d, 0x92, 0x1d, 0x5b, 0xdb, 0xea, 0xdf, 0x92, 0xc4, 0xa0, 0x9e, 0xa3, 0x6b, 0x6e, 0x2a, 0x8d, + 0x6d, 0xcd, 0x75, 0xed, 0xc6, 0x9e, 0x88, 0x1f, 0x7e, 0x2a, 0x65, 0x15, 0xea, 0xc7, 0x94, 0xcc, + 0xbd, 0x2b, 0x4c, 0xdf, 0x2c, 0xa8, 0xeb, 0x29, 0xaf, 0xa1, 0x11, 0x28, 0x5c, 0xdb, 0x32, 0x5d, + 0x8a, 0xbe, 0x86, 0xc6, 0xc2, 0xf6, 0x74, 0x83, 0xf6, 0x17, 0x0e, 0xf1, 0x74, 0xcb, 0x6c, 0x49, + 0x6d, 0x69, 0xa7, 0x82, 0x97, 0xb4, 0xe8, 0x3e, 0xac, 0xf9, 0x9a, 0x21, 0x31, 0x2d, 0x97, 0x6a, + 0x96, 0x39, 0x75, 0x5b, 0xf9, 0xb6, 0xb4, 0x23, 0xe3, 0xb4, 0x41, 0xf9, 0x9b, 0x04, 0xb5, 0x67, + 0xd4, 0xd3, 0x82, 0xc2, 0x68, 0x03, 0x0a, 0xae, 0x47, 0x1c, 0x8f, 0x67, 0x97, 0xb1, 0x2f, 0xa0, + 0x26, 0xc8, 0xd4, 0x9c, 0x8a, 0x34, 0xec, 0x27, 0x7a, 0x04, 0x55, 0x8f, 0xcc, 0xce, 0x88, 0xa7, + 0x5d, 0x51, 0xc7, 0x6d, 0xc9, 0x6d, 0x69, 0xa7, 0xda, 0x6d, 0xee, 0x39, 0xb6, 0xb6, 0xa7, 0x46, + 0xfa, 0xe3, 0x1c, 0x8e, 0xbb, 0xa1, 0x7b, 0x50, 0xb2, 0x6c, 0xb6, 0x4c, 0xb7, 0xb5, 0xc2, 0x23, + 0xd6, 0x78, 0x04, 0x5f, 0xc1, 0xc8, 0x37, 0xe0, 0xc0, 0xe3, 0x00, 0xa0, 0x6c, 0x88, 0x40, 0xe5, + 0x17, 0x50, 0x8d, 0xa5, 0x45, 0x0f, 0x93, 0xd5, 0xa5, 0xb6, 0xbc, 0x53, 0xed, 0xae, 0x2e, 0x55, + 0x4f, 0x94, 0x56, 0x7e, 0x03, 0x10, 0x99, 0x10, 0x82, 0x15, 0x93, 0x18, 0x94, 0x77, 0x59, 0xc3, + 0xfc, 0x37, 0x6b, 0xfd, 0x2d, 0x99, 0x2f, 0x28, 0x6f, 0xb3, 0x86, 0x7d, 0x01, 0x7d, 0x17, 0x56, + 0xbc, 0x1b, 0x9b, 0xf2, 0x0e, 0x1b, 0xa2, 0x43, 0x91, 0x45, 0xbd, 0xb1, 0x29, 0xe6, 0x56, 0xe5, + 0xbf, 0x79, 0x31, 0x47, 0xd1, 0x05, 0x4b, 0x36, 0xd7, 0x0d, 0x3d, 0x9c, 0x23, 0x17, 0xd0, 0x63, + 0x28, 0x3b, 0xd4, 0x65, 0xc8, 0xf0, 0x78, 0x95, 0x6a, 0x77, 0x93, 0x27, 0xc4, 0x42, 0xf9, 0x82, + 0xc1, 0x2b, 0x18, 0x44, 0xe8, 0x8a, 0x76, 0xa1, 0x39, 0xb7, 0xac, 0xeb, 0x09, 0xd1, 0xae, 0xc3, + 0xdd, 0x97, 0x79, 0xde, 0x94, 0x1e, 0x3d, 0x86, 0xda, 0xc2, 0x24, 0xb3, 0x99, 0x43, 0x67, 0x0c, + 0x76, 0x7c, 0xce, 0x8d, 0x60, 0xce, 0xc4, 0xb4, 0x16, 0x9e, 0x9f, 0x1f, 0x27, 0xdc, 0xd0, 0x43, + 0x80, 0x58, 0x50, 0xe1, 0xb6, 0xa0, 0x98, 0x13, 0x3a, 0x84, 0xf5, 0x48, 0x62, 0x76, 0x43, 0x7f, + 0x4f, 0xa7, 0xad, 0xe2, 0x6d, 0xb1, 0x59, 0xde, 0x0c, 0xae, 0xba, 0xa9, 0xcd, 0x17, 0x53, 0x8a, + 0xa9, 0x6b, 0xcd, 0x17, 0xbc, 0xb7, 0x52, 0x5b, 0xda, 0x29, 0xe3, 0xb4, 0x41, 0xf9, 0x8b, 0x04, + 0x1b, 0x59, 0xb3, 0x42, 0x7d, 0x58, 0x0b, 0xf4, 0x7c, 0x1b, 0xd4, 0x60, 0xcb, 0xaa, 0xdd, 0xaf, + 0x12, 0x13, 0x0e, 0xad, 0x38, 0x1d, 0x90, 0xce, 0x42, 0x66, 0x01, 0x50, 0xb3, 0xb2, 0x90, 0x99, + 0x8b, 0xd3, 0x01, 0xca, 0x9f, 0xa5, 0x8c, 0xc5, 0xa0, 0x2e, 0x54, 0x05, 0x27, 0xf0, 0xb5, 0x49, + 0x71, 0x38, 0x45, 0x7a, 0x1c, 0x77, 0x42, 0xcf, 0x61, 0x43, 0x88, 0x63, 0xcf, 0x72, 0xc8, 0x8c, + 0x9e, 0x73, 0xd2, 0x10, 0xd0, 0xb9, 0xb3, 0x17, 0x90, 0xc9, 0x5e, 0xc2, 0x8c, 0x33, 0x83, 0x94, + 0x97, 0x19, 0xcd, 0xa1, 0xfb, 0x31, 0x40, 0x4a, 0xd9, 0x67, 0x38, 0x86, 0x43, 0x4e, 0x0e, 0x8e, + 0x6e, 0xb7, 0xf2, 0x6d, 0x99, 0x9d, 0x10, 0x2e, 0x28, 0xbf, 0x85, 0xba, 0xa0, 0x10, 0x41, 0x55, + 0xdf, 0x81, 0xa2, 0x4b, 0x1d, 0x9d, 0x06, 0x07, 0xb3, 0xca, 0x53, 0x8e, 0xb9, 0x0a, 0x0b, 0x13, + 0xfa, 0x3e, 0xac, 0x18, 0xd4, 0x23, 0xa2, 0x97, 0xf5, 0x60, 0xbc, 0x8b, 0xb9, 0x77, 0x46, 0x3d, + 0x32, 0x25, 0x1e, 0xc1, 0xdc, 0x41, 0xf9, 0x20, 0x41, 0x71, 0x9c, 0x8c, 0x91, 0x62, 0x31, 0xbe, + 0x29, 0x19, 0x83, 0x7e, 0x0a, 0xb5, 0x29, 0xd5, 0x2c, 0xc3, 0x76, 0xa8, 0xeb, 0xd2, 0x69, 0x38, + 0x30, 0x16, 0xd0, 0x8f, 0x19, 0xfc, 0xe0, 0xe3, 0x1c, 0x4e, 0xb8, 0xa3, 0xa7, 0x00, 0xb1, 0x60, + 0x39, 0x16, 0x7c, 0xb6, 0x7f, 0x98, 0x0e, 0x8e, 0x39, 0x1f, 0x94, 0x04, 0x89, 0x28, 0xaf, 0xa0, + 0x91, 0x5c, 0x1a, 0x6a, 0x40, 0x5e, 0x9f, 0x0a, 0xc6, 0xc9, 0xeb, 0x53, 0x74, 0x17, 0x2a, 0x9c, + 0x5d, 0x55, 0xdd, 0xa0, 0x82, 0x5a, 0x23, 0x05, 0x6a, 0x41, 0x89, 0x9a, 0x53, 0x6e, 0xf3, 0x8f, + 0x7a, 0x20, 0x2a, 0x13, 0x40, 0xe9, 0x1e, 0xd0, 0x1e, 0x00, 0xab, 0x62, 0x5b, 0xba, 0xe9, 0x05, + 0x83, 0x6f, 0xf8, 0x0d, 0x07, 0x6a, 0x1c, 0xf3, 0x40, 0x77, 0x61, 0xc5, 0x63, 0xf0, 0xce, 0x73, + 0xcf, 0x72, 0xb0, 0xeb, 0x98, 0x6b, 0x95, 0x9f, 0x43, 0x25, 0x0c, 0x63, 0x0b, 0x65, 0xf7, 0x86, + 0xeb, 0x11, 0xc3, 0x16, 0x7c, 0x16, 0x29, 0x92, 0xb4, 0x29, 0x09, 0xda, 0x54, 0x3a, 0x20, 0xab, + 0x64, 0xf6, 0xf9, 0x3c, 0xab, 0xbc, 0x03, 0x94, 0x1e, 0x2e, 0xbb, 0xf5, 0xa2, 0x4e, 0xf9, 0x71, + 0xf4, 0x33, 0x2d, 0x69, 0xd1, 0x4f, 0x18, 0x8e, 0xed, 0xb9, 0xae, 0x91, 0xa0, 0xa3, 0xed, 0xd4, + 0x7e, 0xfd, 0x92, 0xd5, 0x71, 0xb1, 0xef, 0x86, 0x43, 0x7f, 0xe5, 0x18, 0x36, 0x6f, 0x75, 0x43, + 0xf7, 0xa0, 0xec, 0xd2, 0x99, 0x41, 0xa3, 0xa1, 0xae, 0x8a, 0xc4, 0x63, 0xa1, 0xc6, 0xa1, 0x83, + 0xf2, 0x3b, 0x80, 0x48, 0x8f, 0xbe, 0x86, 0xa2, 0x41, 0x9d, 0x19, 0x9d, 0x0a, 0xbc, 0x36, 0x92, + 0x81, 0x58, 0x58, 0xd1, 0x2e, 0x94, 0x17, 0xa6, 0xf0, 0xcc, 0xc7, 0xf6, 0x2d, 0xf2, 0x0c, 0xed, + 0x8a, 0x05, 0x95, 0x50, 0xcd, 0x86, 0x7b, 0x45, 0x49, 0x00, 0x29, 0xfe, 0x9b, 0xe9, 0x3c, 0xa2, + 0xcf, 0xc5, 0x6c, 0xf9, 0xef, 0x24, 0xd0, 0xe4, 0x65, 0xa0, 0xdd, 0x85, 0xca, 0x64, 0x6e, 0x69, + 0xd7, 0x63, 0xfd, 0x3d, 0xe5, 0x64, 0x27, 0xe3, 0x48, 0xa1, 0xfc, 0x5d, 0x82, 0xfa, 0x98, 0x12, + 0x27, 0x7a, 0x21, 0x3c, 0x5a, 0xbe, 0x7b, 0x3f, 0xeb, 0xe6, 0x0f, 0xdf, 0x15, 0xf9, 0x8c, 0x77, + 0x85, 0x1c, 0xbd, 0x2b, 0xbe, 0xf8, 0x85, 0x70, 0x04, 0xf5, 0xb3, 0x7d, 0x95, 0xcc, 0xce, 0x1d, + 0xcb, 0xa6, 0x8e, 0x77, 0x93, 0x3a, 0x6e, 0x69, 0x28, 0xe5, 0xb3, 0xa0, 0xa4, 0x0c, 0x60, 0x35, + 0x9e, 0x88, 0xa1, 0xb0, 0x0b, 0x60, 0x87, 0x92, 0x80, 0x01, 0x12, 0x7b, 0x14, 0x2b, 0x89, 0x63, + 0x5e, 0xca, 0x13, 0xfe, 0x62, 0x09, 0x57, 0xd3, 0x04, 0xf9, 0x9a, 0xde, 0x88, 0xe5, 0xb0, 0x9f, + 0xe8, 0x2b, 0x28, 0x72, 0xe4, 0x07, 0xeb, 0x10, 0x92, 0xd2, 0x83, 0x7a, 0xb2, 0xfa, 0x83, 0x8c, + 0xea, 0xe1, 0xbc, 0x33, 0x6b, 0x7f, 0x90, 0x18, 0xf9, 0xf8, 0x9b, 0x26, 0x38, 0xf9, 0xc7, 0x4b, + 0x8c, 0xe8, 0x6f, 0x1b, 0x5a, 0x4a, 0x93, 0x45, 0x86, 0x3f, 0x4a, 0x90, 0xa1, 0xcf, 0xa4, 0x1b, + 0xa9, 0xe6, 0x53, 0x4c, 0x18, 0x92, 0xb5, 0xfc, 0x09, 0x82, 0x8f, 0x28, 0xf3, 0x1f, 0x12, 0x6c, + 0xb1, 0x73, 0x38, 0xa7, 0x1e, 0xe5, 0x97, 0xab, 0x8f, 0xb8, 0xe0, 0x8e, 0xff, 0x81, 0x78, 0x89, + 0xf9, 0x57, 0xe7, 0x37, 0x78, 0xc2, 0xb8, 0x7b, 0xf4, 0x1c, 0x63, 0x7b, 0xfd, 0x5a, 0x9f, 0x7b, + 0xd4, 0x19, 0x12, 0x83, 0xaa, 0x01, 0xcd, 0xd5, 0xf0, 0x92, 0x36, 0x42, 0xa5, 0x9c, 0x81, 0xca, + 0x95, 0x4c, 0x54, 0x16, 0x3e, 0x85, 0x4a, 0xe5, 0x4f, 0x12, 0xac, 0x67, 0xb4, 0xf1, 0x85, 0x07, + 0xe7, 0x69, 0x54, 0xda, 0x9f, 0xfd, 0xb7, 0x53, 0x8d, 0x27, 0xe7, 0x94, 0x7d, 0x3c, 0xda, 0x50, + 0x56, 0xc9, 0x8c, 0x35, 0xce, 0xbb, 0x66, 0x44, 0xec, 0x63, 0xa9, 0x86, 0x7d, 0x41, 0x79, 0xc4, + 0x3d, 0x38, 0xfb, 0x7d, 0x02, 0xad, 0x72, 0x0c, 0xad, 0x5d, 0xa8, 0x04, 0x51, 0x2e, 0xfa, 0x5e, + 0xe8, 0xe4, 0xa3, 0xb4, 0x1e, 0x34, 0xc7, 0xed, 0x61, 0xcc, 0x5f, 0x25, 0xd8, 0x48, 0xae, 0x5f, + 0x80, 0x74, 0x17, 0x4a, 0x53, 0xfa, 0x9a, 0x2c, 0xe6, 0x5e, 0x82, 0x32, 0xc3, 0x02, 0xc7, 0x39, + 0x1c, 0x38, 0xa0, 0x1f, 0x42, 0x85, 0xaf, 0x7b, 0x64, 0xce, 0x83, 0x07, 0x51, 0x58, 0x8e, 0xb7, + 0x79, 0x9c, 0xc3, 0x91, 0xc7, 0x17, 0xa0, 0xf1, 0x0f, 0xd0, 0x48, 0x3a, 0xa0, 0x6d, 0x00, 0xfa, + 0xee, 0x8a, 0x2c, 0x5c, 0x4f, 0x7f, 0xeb, 0xc3, 0xb0, 0x8c, 0x63, 0x1a, 0xb4, 0x03, 0xe5, 0xdf, + 0x13, 0xc7, 0xd4, 0xcd, 0xf0, 0x5a, 0xad, 0xf1, 0x3a, 0x2f, 0x7d, 0x25, 0x0e, 0xad, 0xa8, 0x0d, + 0x55, 0x27, 0x7c, 0xd5, 0xb2, 0xaf, 0x27, 0x79, 0x47, 0xc6, 0x71, 0x95, 0xf2, 0x04, 0x4a, 0x22, + 0x2c, 0xf3, 0x0e, 0x6d, 0x41, 0xc9, 0xa0, 0xae, 0x4b, 0x66, 0xc1, 0x2d, 0x1a, 0x88, 0xbb, 0x14, + 0xaa, 0xb1, 0xcf, 0x13, 0x54, 0x81, 0xc2, 0xe0, 0xc5, 0x45, 0xef, 0xb4, 0x99, 0x43, 0x35, 0x28, + 0x0f, 0x47, 0xaa, 0x2f, 0x49, 0x08, 0xa0, 0x88, 0x07, 0x47, 0x83, 0x57, 0xe7, 0xcd, 0x3c, 0xaa, + 0x43, 0x65, 0x38, 0x52, 0x85, 0x28, 0x33, 0xd3, 0xe0, 0xd5, 0xc9, 0x58, 0x1d, 0x37, 0x57, 0x84, + 0x49, 0x88, 0x05, 0x54, 0x02, 0xb9, 0x77, 0x7a, 0xda, 0x2c, 0xee, 0x6a, 0x50, 0x8d, 0x3d, 0x5b, + 0x51, 0x0b, 0x36, 0x2e, 0x86, 0xcf, 0x87, 0xa3, 0x97, 0xc3, 0xcb, 0xb3, 0x81, 0x8a, 0x4f, 0x0e, + 0xc7, 0x97, 0xea, 0xaf, 0xce, 0x07, 0xcd, 0x1c, 0xfa, 0x16, 0x6c, 0x5e, 0x0c, 0x7b, 0x47, 0x47, + 0x78, 0x70, 0xd4, 0x53, 0x07, 0xfd, 0xa4, 0x59, 0x42, 0xdf, 0x84, 0x3b, 0xb7, 0x19, 0xf3, 0xbb, + 0x27, 0x50, 0x8b, 0x7f, 0x41, 0x20, 0x04, 0x8d, 0xfe, 0xe0, 0x59, 0xef, 0xe2, 0x54, 0xbd, 0x1c, + 0x9d, 0xab, 0x27, 0xa3, 0x61, 0x33, 0x87, 0xd6, 0xa0, 0xfe, 0x6c, 0x84, 0x0f, 0x07, 0x97, 0x83, + 0x61, 0xef, 0xe0, 0x74, 0xd0, 0x6f, 0x4a, 0xcc, 0xcd, 0x57, 0xf5, 0x4f, 0xc6, 0xbe, 0x2e, 0xbf, + 0x7b, 0x1f, 0x9a, 0xcb, 0x5c, 0x81, 0xaa, 0x50, 0x12, 0xe9, 0x9a, 0x39, 0x26, 0xa8, 0xbd, 0xa3, + 0x61, 0xef, 0x6c, 0xd0, 0x94, 0xba, 0xff, 0x93, 0xa0, 0xc0, 0xbf, 0x2f, 0xd0, 0x43, 0x28, 0xfa, + 0x1f, 0xe2, 0xc8, 0xe7, 0xca, 0xc4, 0x67, 0xfa, 0xd6, 0x7a, 0x42, 0x27, 0x50, 0xfc, 0x00, 0x0a, + 0x9c, 0x18, 0x50, 0x8c, 0x24, 0x82, 0x00, 0x14, 0x57, 0xf9, 0xfe, 0x0f, 0x24, 0xb4, 0xcf, 0x5e, + 0xb8, 0x8c, 0xae, 0x45, 0x91, 0xc4, 0x85, 0xbb, 0xb5, 0x9e, 0xd0, 0x85, 0x41, 0x03, 0xa8, 0xc5, + 0x3b, 0x42, 0xad, 0xdb, 0x78, 0x61, 0x6b, 0x33, 0xc3, 0x12, 0xa4, 0x39, 0xb8, 0xf3, 0xcf, 0x8f, + 0xdb, 0xd2, 0xbf, 0x3e, 0x6e, 0x4b, 0xff, 0xfe, 0xb8, 0x2d, 0xfd, 0xf1, 0x3f, 0xdb, 0xb9, 0x5f, + 0x17, 0xf8, 0x5f, 0x1d, 0x93, 0x22, 0xff, 0x6b, 0x62, 0xff, 0xff, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x66, 0x87, 0x8b, 0xa9, 0x27, 0x11, 0x00, 0x00, } diff --git a/src/query/generated/proto/rpcpb/query.proto b/src/query/generated/proto/rpcpb/query.proto index acc3fa2f6f..a50a243762 100644 --- a/src/query/generated/proto/rpcpb/query.proto +++ b/src/query/generated/proto/rpcpb/query.proto @@ -56,7 +56,7 @@ enum MatcherType { message FetchOptions { int64 limit = 1; - RestrictFetchOptions restrict = 2; + RestrictQueryOptions restrict = 2; int64 lookbackDuration = 3; FanoutOption unaggregated = 4; FanoutOption aggregated = 5; @@ -64,11 +64,21 @@ message FetchOptions { bool includeResolution = 7; } -message RestrictFetchOptions { +message RestrictQueryOptions { + RestrictFetchType RestrictFetchType = 3; + RestrictFetchTags RestrictFetchTags = 4; +} + +message RestrictFetchType { MetricsType metricsType = 1; policypb.StoragePolicy metricsStoragePolicy = 2; } +message RestrictFetchTags { + TagMatchers restrict = 1; + repeated bytes strip = 2; +} + enum MetricsType { UNKNOWN_METRICS_TYPE = 0; UNAGGREGATED_METRICS_TYPE = 1; diff --git a/src/query/models/matcher.go b/src/query/models/matcher.go index 0991ff8b1a..0a10edfb8a 100644 --- a/src/query/models/matcher.go +++ b/src/query/models/matcher.go @@ -57,6 +57,15 @@ func NewMatcher(t MatchType, n, v []byte) (Matcher, error) { Value: v, } + if len(n) == 0 && t != MatchAll { + return Matcher{}, errors.New("name must be set unless using MatchAll") + } + + if len(v) == 0 && !(t == MatchAll || t == MatchField || t == MatchNotField) { + return Matcher{}, errors.New("field must be set unless using one " + + "of MatchField, MatchNotField, or MatchAll") + } + if t == MatchRegexp || t == MatchNotRegexp { re, err := regexp.Compile("^(?:" + string(v) + ")$") if err != nil { diff --git a/src/query/models/matcher_test.go b/src/query/models/matcher_test.go index af60022e57..7bb1de73e5 100644 --- a/src/query/models/matcher_test.go +++ b/src/query/models/matcher_test.go @@ -27,20 +27,12 @@ import ( "github.com/stretchr/testify/require" ) -func newMatcher(t *testing.T, mType MatchType, value string) Matcher { - m, err := NewMatcher(mType, []byte{}, []byte(value)) +func TestMatcherString(t *testing.T) { + m, err := NewMatcher(MatchEqual, []byte("foo"), []byte("bar")) require.NoError(t, err) require.NotNil(t, m) - - return m -} - -func TestMatcher_String(t *testing.T) { - m := newMatcher(t, MatchEqual, "foo") - m.Name = []byte(`key`) - - assert.Equal(t, `key="foo"`, m.String()) - assert.Equal(t, `key="foo"`, (&m).String()) + assert.Equal(t, `foo="bar"`, m.String()) + assert.Equal(t, `foo="bar"`, (&m).String()) } func TestMatchType(t *testing.T) { diff --git a/src/query/storage/fanout/storage.go b/src/query/storage/fanout/storage.go index a2062739ea..81430369f8 100644 --- a/src/query/storage/fanout/storage.go +++ b/src/query/storage/fanout/storage.go @@ -21,6 +21,7 @@ package fanout import ( + "bytes" "context" "fmt" "sync" @@ -332,9 +333,44 @@ func (s *fanoutStorage) CompleteTags( built := accumulatedTags.Build() built.Metadata = metadata + built = applyOptions(built, options) return &built, nil } +func applyOptions( + result storage.CompleteTagsResult, + opts *storage.FetchOptions, +) storage.CompleteTagsResult { + if opts.RestrictQueryOptions == nil { + return result + } + + filter := opts.RestrictQueryOptions.GetRestrictByTag().GetFilterByNames() + if len(filter) > 0 { + // Filter out unwanted tags inplace. + filteredList := result.CompletedTags[:0] + for _, s := range result.CompletedTags { + skip := false + for _, name := range filter { + if bytes.Equal(s.Name, name) { + skip = true + break + } + } + + if skip { + continue + } + + filteredList = append(filteredList, s) + } + + result.CompletedTags = filteredList + } + + return result +} + func (s *fanoutStorage) Write(ctx context.Context, query *storage.WriteQuery) error { // TODO: Consider removing this lookup on every write by maintaining diff --git a/src/query/storage/index.go b/src/query/storage/index.go index 0cff5702dc..58a2790748 100644 --- a/src/query/storage/index.go +++ b/src/query/storage/index.go @@ -123,9 +123,11 @@ func FetchOptionsToAggregateOptions( // FetchQueryToM3Query converts an m3coordinator fetch query to an M3 query. func FetchQueryToM3Query( fetchQuery *FetchQuery, + options *FetchOptions, ) (index.Query, error) { + fetchQuery = fetchQuery.WithAppliedOptions(options) matchers := fetchQuery.TagMatchers - // If no matchers provided, explicitly set this to an AllQuery + // If no matchers provided, explicitly set this to an AllQuery. if len(matchers) == 0 { return index.Query{ Query: idx.NewAllQuery(), diff --git a/src/query/storage/index_test.go b/src/query/storage/index_test.go index c1138063a5..9d7cedbf5d 100644 --- a/src/query/storage/index_test.go +++ b/src/query/storage/index_test.go @@ -201,7 +201,7 @@ func TestFetchQueryToM3Query(t *testing.T) { Interval: 15 * time.Second, } - m3Query, err := FetchQueryToM3Query(fetchQuery) + m3Query, err := FetchQueryToM3Query(fetchQuery, nil) require.NoError(t, err) assert.Equal(t, test.expected, m3Query.String()) }) diff --git a/src/query/storage/m3/cluster_resolver.go b/src/query/storage/m3/cluster_resolver.go index 7955cb08b5..4ddecca550 100644 --- a/src/query/storage/m3/cluster_resolver.go +++ b/src/query/storage/m3/cluster_resolver.go @@ -76,12 +76,12 @@ func resolveClusterNamespacesForQuery( now, start, end time.Time, clusters Clusters, opts *storage.FanoutOptions, - restrict *storage.RestrictFetchOptions, + restrict *storage.RestrictQueryOptions, ) (queryFanoutType, ClusterNamespaces, error) { - if restrict != nil { + if typeRestrict := restrict.GetRestrictByType(); typeRestrict != nil { // If a specific restriction is set, then attempt to satisfy. - return resolveClusterNamespacesForQueryWithRestrictFetchOptions(now, - start, clusters, restrict) + return resolveClusterNamespacesForQueryWithRestrictQueryOptions(now, + start, clusters, *typeRestrict) } // First check if the unaggregated cluster can fully satisfy the query range. @@ -295,18 +295,19 @@ func aggregatedNamespaces( return slices } -// resolveClusterNamespacesForQueryWithRestrictFetchOptions returns the cluster +// resolveClusterNamespacesForQueryWithRestrictQueryOptions returns the cluster // namespace referred to by the restrict fetch options or an error if it // cannot be found. -func resolveClusterNamespacesForQueryWithRestrictFetchOptions( +func resolveClusterNamespacesForQueryWithRestrictQueryOptions( now, start time.Time, clusters Clusters, - restrict *storage.RestrictFetchOptions, + restrict storage.RestrictByType, ) (queryFanoutType, ClusterNamespaces, error) { coversRangeFilter := newCoversRangeFilter(coversRangeFilterOptions{ now: now, queryStart: start, }) + result := func( namespace ClusterNamespace, err error, @@ -314,10 +315,12 @@ func resolveClusterNamespacesForQueryWithRestrictFetchOptions( if err != nil { return 0, nil, err } + if coversRangeFilter(namespace) { return namespaceCoversAllQueryRange, ClusterNamespaces{namespace}, nil } + return namespaceCoversPartialQueryRange, ClusterNamespaces{namespace}, nil } @@ -335,6 +338,7 @@ func resolveClusterNamespacesForQueryWithRestrictFetchOptions( fmt.Errorf("could not find namespace for storage policy: %v", restrict.StoragePolicy.String())) } + return result(ns, nil) default: return result(nil, diff --git a/src/query/storage/m3/cluster_resolver_test.go b/src/query/storage/m3/cluster_resolver_test.go index b93b95facf..c0fa180971 100644 --- a/src/query/storage/m3/cluster_resolver_test.go +++ b/src/query/storage/m3/cluster_resolver_test.go @@ -163,7 +163,7 @@ var testCases = []struct { name string queryLength time.Duration opts *storage.FanoutOptions - restrict *storage.RestrictFetchOptions + restrict *storage.RestrictQueryOptions expectedType queryFanoutType expectedClusterNames []string expectedErr error @@ -311,8 +311,10 @@ var testCases = []struct { { name: "restrict to unaggregated", queryLength: time.Hour * 1000, - restrict: &storage.RestrictFetchOptions{ - MetricsType: storage.UnaggregatedMetricsType, + restrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnaggregatedMetricsType, + }, }, expectedType: namespaceCoversPartialQueryRange, expectedClusterNames: []string{"UNAGG"}, @@ -320,10 +322,12 @@ var testCases = []struct { { name: "restrict to aggregate filtered", queryLength: time.Hour * 1000, - restrict: &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy( - genResolution.String() + ":" + genRetentionFiltered.String()), + restrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy( + genResolution.String() + ":" + genRetentionFiltered.String()), + }, }, expectedType: namespaceCoversPartialQueryRange, expectedClusterNames: []string{"AGG_FILTERED"}, @@ -331,10 +335,12 @@ var testCases = []struct { { name: "restrict to aggregate unfiltered", queryLength: time.Hour * 1000, - restrict: &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy( - genResolution.String() + ":" + genRetentionUnfiltered.String()), + restrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy( + genResolution.String() + ":" + genRetentionUnfiltered.String()), + }, }, expectedType: namespaceCoversPartialQueryRange, expectedClusterNames: []string{"AGG_NO_FILTER"}, @@ -342,17 +348,21 @@ var testCases = []struct { { name: "restrict with unknown metrics type", queryLength: time.Hour * 1000, - restrict: &storage.RestrictFetchOptions{ - MetricsType: storage.UnknownMetricsType, + restrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnknownMetricsType, + }, }, expectedErrContains: "unrecognized metrics type:", }, { name: "restrict with unknown storage policy", queryLength: time.Hour * 1000, - restrict: &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy("1s:100d"), + restrict: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy("1s:100d"), + }, }, expectedErrContains: "could not find namespace for storage policy:", }, diff --git a/src/query/storage/m3/storage.go b/src/query/storage/m3/storage.go index fb488bd9c4..165c126cb3 100644 --- a/src/query/storage/m3/storage.go +++ b/src/query/storage/m3/storage.go @@ -286,7 +286,7 @@ func (s *m3storage) fetchCompressed( default: } - m3query, err := storage.FetchQueryToM3Query(query) + m3query, err := storage.FetchQueryToM3Query(query, options) if err != nil { return nil, err } @@ -301,7 +301,7 @@ func (s *m3storage) fetchCompressed( query.End, s.clusters, options.FanoutOptions, - options.RestrictFetchOptions, + options.RestrictQueryOptions, ) if err != nil { return nil, err @@ -425,7 +425,7 @@ func (s *m3storage) CompleteTags( TagMatchers: query.TagMatchers, } - m3query, err := storage.FetchQueryToM3Query(fetchQuery) + m3query, err := storage.FetchQueryToM3Query(fetchQuery, options) if err != nil { return nil, err } @@ -552,7 +552,7 @@ func (s *m3storage) SearchCompressed( default: } - m3query, err := storage.FetchQueryToM3Query(query) + m3query, err := storage.FetchQueryToM3Query(query, options) if err != nil { return tagResult, noop, err } diff --git a/src/query/storage/restrict_query_options.go b/src/query/storage/restrict_query_options.go new file mode 100644 index 0000000000..19aa847fa7 --- /dev/null +++ b/src/query/storage/restrict_query_options.go @@ -0,0 +1,163 @@ +// Copyright (c) 2019 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 storage + +import ( + "bytes" + "fmt" + + "github.com/m3db/m3/src/metrics/policy" + "github.com/m3db/m3/src/query/models" +) + +// Validate will validate the restrict fetch options. +func (o *RestrictQueryOptions) Validate() error { + if o.RestrictByType != nil { + return o.RestrictByType.Validate() + } + + return nil +} + +// GetRestrictByType provides the type restrictions if present; nil otherwise. +func (o *RestrictQueryOptions) GetRestrictByType() *RestrictByType { + if o == nil { + return nil + } + + return o.RestrictByType +} + +// GetRestrictByTag provides the tag restrictions if present; nil otherwise. +func (o *RestrictQueryOptions) GetRestrictByTag() *RestrictByTag { + if o == nil { + return nil + } + + return o.RestrictByTag +} + +// GetMatchers provides the tag matchers by which results are restricted if +// present; nil otherwise. +func (o *RestrictByTag) GetMatchers() models.Matchers { + if o == nil { + return nil + } + + return o.Restrict +} + +// Validate will validate the restrict type restrictions. +func (o *RestrictByType) Validate() error { + switch o.MetricsType { + case UnaggregatedMetricsType: + if o.StoragePolicy != policy.EmptyStoragePolicy { + return fmt.Errorf( + "expected no storage policy for unaggregated metrics type, "+ + "instead got: %v", o.StoragePolicy.String()) + } + case AggregatedMetricsType: + if v := o.StoragePolicy.Resolution().Window; v <= 0 { + return fmt.Errorf( + "expected positive resolution window, instead got: %v", v) + } + if v := o.StoragePolicy.Resolution().Precision; v <= 0 { + return fmt.Errorf( + "expected positive resolution precision, instead got: %v", v) + } + if v := o.StoragePolicy.Retention().Duration(); v <= 0 { + return fmt.Errorf( + "expected positive retention, instead got: %v", v) + } + default: + return fmt.Errorf( + "unknown metrics type: %v", o.MetricsType) + } + return nil +} + +// GetFilterByNames returns the tag names to filter out of the response. +func (o *RestrictByTag) GetFilterByNames() [][]byte { + if o == nil { + return nil + } + + if o.Strip != nil { + return o.Strip + } + + o.Strip = make([][]byte, 0, len(o.Restrict)) + for _, r := range o.Restrict { + o.Strip = append(o.Strip, r.Name) + } + + return o.Strip +} + +// WithAppliedOptions returns a copy of the fetch query applied options +// that restricts the fetch with respect to labels that must be applied. +func (q *FetchQuery) WithAppliedOptions( + opts *FetchOptions, +) *FetchQuery { + result := *q + if opts == nil { + return &result + } + + restrictOpts := opts.RestrictQueryOptions + if restrictOpts == nil { + return &result + } + + restrict := restrictOpts.GetRestrictByTag().GetMatchers() + if len(restrict) == 0 { + return &result + } + + // Since must apply matchers will always be small (usually 1) + // it's better to not allocate intermediate datastructure and just + // perform n^2 matching. + existing := result.TagMatchers + for _, existingMatcher := range result.TagMatchers { + willBeOverridden := false + for _, matcher := range restrict { + if bytes.Equal(existingMatcher.Name, matcher.Name) { + willBeOverridden = true + break + } + } + + if willBeOverridden { + // We'll override this when we append the restrict matchers. + continue + } + + existing = append(existing, existingMatcher) + } + + // Now append the must apply matchers. + result.TagMatchers = append(existing, restrict...) + return &result +} + +func (q *FetchQuery) String() string { + return q.Raw +} diff --git a/src/query/storage/restrict_query_options_test.go b/src/query/storage/restrict_query_options_test.go new file mode 100644 index 0000000000..f0825001a5 --- /dev/null +++ b/src/query/storage/restrict_query_options_test.go @@ -0,0 +1,53 @@ +// Copyright (c) 2019 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 storage + +import ( + "testing" + + "github.com/m3db/m3/src/query/models" + + "github.com/stretchr/testify/require" +) + +func TestGetRestrict(t *testing.T) { + var opts *RestrictQueryOptions + require.Nil(t, opts.GetRestrictByTag()) + require.Nil(t, opts.GetRestrictByType()) + + opts = &RestrictQueryOptions{} + require.Nil(t, opts.GetRestrictByTag()) + require.Nil(t, opts.GetRestrictByType()) + + opts.RestrictByTag = &RestrictByTag{} + require.NotNil(t, opts.GetRestrictByTag()) + + matcher, err := models.NewMatcher(models.MatchEqual, []byte("f"), []byte("b")) + require.NoError(t, err) + matchers := models.Matchers{matcher} + + opts.RestrictByTag.Restrict = matchers + require.Equal(t, matchers, opts.GetRestrictByTag().GetMatchers()) + + byType := &RestrictByType{} + opts.RestrictByType = byType + require.Equal(t, byType, opts.GetRestrictByType()) +} diff --git a/src/query/storage/types.go b/src/query/storage/types.go index 139bc59600..9aeb599a5b 100644 --- a/src/query/storage/types.go +++ b/src/query/storage/types.go @@ -22,14 +22,12 @@ package storage import ( "context" - "errors" "fmt" "time" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/cost" - "github.com/m3db/m3/src/query/generated/proto/rpcpb" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/ts" xtime "github.com/m3db/m3/src/x/time" @@ -37,10 +35,6 @@ import ( "github.com/uber-go/tally" ) -var ( - errNoRestrictFetchOptionsProtoMsg = errors.New("no restrict fetch options proto message") -) - // Type describes the type of storage. type Type int @@ -107,10 +101,6 @@ type FetchQuery struct { Interval time.Duration `json:"interval"` } -func (q *FetchQuery) String() string { - return q.Raw -} - // FetchOptions represents the options for fetch query. type FetchOptions struct { // Remote is set when this fetch is originated by a remote grpc call. @@ -121,9 +111,9 @@ type FetchOptions struct { BlockType models.FetchedBlockType // FanoutOptions are the options for the fetch namespace fanout. FanoutOptions *FanoutOptions - // RestrictFetchOptions restricts the fetch to a specific set of + // RestrictQueryOptions restricts the fetch to a specific set of // conditions. - RestrictFetchOptions *RestrictFetchOptions + RestrictQueryOptions *RestrictQueryOptions // Step is the configured step size. Step time.Duration // LookbackDuration if set overrides the default lookback duration. @@ -199,18 +189,27 @@ func (o *FetchOptions) QueryFetchOptions( if r.Limit <= 0 { r.Limit = queryCtx.Options.LimitMaxTimeseries } - if r.RestrictFetchOptions == nil && queryCtx.Options.RestrictFetchType != nil { + + // Use inbuilt options for type restriction if none found. + if r.RestrictQueryOptions.GetRestrictByType() == nil && + queryCtx.Options.RestrictFetchType != nil { v := queryCtx.Options.RestrictFetchType - restrict := RestrictFetchOptions{ + restrict := &RestrictByType{ MetricsType: MetricsType(v.MetricsType), StoragePolicy: v.StoragePolicy, } + if err := restrict.Validate(); err != nil { return nil, err } - r.RestrictFetchOptions = &restrict + if r.RestrictQueryOptions == nil { + r.RestrictQueryOptions = &RestrictQueryOptions{} + } + + r.RestrictQueryOptions.RestrictByType = restrict } + return r, nil } @@ -220,8 +219,8 @@ func (o *FetchOptions) Clone() *FetchOptions { return &result } -// RestrictFetchOptions restricts the fetch to a specific set of conditions. -type RestrictFetchOptions struct { +// RestrictByType are specific restrictions to stick to a single data type. +type RestrictByType struct { // MetricsType restricts the type of metrics being returned. MetricsType MetricsType // StoragePolicy is required if metrics type is not unaggregated @@ -229,95 +228,27 @@ type RestrictFetchOptions struct { StoragePolicy policy.StoragePolicy } -// NewRestrictFetchOptionsFromProto returns a restrict fetch options from -// protobuf message. -// TODO: (arnikola) extract these out of types.go -func NewRestrictFetchOptionsFromProto( - p *rpcpb.RestrictFetchOptions, -) (RestrictFetchOptions, error) { - var result RestrictFetchOptions - - if p == nil { - return result, errNoRestrictFetchOptionsProtoMsg - } - - switch p.MetricsType { - case rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE: - result.MetricsType = UnaggregatedMetricsType - case rpcpb.MetricsType_AGGREGATED_METRICS_TYPE: - result.MetricsType = AggregatedMetricsType - } - - if p.MetricsStoragePolicy != nil { - storagePolicy, err := policy.NewStoragePolicyFromProto( - p.MetricsStoragePolicy) - if err != nil { - return result, err - } - - result.StoragePolicy = storagePolicy - } - - // Validate the resulting options. - if err := result.Validate(); err != nil { - return result, err - } - - return result, nil -} - -// Validate will validate the restrict fetch options. -func (o RestrictFetchOptions) Validate() error { - switch o.MetricsType { - case UnaggregatedMetricsType: - if o.StoragePolicy != policy.EmptyStoragePolicy { - return fmt.Errorf( - "expected no storage policy for unaggregated metrics type, "+ - "instead got: %v", o.StoragePolicy.String()) - } - case AggregatedMetricsType: - if v := o.StoragePolicy.Resolution().Window; v <= 0 { - return fmt.Errorf( - "expected positive resolution window, instead got: %v", v) - } - if v := o.StoragePolicy.Resolution().Precision; v <= 0 { - return fmt.Errorf( - "expected positive resolution precision, instead got: %v", v) - } - if v := o.StoragePolicy.Retention().Duration(); v <= 0 { - return fmt.Errorf( - "expected positive retention, instead got: %v", v) - } - default: - return fmt.Errorf( - "unknown metrics type: %v", o.MetricsType) - } - return nil +// RestrictByTag are specific restrictions to enforce behavior for given +// tags. +type RestrictByTag struct { + // Restrict is a set of override matchers to apply to a fetch + // regardless of the existing fetch matchers, they should replace any + // existing matchers part of a fetch if they collide. + Restrict models.Matchers + // Strip is a set of tag names to strip from the response. + // + // NB: If this is unset, but Restrict is set, all tag names appearing in any + // of the Restrict matchers are removed. + Strip [][]byte } -// Proto returns the protobuf message that corresponds to RestrictFetchOptions. -func (o RestrictFetchOptions) Proto() (*rpcpb.RestrictFetchOptions, error) { - if err := o.Validate(); err != nil { - return nil, err - } - - result := &rpcpb.RestrictFetchOptions{} - - switch o.MetricsType { - case UnaggregatedMetricsType: - result.MetricsType = rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE - case AggregatedMetricsType: - result.MetricsType = rpcpb.MetricsType_AGGREGATED_METRICS_TYPE - - storagePolicyProto, err := o.StoragePolicy.Proto() - if err != nil { - return nil, err - } - - result.MetricsStoragePolicy = storagePolicyProto - } - - return result, nil +// RestrictQueryOptions restricts the query to a specific set of conditions. +type RestrictQueryOptions struct { + // RestrictByType are specific restrictions to stick to a single data type. + RestrictByType *RestrictByType + // RestrictByTag are specific restrictions to enforce behavior for given + // tags. + RestrictByTag *RestrictByTag } // Querier handles queries against a storage. diff --git a/src/query/storage/types_test.go b/src/query/storage/types_test.go deleted file mode 100644 index 9774fd34e3..0000000000 --- a/src/query/storage/types_test.go +++ /dev/null @@ -1,214 +0,0 @@ -// Copyright (c) 2019 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 storage - -import ( - "fmt" - "math" - "strings" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/generated/proto/policypb" - "github.com/m3db/m3/src/metrics/policy" - "github.com/m3db/m3/src/query/generated/proto/rpcpb" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func TestNewRestrictFetchOptionsFromProto(t *testing.T) { - tests := []struct { - value *rpcpb.RestrictFetchOptions - expected RestrictFetchOptions - errContains string - }{ - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, - }, - expected: RestrictFetchOptions{ - MetricsType: UnaggregatedMetricsType, - }, - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: int64(time.Minute), - Precision: int64(time.Second), - }, - Retention: &policypb.Retention{ - Period: int64(24 * time.Hour), - }, - }, - }, - expected: RestrictFetchOptions{ - MetricsType: AggregatedMetricsType, - StoragePolicy: policy.NewStoragePolicy(time.Minute, - xtime.Second, 24*time.Hour), - }, - }, - { - value: nil, - errContains: errNoRestrictFetchOptionsProtoMsg.Error(), - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_UNKNOWN_METRICS_TYPE, - }, - errContains: "unknown metrics type:", - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: int64(time.Minute), - Precision: int64(time.Second), - }, - Retention: &policypb.Retention{ - Period: int64(24 * time.Hour), - }, - }, - }, - errContains: "expected no storage policy for unaggregated metrics", - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: -1, - }, - }, - }, - errContains: "unable to convert from duration to time unit", - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: int64(time.Minute), - Precision: int64(-1), - }, - }, - }, - errContains: "unable to convert from duration to time unit", - }, - { - value: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: int64(time.Minute), - Precision: int64(time.Second), - }, - Retention: &policypb.Retention{ - Period: int64(-1), - }, - }, - }, - errContains: "expected positive retention", - }, - } - for _, test := range tests { - t.Run(fmt.Sprintf("%s", test.value), func(t *testing.T) { - result, err := NewRestrictFetchOptionsFromProto(test.value) - if test.errContains == "" { - require.NoError(t, err) - assert.Equal(t, test.expected, result) - return - } - - require.Error(t, err) - assert.True(t, - strings.Contains(err.Error(), test.errContains), - fmt.Sprintf("err=%v, want_contains=%v", err.Error(), test.errContains)) - }) - } -} - -func TestRestrictFetchOptionsProto(t *testing.T) { - tests := []struct { - value RestrictFetchOptions - expected *rpcpb.RestrictFetchOptions - errContains string - }{ - { - value: RestrictFetchOptions{ - MetricsType: UnaggregatedMetricsType, - }, - expected: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, - }, - }, - { - value: RestrictFetchOptions{ - MetricsType: AggregatedMetricsType, - StoragePolicy: policy.NewStoragePolicy(time.Minute, - xtime.Second, 24*time.Hour), - }, - expected: &rpcpb.RestrictFetchOptions{ - MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, - MetricsStoragePolicy: &policypb.StoragePolicy{ - Resolution: &policypb.Resolution{ - WindowSize: int64(time.Minute), - Precision: int64(time.Second), - }, - Retention: &policypb.Retention{ - Period: int64(24 * time.Hour), - }, - }, - }, - }, - { - value: RestrictFetchOptions{ - MetricsType: MetricsType(uint(math.MaxUint16)), - }, - errContains: "unknown metrics type:", - }, - { - value: RestrictFetchOptions{ - MetricsType: UnaggregatedMetricsType, - StoragePolicy: policy.NewStoragePolicy(time.Minute, - xtime.Second, 24*time.Hour), - }, - errContains: "expected no storage policy for unaggregated metrics", - }, - } - for _, test := range tests { - t.Run(fmt.Sprintf("%s", test.value), func(t *testing.T) { - result, err := test.value.Proto() - if test.errContains == "" { - require.NoError(t, err) - assert.Equal(t, test.expected, result) - return - } - - require.Error(t, err) - assert.True(t, strings.Contains(err.Error(), test.errContains)) - }) - } -} diff --git a/src/query/tsdb/remote/codecs.go b/src/query/tsdb/remote/codecs.go index 773850a18d..706866a597 100644 --- a/src/query/tsdb/remote/codecs.go +++ b/src/query/tsdb/remote/codecs.go @@ -27,8 +27,10 @@ import ( "strings" "time" + "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/block" + "github.com/m3db/m3/src/query/generated/proto/rpcpb" rpc "github.com/m3db/m3/src/query/generated/proto/rpcpb" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/storage" @@ -119,6 +121,10 @@ func encodeFetchRequest( } func encodeTagMatchers(modelMatchers models.Matchers) (*rpc.TagMatchers, error) { + if modelMatchers == nil { + return nil, nil + } + matchers := make([]*rpc.TagMatcher, len(modelMatchers)) for i, matcher := range modelMatchers { t, err := encodeMatcherTypeToProto(matcher.Type) @@ -148,7 +154,7 @@ func encodeFanoutOption(opt storage.FanoutOption) (rpc.FanoutOption, error) { return rpc.FanoutOption_FORCE_ENABLED, nil } - return 0, fmt.Errorf("unknown fanout option for proto encoding: %v\n", opt) + return 0, fmt.Errorf("unknown fanout option for proto encoding: %v", opt) } func encodeFetchOptions(options *storage.FetchOptions) (*rpc.FetchOptions, error) { @@ -180,8 +186,8 @@ func encodeFetchOptions(options *storage.FetchOptions) (*rpc.FetchOptions, error } result.AggregatedOptimized = aggOpt - if v := options.RestrictFetchOptions; v != nil { - restrict, err := v.Proto() + if v := options.RestrictQueryOptions; v != nil { + restrict, err := encodeRestrictQueryOptions(v) if err != nil { return nil, err } @@ -196,6 +202,72 @@ func encodeFetchOptions(options *storage.FetchOptions) (*rpc.FetchOptions, error return result, nil } +func encodeRestrictQueryOptionsByType( + o *storage.RestrictByType, +) (*rpcpb.RestrictFetchType, error) { + if o == nil { + return nil, nil + } + + if err := o.Validate(); err != nil { + return nil, err + } + + result := &rpcpb.RestrictFetchType{} + switch o.MetricsType { + case storage.UnaggregatedMetricsType: + result.MetricsType = rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE + case storage.AggregatedMetricsType: + result.MetricsType = rpcpb.MetricsType_AGGREGATED_METRICS_TYPE + + storagePolicyProto, err := o.StoragePolicy.Proto() + if err != nil { + return nil, err + } + + result.MetricsStoragePolicy = storagePolicyProto + } + + return result, nil +} + +func encodeRestrictQueryOptionsByTag( + o *storage.RestrictByTag, +) (*rpcpb.RestrictFetchTags, error) { + if o == nil { + return nil, nil + } + + matchers, err := encodeTagMatchers(o.GetMatchers()) + if err != nil { + return nil, err + } + + return &rpcpb.RestrictFetchTags{ + Restrict: matchers, + Strip: o.Strip, + }, nil +} + +func encodeRestrictQueryOptions( + o *storage.RestrictQueryOptions, +) (*rpcpb.RestrictQueryOptions, error) { + byType, err := encodeRestrictQueryOptionsByType(o.GetRestrictByType()) + if err != nil { + return nil, err + } + + byTags, err := encodeRestrictQueryOptionsByTag(o.GetRestrictByTag()) + if err != nil { + return nil, err + } + + return &rpcpb.RestrictQueryOptions{ + RestrictFetchType: byType, + RestrictFetchTags: byTags, + }, nil +} + func encodeMatcherTypeToProto(t models.MatchType) (rpc.MatcherType, error) { switch t { case models.MatchEqual: @@ -302,7 +374,77 @@ func decodeFanoutOption(opt rpc.FanoutOption) (storage.FanoutOption, error) { return storage.FanoutForceEnable, nil } - return 0, fmt.Errorf("unknown fanout option for proto encoding: %v\n", opt) + return 0, fmt.Errorf("unknown fanout option for proto encoding: %v", opt) +} + +func decodeRestrictQueryOptionsByType( + p *rpc.RestrictFetchType, +) (*storage.RestrictByType, error) { + if p == nil { + return nil, nil + } + + result := &storage.RestrictByType{} + switch p.GetMetricsType() { + case rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE: + result.MetricsType = storage.UnaggregatedMetricsType + case rpcpb.MetricsType_AGGREGATED_METRICS_TYPE: + result.MetricsType = storage.AggregatedMetricsType + } + + if p.GetMetricsStoragePolicy() != nil { + storagePolicy, err := policy.NewStoragePolicyFromProto( + p.MetricsStoragePolicy) + if err != nil { + return result, err + } + + result.StoragePolicy = storagePolicy + } + + if err := result.Validate(); err != nil { + return nil, err + + } + + return result, nil +} + +func decodeRestrictQueryOptionsByTag( + p *rpc.RestrictFetchTags, +) (*storage.RestrictByTag, error) { + if p == nil { + return nil, nil + } + + matchers, err := decodeTagMatchers(p.GetRestrict()) + if err != nil { + return nil, err + } + + return &storage.RestrictByTag{ + Restrict: matchers, + Strip: p.Strip, + }, nil +} + +func decodeRestrictQueryOptions( + p *rpc.RestrictQueryOptions, +) (*storage.RestrictQueryOptions, error) { + byType, err := decodeRestrictQueryOptionsByType(p.GetRestrictFetchType()) + if err != nil { + return nil, err + } + + byTag, err := decodeRestrictQueryOptionsByTag(p.GetRestrictFetchTags()) + if err != nil { + return nil, err + } + + return &storage.RestrictQueryOptions{ + RestrictByType: byType, + RestrictByTag: byTag, + }, nil } func decodeFetchOptions(rpcFetchOptions *rpc.FetchOptions) (*storage.FetchOptions, error) { @@ -336,11 +478,12 @@ func decodeFetchOptions(rpcFetchOptions *rpc.FetchOptions) (*storage.FetchOption } if v := rpcFetchOptions.Restrict; v != nil { - restrict, err := storage.NewRestrictFetchOptionsFromProto(v) + restrict, err := decodeRestrictQueryOptions(v) if err != nil { return nil, err } - result.RestrictFetchOptions = &restrict + + result.RestrictQueryOptions = restrict } if v := rpcFetchOptions.LookbackDuration; v > 0 { diff --git a/src/query/tsdb/remote/codecs_test.go b/src/query/tsdb/remote/codecs_test.go index 1996f18e5c..e0b5a0d8c2 100644 --- a/src/query/tsdb/remote/codecs_test.go +++ b/src/query/tsdb/remote/codecs_test.go @@ -22,18 +22,24 @@ package remote import ( "context" + "fmt" + "math" "net/http" + "strings" "testing" "time" + "github.com/m3db/m3/src/metrics/generated/proto/policypb" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/query/api/v1/handler" + "github.com/m3db/m3/src/query/generated/proto/rpcpb" rpc "github.com/m3db/m3/src/query/generated/proto/rpcpb" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/test" "github.com/m3db/m3/src/query/util/logging" "github.com/m3db/m3/src/x/instrument" + xtime "github.com/m3db/m3/src/x/time" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -126,9 +132,11 @@ func TestEncodeFetchMessage(t *testing.T) { rQ, start, end := createStorageFetchQuery(t) fetchOpts := storage.NewFetchOptions() fetchOpts.Limit = 42 - fetchOpts.RestrictFetchOptions = &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + fetchOpts.RestrictQueryOptions = &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + }, } lookback := time.Minute fetchOpts.LookbackDuration = &lookback @@ -149,11 +157,15 @@ func TestEncodeFetchMessage(t *testing.T) { require.NotNil(t, grpcQ.Options) assert.Equal(t, int64(42), grpcQ.Options.Limit) require.NotNil(t, grpcQ.Options.Restrict) - assert.Equal(t, rpc.MetricsType_AGGREGATED_METRICS_TYPE, grpcQ.Options.Restrict.MetricsType) - require.NotNil(t, grpcQ.Options.Restrict.MetricsStoragePolicy) - expectedStoragePolicyProto, err := fetchOpts.RestrictFetchOptions.StoragePolicy.Proto() + require.NotNil(t, grpcQ.Options.Restrict.RestrictFetchType) + assert.Equal(t, rpc.MetricsType_AGGREGATED_METRICS_TYPE, + grpcQ.Options.Restrict.RestrictFetchType.MetricsType) + require.NotNil(t, grpcQ.Options.Restrict.RestrictFetchType.MetricsStoragePolicy) + expectedStoragePolicyProto, err := fetchOpts.RestrictQueryOptions. + RestrictByType.StoragePolicy.Proto() require.NoError(t, err) - assert.Equal(t, expectedStoragePolicyProto, grpcQ.Options.Restrict.MetricsStoragePolicy) + assert.Equal(t, expectedStoragePolicyProto, grpcQ.Options.Restrict. + RestrictFetchType.MetricsStoragePolicy) assert.Equal(t, lookback, time.Duration(grpcQ.Options.LookbackDuration)) } @@ -161,9 +173,11 @@ func TestEncodeDecodeFetchQuery(t *testing.T) { rQ, _, _ := createStorageFetchQuery(t) fetchOpts := storage.NewFetchOptions() fetchOpts.Limit = 42 - fetchOpts.RestrictFetchOptions = &storage.RestrictFetchOptions{ - MetricsType: storage.AggregatedMetricsType, - StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + fetchOpts.RestrictQueryOptions = &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.MustParseStoragePolicy("1m:14d"), + }, } lookback := time.Minute fetchOpts.LookbackDuration = &lookback @@ -177,10 +191,12 @@ func TestEncodeDecodeFetchQuery(t *testing.T) { require.NoError(t, err) require.NotNil(t, revertedOpts) require.Equal(t, fetchOpts.Limit, revertedOpts.Limit) - require.Equal(t, fetchOpts.RestrictFetchOptions.MetricsType, - revertedOpts.RestrictFetchOptions.MetricsType) - require.Equal(t, fetchOpts.RestrictFetchOptions.StoragePolicy.String(), - revertedOpts.RestrictFetchOptions.StoragePolicy.String()) + require.Equal(t, fetchOpts.RestrictQueryOptions. + RestrictByType.MetricsType, + revertedOpts.RestrictQueryOptions.RestrictByType.MetricsType) + require.Equal(t, fetchOpts.RestrictQueryOptions. + RestrictByType.StoragePolicy.String(), + revertedOpts.RestrictQueryOptions.RestrictByType.StoragePolicy.String()) require.NotNil(t, revertedOpts.LookbackDuration) require.Equal(t, lookback, *revertedOpts.LookbackDuration) @@ -220,3 +236,269 @@ func TestRetrieveMetadata(t *testing.T) { require.Equal(t, requestID, logging.ReadContextID(encodedCtx)) } + +func TestNewRestrictQueryOptionsFromProto(t *testing.T) { + tests := []struct { + value *rpcpb.RestrictQueryOptions + expected *storage.RestrictQueryOptions + errContains string + }{ + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, + }, + }, + expected: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnaggregatedMetricsType, + }, + }, + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: int64(time.Minute), + Precision: int64(time.Second), + }, + Retention: &policypb.Retention{ + Period: int64(24 * time.Hour), + }, + }, + }, + RestrictFetchTags: &rpc.RestrictFetchTags{ + Restrict: &rpc.TagMatchers{ + TagMatchers: []*rpc.TagMatcher{ + newRPCMatcher(rpc.MatcherType_NOTREGEXP, "foo", "bar"), + newRPCMatcher(rpc.MatcherType_EQUAL, "baz", "qux"), + }, + }, + Strip: [][]byte{ + []byte("foobar"), + }, + }, + }, + expected: &storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.NewStoragePolicy(time.Minute, + xtime.Second, 24*time.Hour), + }, + RestrictByTag: &storage.RestrictByTag{ + Restrict: []models.Matcher{ + mustNewMatcher(models.MatchNotRegexp, "foo", "bar"), + mustNewMatcher(models.MatchEqual, "baz", "qux"), + }, + Strip: [][]byte{ + []byte("foobar"), + }, + }, + }, + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_UNKNOWN_METRICS_TYPE, + }, + }, + errContains: "unknown metrics type:", + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: int64(time.Minute), + Precision: int64(time.Second), + }, + Retention: &policypb.Retention{ + Period: int64(24 * time.Hour), + }, + }, + }, + }, + errContains: "expected no storage policy for unaggregated metrics", + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: -1, + }, + }, + }, + }, + errContains: "unable to convert from duration to time unit", + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: int64(time.Minute), + Precision: int64(-1), + }, + }, + }, + }, + errContains: "unable to convert from duration to time unit", + }, + { + value: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: int64(time.Minute), + Precision: int64(time.Second), + }, + Retention: &policypb.Retention{ + Period: int64(-1), + }, + }, + }, + }, + errContains: "expected positive retention", + }, + } + for _, test := range tests { + t.Run(fmt.Sprintf("%+v", test), func(t *testing.T) { + result, err := decodeRestrictQueryOptions(test.value) + if test.errContains == "" { + require.NoError(t, err) + assert.Equal(t, test.expected, result) + return + } + + require.Error(t, err) + assert.True(t, + strings.Contains(err.Error(), test.errContains), + fmt.Sprintf("err=%v, want_contains=%v", err.Error(), test.errContains)) + }) + } +} + +func mustNewMatcher(t models.MatchType, n, v string) models.Matcher { + matcher, err := models.NewMatcher(t, []byte(n), []byte(v)) + if err != nil { + panic(err) + } + + return matcher +} + +func newRPCMatcher(t rpc.MatcherType, n, v string) *rpc.TagMatcher { + return &rpc.TagMatcher{Name: []byte(n), Value: []byte(v), Type: t} +} + +func TestRestrictQueryOptionsProto(t *testing.T) { + tests := []struct { + value storage.RestrictQueryOptions + expected *rpcpb.RestrictQueryOptions + errContains string + }{ + { + value: storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnaggregatedMetricsType, + }, + RestrictByTag: &storage.RestrictByTag{ + Restrict: []models.Matcher{ + mustNewMatcher(models.MatchNotRegexp, "foo", "bar"), + }, + Strip: [][]byte{[]byte("foobar")}, + }, + }, + expected: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_UNAGGREGATED_METRICS_TYPE, + }, + RestrictFetchTags: &rpcpb.RestrictFetchTags{ + Restrict: &rpc.TagMatchers{ + TagMatchers: []*rpc.TagMatcher{ + newRPCMatcher(rpc.MatcherType_NOTREGEXP, "foo", "bar"), + }, + }, + Strip: [][]byte{[]byte("foobar")}, + }, + }, + }, + { + value: storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.AggregatedMetricsType, + StoragePolicy: policy.NewStoragePolicy(time.Minute, + xtime.Second, 24*time.Hour), + }, + RestrictByTag: &storage.RestrictByTag{ + Restrict: models.Matchers{ + mustNewMatcher(models.MatchNotRegexp, "foo", "bar"), + mustNewMatcher(models.MatchEqual, "baz", "qux"), + }, + Strip: [][]byte{[]byte("foobar")}, + }, + }, + expected: &rpcpb.RestrictQueryOptions{ + RestrictFetchType: &rpcpb.RestrictFetchType{ + MetricsType: rpcpb.MetricsType_AGGREGATED_METRICS_TYPE, + MetricsStoragePolicy: &policypb.StoragePolicy{ + Resolution: &policypb.Resolution{ + WindowSize: int64(time.Minute), + Precision: int64(time.Second), + }, + Retention: &policypb.Retention{ + Period: int64(24 * time.Hour), + }, + }, + }, + RestrictFetchTags: &rpcpb.RestrictFetchTags{ + Restrict: &rpc.TagMatchers{ + TagMatchers: []*rpc.TagMatcher{ + newRPCMatcher(rpc.MatcherType_NOTREGEXP, "foo", "bar"), + newRPCMatcher(rpc.MatcherType_EQUAL, "baz", "qux"), + }, + }, + Strip: [][]byte{[]byte("foobar")}, + }, + }, + }, + { + value: storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.MetricsType(uint(math.MaxUint16)), + }, + }, + errContains: "unknown metrics type:", + }, + { + value: storage.RestrictQueryOptions{ + RestrictByType: &storage.RestrictByType{ + MetricsType: storage.UnaggregatedMetricsType, + StoragePolicy: policy.NewStoragePolicy(time.Minute, + xtime.Second, 24*time.Hour), + }, + }, + errContains: "expected no storage policy for unaggregated metrics", + }, + } + for _, test := range tests { + t.Run(fmt.Sprintf("%+v", test.value), func(t *testing.T) { + result, err := encodeRestrictQueryOptions(&test.value) + if test.errContains == "" { + require.NoError(t, err) + require.Equal(t, test.expected, result) + return + } + + require.Error(t, err) + assert.True(t, strings.Contains(err.Error(), test.errContains)) + }) + } +} From f47d35003e436334b11ccd80275bc88d391339a7 Mon Sep 17 00:00:00 2001 From: Matt Schallert Date: Tue, 3 Dec 2019 10:26:38 -0500 Subject: [PATCH 10/11] [query] Allow GET + POST for read APIs (#2055) --- .../v1/handler/prometheus/native/common.go | 22 ++++++++++++++----- .../handler/prometheus/native/common_test.go | 11 ++++++++++ .../api/v1/handler/prometheus/native/read.go | 10 ++++++--- .../prometheus/native/read_instantaneous.go | 10 +++++++-- .../native/read_instantaneous_test.go | 4 ++-- src/query/api/v1/httpd/handler.go | 4 ++-- src/query/api/v1/httpd/handler_test.go | 2 +- 7 files changed, 48 insertions(+), 15 deletions(-) diff --git a/src/query/api/v1/handler/prometheus/native/common.go b/src/query/api/v1/handler/prometheus/native/common.go index 3908f0ce0e..5e71d60698 100644 --- a/src/query/api/v1/handler/prometheus/native/common.go +++ b/src/query/api/v1/handler/prometheus/native/common.go @@ -25,7 +25,6 @@ import ( "io" "math" "net/http" - "net/url" "strconv" "strings" "time" @@ -81,6 +80,10 @@ func parseParams( ) (models.RequestParams, *xhttp.ParseError) { var params models.RequestParams + if err := r.ParseForm(); err != nil { + return params, xhttp.NewParseError(fmt.Errorf(formatErrStr, timeParam, err), http.StatusBadRequest) + } + params.Now = time.Now() if v := r.FormValue(timeParam); v != "" { var err error @@ -202,12 +205,13 @@ func parseInstantaneousParams( fetchOpts *storage.FetchOptions, instrumentOpts instrument.Options, ) (models.RequestParams, *xhttp.ParseError) { + if err := r.ParseForm(); err != nil { + return models.RequestParams{}, xhttp.NewParseError(err, http.StatusBadRequest) + } + if fetchOpts.Step == 0 { fetchOpts.Step = time.Second } - if r.Form == nil { - r.Form = make(url.Values) - } r.Form.Set(startParam, nowTimeValue) r.Form.Set(endParam, nowTimeValue) @@ -221,7 +225,15 @@ func parseInstantaneousParams( } func parseQuery(r *http.Request) (string, error) { - queries, ok := r.URL.Query()[queryParam] + if err := r.ParseForm(); err != nil { + return "", err + } + + // NB(schallert): r.Form is generic over GET and POST requests, with body + // parameters taking precedence over URL parameters (see r.ParseForm() docs + // for more details). We depend on the generic behavior for properly parsing + // POST and GET queries. + queries, ok := r.Form[queryParam] if !ok || len(queries) == 0 || queries[0] == "" { return "", errors.ErrNoQueryFound } diff --git a/src/query/api/v1/handler/prometheus/native/common_test.go b/src/query/api/v1/handler/prometheus/native/common_test.go index c9b928ba8a..5f1376cbf9 100644 --- a/src/query/api/v1/handler/prometheus/native/common_test.go +++ b/src/query/api/v1/handler/prometheus/native/common_test.go @@ -27,6 +27,7 @@ import ( "net/http" "net/http/httptest" "net/url" + "strings" "testing" "time" @@ -86,6 +87,16 @@ func TestParamParsing(t *testing.T) { require.Equal(t, promQuery, r.Query) } +func TestParamParsing_POST(t *testing.T) { + params := defaultParams().Encode() + req := httptest.NewRequest("POST", PromReadURL, strings.NewReader(params)) + req.Header.Add("Content-Type", "application/x-www-form-urlencoded") + + r, err := testParseParams(req) + require.Nil(t, err, "unable to parse request") + require.Equal(t, promQuery, r.Query) +} + func TestInstantaneousParamParsing(t *testing.T) { req := httptest.NewRequest("GET", PromReadURL, nil) params := url.Values{} diff --git a/src/query/api/v1/handler/prometheus/native/read.go b/src/query/api/v1/handler/prometheus/native/read.go index aabd3b78d8..5f545f80f6 100644 --- a/src/query/api/v1/handler/prometheus/native/read.go +++ b/src/query/api/v1/handler/prometheus/native/read.go @@ -49,14 +49,18 @@ const ( // default URL for the query range endpoint found on a Prometheus server PromReadURL = handler.RoutePrefixV1 + "/query_range" - // PromReadHTTPMethod is the HTTP method used with this resource. - PromReadHTTPMethod = http.MethodGet - // TODO: Move to config initialBlockAlloc = 10 ) var ( + // PromReadHTTPMethods is the valid HTTP methods used with this + // resource. + PromReadHTTPMethods = []string{ + http.MethodGet, + http.MethodPost, + } + emptySeriesList = []*ts.Series{} emptyReqParams = models.RequestParams{} ) diff --git a/src/query/api/v1/handler/prometheus/native/read_instantaneous.go b/src/query/api/v1/handler/prometheus/native/read_instantaneous.go index 154b249256..4162e74426 100644 --- a/src/query/api/v1/handler/prometheus/native/read_instantaneous.go +++ b/src/query/api/v1/handler/prometheus/native/read_instantaneous.go @@ -40,9 +40,15 @@ const ( // handler, this matches the default URL for the query endpoint // found on a Prometheus server PromReadInstantURL = handler.RoutePrefixV1 + "/query" +) - // PromReadInstantHTTPMethod is the HTTP method used with this resource. - PromReadInstantHTTPMethod = http.MethodGet +var ( + // PromReadInstantHTTPMethods is the valid HTTP methods used with this + // resource. + PromReadInstantHTTPMethods = []string{ + http.MethodGet, + http.MethodPost, + } ) // PromReadInstantHandler represents a handler for prometheus instantaneous read endpoint. diff --git a/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go b/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go index 9902083dbc..8fe2c2a845 100644 --- a/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go +++ b/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go @@ -106,7 +106,7 @@ func testPromReadInstantHandler( test.NewBlockFromValues(bounds, values) setup.Storage.SetFetchBlocksResult(block.Result{Blocks: []block.Block{b}}, nil) - req := httptest.NewRequest(PromReadInstantHTTPMethod, PromReadInstantURL, nil) + req := httptest.NewRequest(PromReadInstantHTTPMethods[0], PromReadInstantURL, nil) params := url.Values{} params.Set(queryParam, "dummy0{}") @@ -171,7 +171,7 @@ func TestPromReadInstantHandlerStorageError(t *testing.T) { storageErr := fmt.Errorf("storage err") setup.Storage.SetFetchBlocksResult(block.Result{}, storageErr) - req := httptest.NewRequest(PromReadInstantHTTPMethod, PromReadInstantURL, nil) + req := httptest.NewRequest(PromReadInstantHTTPMethods[0], PromReadInstantURL, nil) params := url.Values{} params.Set(queryParam, "dummy0{}") diff --git a/src/query/api/v1/httpd/handler.go b/src/query/api/v1/httpd/handler.go index 658fc60f2e..ba07394194 100644 --- a/src/query/api/v1/httpd/handler.go +++ b/src/query/api/v1/httpd/handler.go @@ -227,11 +227,11 @@ func (h *Handler) RegisterRoutes() error { ).Methods(remote.PromWriteHTTPMethod) h.router.HandleFunc(native.PromReadURL, wrapped(nativePromReadHandler).ServeHTTP, - ).Methods(native.PromReadHTTPMethod) + ).Methods(native.PromReadHTTPMethods...) h.router.HandleFunc(native.PromReadInstantURL, wrapped(native.NewPromReadInstantHandler(h.engine, h.fetchOptionsBuilder, h.tagOptions, h.timeoutOpts, h.instrumentOpts)).ServeHTTP, - ).Methods(native.PromReadInstantHTTPMethod) + ).Methods(native.PromReadInstantHTTPMethods...) // Native M3 search and write endpoints h.router.HandleFunc(handler.SearchURL, diff --git a/src/query/api/v1/httpd/handler_test.go b/src/query/api/v1/httpd/handler_test.go index f08430484b..9380ac15d7 100644 --- a/src/query/api/v1/httpd/handler_test.go +++ b/src/query/api/v1/httpd/handler_test.go @@ -212,7 +212,7 @@ func TestPromNativeReadPost(t *testing.T) { require.NoError(t, err, "unable to setup handler") h.RegisterRoutes() h.Router().ServeHTTP(res, req) - require.Equal(t, res.Code, http.StatusMethodNotAllowed, "POST method not defined") + require.Equal(t, res.Code, http.StatusBadRequest, "Empty request") } func TestJSONWritePost(t *testing.T) { From a585b99328b57720a45a24946da33db9c0535497 Mon Sep 17 00:00:00 2001 From: Andrew Mains Date: Wed, 4 Dec 2019 12:08:49 -0500 Subject: [PATCH 11/11] Make everything accept multiple configs (#2034) # What this PR does / why we need it: Some of our services now accept multiple -f options, and I need it for more of them (specifically aggregator). I went ahead and: -Factored the options out into a package (configflag) - Used the module in all main files that currently use flag with a -f option. I left anything that used a different lib alone. -Added a debug dump option, -d. Usage: "Dump configuration and exit" Does this PR introduce a user-facing and/or backwards incompatible change?: **All**: services now take multiple config files on -f; resulting merged config can be dumped with -d. Does this PR require updating code package or user-facing documentation?: NONE --- src/cmd/services/m3aggregator/main/main.go | 19 +- src/cmd/services/m3collector/main/main.go | 20 +- src/cmd/services/m3coordinator/main/main.go | 22 ++- src/cmd/services/m3ctl/main/main.go | 19 +- src/cmd/services/m3dbnode/main/main.go | 19 +- src/cmd/services/m3query/main/main.go | 21 +- src/collector/server/server.go | 16 +- src/query/server/server.go | 18 +- src/x/config/config.go | 7 + src/x/config/configflag/doc.go | 23 +++ src/x/config/configflag/example_test.go | 69 +++++++ src/x/config/configflag/flag.go | 144 ++++++++++++++ src/x/config/configflag/flag_test.go | 185 ++++++++++++++++++ src/x/config/configflag/generate.go | 23 +++ src/x/config/configflag/os_mock_test.go | 81 ++++++++ src/x/config/configflag/testdata/config1.yaml | 1 + src/x/config/configflag/testdata/config2.yaml | 1 + src/x/config/example_test.go | 12 -- src/x/config/flag.go | 30 --- src/x/config/flag_test.go | 91 --------- 20 files changed, 595 insertions(+), 226 deletions(-) create mode 100644 src/x/config/configflag/doc.go create mode 100644 src/x/config/configflag/example_test.go create mode 100644 src/x/config/configflag/flag.go create mode 100644 src/x/config/configflag/flag_test.go create mode 100644 src/x/config/configflag/generate.go create mode 100644 src/x/config/configflag/os_mock_test.go create mode 100644 src/x/config/configflag/testdata/config1.yaml create mode 100644 src/x/config/configflag/testdata/config2.yaml delete mode 100644 src/x/config/flag_test.go diff --git a/src/cmd/services/m3aggregator/main/main.go b/src/cmd/services/m3aggregator/main/main.go index bb057141b5..739fe62631 100644 --- a/src/cmd/services/m3aggregator/main/main.go +++ b/src/cmd/services/m3aggregator/main/main.go @@ -23,6 +23,7 @@ package main import ( "flag" "fmt" + "log" "os" "os/signal" "syscall" @@ -32,6 +33,7 @@ import ( "github.com/m3db/m3/src/cmd/services/m3aggregator/config" "github.com/m3db/m3/src/cmd/services/m3aggregator/serve" xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" "github.com/m3db/m3/src/x/instrument" @@ -42,25 +44,18 @@ const ( gracefulShutdownTimeout = 15 * time.Second ) -var ( - configFile = flag.String("f", "", "configuration file") -) - func main() { - flag.Parse() + var cfgOpts configflag.Options + cfgOpts.Register() - if len(*configFile) == 0 { - flag.Usage() - os.Exit(1) - } + flag.Parse() // Set globals for etcd related packages. etcd.SetGlobals() var cfg config.Configuration - if err := xconfig.LoadFile(&cfg, *configFile, xconfig.Options{}); err != nil { - fmt.Printf("error loading config file: %v\n", err) - os.Exit(1) + if err := cfgOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) } // Create logger and metrics scope. diff --git a/src/cmd/services/m3collector/main/main.go b/src/cmd/services/m3collector/main/main.go index c2e0eab97e..f06463a65a 100644 --- a/src/cmd/services/m3collector/main/main.go +++ b/src/cmd/services/m3collector/main/main.go @@ -22,29 +22,31 @@ package main import ( "flag" + "log" _ "net/http/pprof" // pprof: for debug listen server if configured - "os" + "github.com/m3db/m3/src/cmd/services/m3collector/config" "github.com/m3db/m3/src/collector/server" + xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" ) -var ( - configFile = flag.String("f", "", "configuration file") -) - func main() { + var configOpts configflag.Options + configOpts.Register() + flag.Parse() - if len(*configFile) == 0 { - flag.Usage() - os.Exit(1) + var cfg config.Configuration + if err := configOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) } // Set globals for etcd related packages. etcd.SetGlobals() server.Run(server.RunOptions{ - ConfigFile: *configFile, + Config: cfg, }) } diff --git a/src/cmd/services/m3coordinator/main/main.go b/src/cmd/services/m3coordinator/main/main.go index 08ea646e38..e3a5542934 100644 --- a/src/cmd/services/m3coordinator/main/main.go +++ b/src/cmd/services/m3coordinator/main/main.go @@ -22,29 +22,31 @@ package main import ( "flag" + "log" _ "net/http/pprof" // pprof: for debug listen server if configured - "os" + "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/query/server" xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" ) -var configFiles xconfig.FlagStringSlice - func main() { - flag.Var(&configFiles, "f", "configuration file(s)") - flag.Parse() + var cfgOpts configflag.Options + cfgOpts.Register() - if len(configFiles) == 0 || len(configFiles[0]) == 0 { - flag.Usage() - os.Exit(1) - } + flag.Parse() // Set globals for etcd related packages. etcd.SetGlobals() + var cfg config.Configuration + if err := cfgOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) + } + server.Run(server.RunOptions{ - ConfigFiles: configFiles, + Config: cfg, }) } diff --git a/src/cmd/services/m3ctl/main/main.go b/src/cmd/services/m3ctl/main/main.go index 93d2ac2f5b..2af1e64d87 100644 --- a/src/cmd/services/m3ctl/main/main.go +++ b/src/cmd/services/m3ctl/main/main.go @@ -23,6 +23,7 @@ package main import ( "flag" "fmt" + "log" "os" "os/signal" "strconv" @@ -36,6 +37,7 @@ import ( "github.com/m3db/m3/src/ctl/service/r2" "github.com/m3db/m3/src/x/clock" xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" "github.com/m3db/m3/src/x/instrument" ) @@ -47,21 +49,20 @@ const ( ) func main() { - configFile := flag.String("f", "m3ctl.yml", "configuration file") - flag.Parse() - - if len(*configFile) == 0 { - flag.Usage() - os.Exit(1) + configOpts := configflag.Options{ + ConfigFiles: configflag.FlagStringSlice{Value: []string{"m3ctl.yml"}}, } + configOpts.Register() + + flag.Parse() + // Set globals for etcd related packages. etcd.SetGlobals() var cfg config.Configuration - if err := xconfig.LoadFile(&cfg, *configFile, xconfig.Options{}); err != nil { - fmt.Printf("error loading config file: %v\n", err) - os.Exit(1) + if err := configOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) } rawLogger, err := cfg.Logging.BuildLogger() diff --git a/src/cmd/services/m3dbnode/main/main.go b/src/cmd/services/m3dbnode/main/main.go index d246ecfe37..1c37497e42 100644 --- a/src/cmd/services/m3dbnode/main/main.go +++ b/src/cmd/services/m3dbnode/main/main.go @@ -23,6 +23,7 @@ package main import ( "flag" "fmt" + "log" _ "net/http/pprof" // pprof: for debug listen server if configured "os" "os/signal" @@ -34,29 +35,23 @@ import ( dbserver "github.com/m3db/m3/src/dbnode/server" coordinatorserver "github.com/m3db/m3/src/query/server" xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" xos "github.com/m3db/m3/src/x/os" ) -var ( - configFile = flag.String("f", "", "configuration file") -) - func main() { - flag.Parse() + var cfgOpts configflag.Options + cfgOpts.Register() - if len(*configFile) == 0 { - flag.Usage() - os.Exit(1) - } + flag.Parse() // Set globals for etcd related packages. etcd.SetGlobals() var cfg config.Configuration - if err := xconfig.LoadFile(&cfg, *configFile, xconfig.Options{}); err != nil { - fmt.Fprintf(os.Stderr, "unable to load config from %s: %v\n", *configFile, err) - os.Exit(1) + if err := cfgOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) } if err := cfg.InitDefaultsAndValidate(); err != nil { diff --git a/src/cmd/services/m3query/main/main.go b/src/cmd/services/m3query/main/main.go index 08ea646e38..6d7e6bd3ec 100644 --- a/src/cmd/services/m3query/main/main.go +++ b/src/cmd/services/m3query/main/main.go @@ -22,29 +22,30 @@ package main import ( "flag" + "log" _ "net/http/pprof" // pprof: for debug listen server if configured - "os" + "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/query/server" xconfig "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" "github.com/m3db/m3/src/x/etcd" ) -var configFiles xconfig.FlagStringSlice - func main() { - flag.Var(&configFiles, "f", "configuration file(s)") - flag.Parse() + var configOpts configflag.Options + configOpts.Register() - if len(configFiles) == 0 || len(configFiles[0]) == 0 { - flag.Usage() - os.Exit(1) - } + flag.Parse() // Set globals for etcd related packages. etcd.SetGlobals() + var cfg config.Configuration + if err := configOpts.MainLoad(&cfg, xconfig.Options{}); err != nil { + log.Fatal(err.Error()) + } server.Run(server.RunOptions{ - ConfigFiles: configFiles, + Config: cfg, }) } diff --git a/src/collector/server/server.go b/src/collector/server/server.go index 1504b8c576..edf0573c62 100644 --- a/src/collector/server/server.go +++ b/src/collector/server/server.go @@ -44,11 +44,7 @@ import ( // RunOptions provides options for running the server // with backwards compatibility if only solely adding fields. type RunOptions struct { - // ConfigFile is the config file to use. - ConfigFile string - - // Config is an alternate way to provide configuration and will be used - // instead of parsing ConfigFile if ConfigFile is not specified. + // Config will be used to configure the application. Config config.Configuration // InterruptCh is a programmatic interrupt channel to supply to @@ -58,15 +54,7 @@ type RunOptions struct { // Run runs the server programmatically given a filename for the configuration file. func Run(runOpts RunOptions) { - var cfg config.Configuration - if runOpts.ConfigFile != "" { - if err := xconfig.LoadFile(&cfg, runOpts.ConfigFile, xconfig.Options{}); err != nil { - fmt.Fprintf(os.Stderr, "unable to load %s: %v", runOpts.ConfigFile, err) - os.Exit(1) - } - } else { - cfg = runOpts.Config - } + cfg := runOpts.Config ctx := context.Background() logger, err := cfg.Logging.Build() diff --git a/src/query/server/server.go b/src/query/server/server.go index 896574b0d3..cb203afecb 100644 --- a/src/query/server/server.go +++ b/src/query/server/server.go @@ -97,10 +97,6 @@ type cleanupFn func() error // RunOptions provides options for running the server // with backwards compatibility if only solely adding fields. type RunOptions struct { - // ConfigFiles is the array of config files to use. All files of the array - // get merged together. - ConfigFiles []string - // Config is an alternate way to provide configuration and will be used // instead of parsing ConfigFile if ConfigFile is not specified. Config config.Configuration @@ -127,19 +123,7 @@ type RunOptions struct { func Run(runOpts RunOptions) { rand.Seed(time.Now().UnixNano()) - var cfg config.Configuration - if len(runOpts.ConfigFiles) > 0 { - err := xconfig.LoadFiles(&cfg, runOpts.ConfigFiles, xconfig.Options{}) - if err != nil { - fmt.Fprintf(os.Stderr, "unable to load %s: %v", runOpts.ConfigFiles, err) - os.Exit(1) - } - - fmt.Fprintf(os.Stdout, "using %s config files: %v", - serviceName, runOpts.ConfigFiles) - } else { - cfg = runOpts.Config - } + cfg := runOpts.Config logger, err := cfg.Logging.BuildLogger() if err != nil { diff --git a/src/x/config/config.go b/src/x/config/config.go index 8da57a7a51..6cbfe2ee77 100644 --- a/src/x/config/config.go +++ b/src/x/config/config.go @@ -23,6 +23,7 @@ package config import ( "errors" + "io" "os" "reflect" "strings" @@ -30,6 +31,7 @@ import ( "go.uber.org/config" "go.uber.org/zap" validator "gopkg.in/validator.v2" + "gopkg.in/yaml.v2" ) const ( @@ -100,6 +102,11 @@ func LoadFiles(dst interface{}, files []string, opts Options) error { return validator.Validate(dst) } +// Dump writes the given configuration to stream dst as YAML. +func Dump(cfg interface{}, dst io.Writer) error { + return yaml.NewEncoder(dst).Encode(cfg) +} + // deprecationCheck checks the config for deprecated fields and returns any in // slice of strings. func deprecationCheck(cfg interface{}, df []string) []string { diff --git a/src/x/config/configflag/doc.go b/src/x/config/configflag/doc.go new file mode 100644 index 0000000000..b25f5b800b --- /dev/null +++ b/src/x/config/configflag/doc.go @@ -0,0 +1,23 @@ +// Copyright (c) 2019 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 configflag provides a utility for registering config related command +// line options with the stdlib flag package. +package configflag diff --git a/src/x/config/configflag/example_test.go b/src/x/config/configflag/example_test.go new file mode 100644 index 0000000000..fbf0947f9f --- /dev/null +++ b/src/x/config/configflag/example_test.go @@ -0,0 +1,69 @@ +// Copyright (c) 2019 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 configflag_test + +import ( + "flag" + "fmt" + + "github.com/m3db/m3/src/x/config" + "github.com/m3db/m3/src/x/config/configflag" +) + +// The FlagStringSlice allows for multiple values when used as a flag variable. +func ExampleFlagStringSlice() { + var configFiles configflag.FlagStringSlice + fs := flag.NewFlagSet("config", flag.PanicOnError) + fs.Var(&configFiles, "f", "config files") + noError(fs.Parse([]string{"-f", "file1.yaml", "-f", "file2.yaml", "-f", "file3.yaml"})) + + fmt.Println("Config files:", configFiles.Value) + // Output: + // Config files: [file1.yaml file2.yaml file3.yaml] +} + +// Options supports registration of config related flags, followed by config +// loading. +func ExampleOptionsRegister() { + var cfgOpts configflag.Options + + var flags flag.FlagSet + + // normal use would use Register() (default flagset) + cfgOpts.RegisterFlagSet(&flags) + + noError(flags.Parse([]string{"-f", "./testdata/config1.yaml", "-f", "./testdata/config2.yaml"})) + + var cfg struct { + Foo int `yaml:"foo"` + Bar string `yaml:"bar"` + } + noError(cfgOpts.MainLoad(&cfg, config.Options{})) + + fmt.Println(cfg) + // Output: {1 bar} +} + +func noError(err error) { + if err != nil { + panic(err.Error()) + } +} diff --git a/src/x/config/configflag/flag.go b/src/x/config/configflag/flag.go new file mode 100644 index 0000000000..2e1f5bd873 --- /dev/null +++ b/src/x/config/configflag/flag.go @@ -0,0 +1,144 @@ +// Copyright (c) 2019 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 configflag + +import ( + "errors" + "flag" + "fmt" + "io" + "os" + + "github.com/m3db/m3/src/x/config" +) + +var _ flag.Value = (*FlagStringSlice)(nil) + +// Options represents the values of config command line flags +type Options struct { + // set by commandline flags + // ConfigFiles (-f) is a list of config files to load + ConfigFiles FlagStringSlice + + // ShouldDumpConfigAndExit (-d) causes MainLoad to print config to stdout, + // and then exit. + ShouldDumpConfigAndExit bool + + // for Usage() + cmd *flag.FlagSet + + // test mocking options + osFns osIface +} + +// Register registers commandline options with the default flagset. +func (opts *Options) Register() { + opts.RegisterFlagSet(flag.CommandLine) +} + +// RegisterFlagSet registers commandline options with the given flagset. +func (opts *Options) RegisterFlagSet(cmd *flag.FlagSet) { + opts.cmd = cmd + + cmd.Var(&opts.ConfigFiles, "f", "Configuration files to load") + cmd.BoolVar(&opts.ShouldDumpConfigAndExit, "d", false, "Dump configuration and exit") +} + +// MainLoad is a convenience method, intended for use in main(), which handles all +// config commandline options. It: +// - Dumps config and exits if -d was passed. +// - Loads configuration otherwise. +// Users who want a subset of this behavior should call individual methods. +func (opts *Options) MainLoad(target interface{}, loadOpts config.Options) error { + osFns := opts.osFns + if osFns == nil { + osFns = realOS{} + } + + if len(opts.ConfigFiles.Value) == 0 { + opts.cmd.Usage() + return errors.New("-f is required (no config files provided)") + } + + if err := config.LoadFiles(target, opts.ConfigFiles.Value, loadOpts); err != nil { + return fmt.Errorf("unable to load config from %s: %v", opts.ConfigFiles.Value, err) + } + + if opts.ShouldDumpConfigAndExit { + if err := config.Dump(target, osFns.Stdout()); err != nil { + return fmt.Errorf("failed to dump config: %v", err) + } + + osFns.Exit(0) + } + return nil +} + +// FlagStringSlice represents a slice of strings. When used as a flag variable, +// it allows for multiple string values. For example, it can be used like this: +// var configFiles FlagStringSlice +// flag.Var(&configFiles, "f", "configuration file(s)") +// Then it can be invoked like this: +// ./app -f file1.yaml -f file2.yaml -f valueN.yaml +// Finally, when the flags are parsed, the variable contains all the values. +type FlagStringSlice struct { + Value []string + + overridden bool +} + +// String() returns a string implementation of the slice. +func (i *FlagStringSlice) String() string { + if i == nil { + return "" + } + return fmt.Sprintf("%v", i.Value) +} + +// Set appends a string value to the slice. +func (i *FlagStringSlice) Set(value string) error { + // on first call, reset + // afterwards, append. This allows better defaulting behavior (defaults + // are overridden by explicitly specified flags). + if !i.overridden { + // make this a new slice. + i.overridden = true + i.Value = nil + } + + i.Value = append(i.Value, value) + return nil +} + +type osIface interface { + Exit(status int) + Stdout() io.Writer +} + +type realOS struct{} + +func (r realOS) Exit(status int) { + os.Exit(status) +} + +func (r realOS) Stdout() io.Writer { + return os.Stdout +} diff --git a/src/x/config/configflag/flag_test.go b/src/x/config/configflag/flag_test.go new file mode 100644 index 0000000000..25cae70922 --- /dev/null +++ b/src/x/config/configflag/flag_test.go @@ -0,0 +1,185 @@ +// Copyright (c) 2019 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 configflag + +import ( + "bytes" + "flag" + "testing" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" + + "github.com/m3db/m3/src/x/config" +) + +type testConfig struct { + Foo int `yaml:"foo"` + Bar string `yaml:"bar"` +} + +func TestCommandLineOptions(t *testing.T) { + type testContext struct { + Flags flag.FlagSet + Opts Options + MockOS *MockosIface + UsageCalled bool + } + + setup := func(t *testing.T) (*testContext, func()) { + ctrl := gomock.NewController(t) + mockOS := NewMockosIface(ctrl) + + tctx := &testContext{ + Opts: Options{osFns: mockOS}, + MockOS: mockOS, + } + + tctx.Flags.Usage = func() { + tctx.UsageCalled = true + } + + tctx.Opts.RegisterFlagSet(&tctx.Flags) + + return tctx, ctrl.Finish + } + + expectedConfig := testConfig{Foo: 1, Bar: "bar"} + configFileOpts := []string{"-f", "./testdata/config1.yaml", "-f", "./testdata/config2.yaml"} + + t.Run("loads config from files", func(t *testing.T) { + tctx, teardown := setup(t) + defer teardown() + + require.NoError(t, tctx.Flags.Parse(configFileOpts)) + + var cfg testConfig + require.NoError(t, tctx.Opts.MainLoad(&cfg, config.Options{})) + assert.Equal(t, expectedConfig, cfg) + }) + + t.Run("dumps config and exits on d", func(t *testing.T) { + tctx, teardown := setup(t) + defer teardown() + + stdout := &bytes.Buffer{} + + tctx.MockOS.EXPECT().Exit(0).Times(1) + tctx.MockOS.EXPECT().Stdout().Return(stdout) + + args := append([]string{"-d"}, configFileOpts...) + require.NoError(t, tctx.Flags.Parse(args)) + + var cfg testConfig + require.NoError(t, tctx.Opts.MainLoad(&cfg, config.Options{})) + + var actual testConfig + require.NoError(t, yaml.NewDecoder(bytes.NewReader(stdout.Bytes())).Decode(&actual)) + assert.Equal(t, expectedConfig, actual) + }) + + t.Run("errors on no configs", func(t *testing.T) { + tctx, teardown := setup(t) + defer teardown() + + require.NoError(t, tctx.Flags.Parse(nil)) + + require.EqualError(t, + tctx.Opts.MainLoad(nil, config.Options{}), + "-f is required (no config files provided)") + assert.True(t, tctx.UsageCalled) + }) +} + +func TestFlagArray(t *testing.T) { + tests := []struct { + args []string + defaults FlagStringSlice + name string + want string + }{ + { + name: "single value", + args: []string{"-f", "./some/file/path/here.yaml"}, + want: "[./some/file/path/here.yaml]", + }, + { + name: "single empty value", + args: []string{"-f", ""}, + want: "[]", + }, + { + name: "two value", + args: []string{"-f", "file1.yaml", "-f", "file2.yaml"}, + want: "[file1.yaml file2.yaml]", + }, + { + name: "two value one of which empty", + args: []string{"-f", "", "-f", "file2.yaml"}, + want: "[ file2.yaml]", + }, + { + name: "three values", + args: []string{"-f", "file1.yaml", "-f", "file2.yaml", "-f", "file3.yaml"}, + want: "[file1.yaml file2.yaml file3.yaml]", + }, + { + name: "default and no flag", + args: nil, + want: "[file1.yaml]", + defaults: FlagStringSlice{Value: []string{"file1.yaml"}}, + }, + { + name: "default is overridden by flag", + args: []string{"-f", "override.yaml"}, + want: "[override.yaml]", + defaults: FlagStringSlice{Value: []string{"file1.yaml"}}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + fs := flag.NewFlagSet(tt.name, flag.PanicOnError) + fs.Var(&tt.defaults, "f", "config files") + err := fs.Parse(tt.args) + require.NoError(t, err, "error parsing flags") + assert.Equal(t, tt.want, tt.defaults.String(), "unexpected output") + }) + } +} + +func TestFlagStringSliceNilFlag(t *testing.T) { + var s *FlagStringSlice + assert.Equal(t, "", s.String(), "nil string slice representation") +} + +func TestFlagStringSliceWithOtherFlags(t *testing.T) { + var values FlagStringSlice + var x string + fs := flag.NewFlagSet("app", flag.PanicOnError) + fs.StringVar(&x, "x", "", "some random var") + fs.Var(&values, "f", "config files") + require.NoError(t, fs.Parse([]string{"-f", "file1.yaml", "-x", "file2.yaml", "-f", "file3.yaml"})) + assert.Equal(t, "[file1.yaml file3.yaml]", values.String(), "flag string slice representation") + assert.Equal(t, "file2.yaml", x, "x value") +} diff --git a/src/x/config/configflag/generate.go b/src/x/config/configflag/generate.go new file mode 100644 index 0000000000..251931c75f --- /dev/null +++ b/src/x/config/configflag/generate.go @@ -0,0 +1,23 @@ +// Copyright (c) 2019 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 configflag + +//go:generate sh -c "mockgen -package=configflag -destination=$GOPATH/src/github.com/m3db/m3/src/x/config/configflag/os_mock_test.go -source=$GOPATH/src/github.com/m3db/m3/src/x/config/configflag/flag.go" diff --git a/src/x/config/configflag/os_mock_test.go b/src/x/config/configflag/os_mock_test.go new file mode 100644 index 0000000000..84c64db8de --- /dev/null +++ b/src/x/config/configflag/os_mock_test.go @@ -0,0 +1,81 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: /Users/andrewmains/Code/gocode/src/github.com/m3db/m3/src/x/config/configflag/flag.go + +// Copyright (c) 2019 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 configflag is a generated GoMock package. +package configflag + +import ( + io "io" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" +) + +// MockosIface is a mock of osIface interface +type MockosIface struct { + ctrl *gomock.Controller + recorder *MockosIfaceMockRecorder +} + +// MockosIfaceMockRecorder is the mock recorder for MockosIface +type MockosIfaceMockRecorder struct { + mock *MockosIface +} + +// NewMockosIface creates a new mock instance +func NewMockosIface(ctrl *gomock.Controller) *MockosIface { + mock := &MockosIface{ctrl: ctrl} + mock.recorder = &MockosIfaceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockosIface) EXPECT() *MockosIfaceMockRecorder { + return m.recorder +} + +// Exit mocks base method +func (m *MockosIface) Exit(status int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Exit", status) +} + +// Exit indicates an expected call of Exit +func (mr *MockosIfaceMockRecorder) Exit(status interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Exit", reflect.TypeOf((*MockosIface)(nil).Exit), status) +} + +// Stdout mocks base method +func (m *MockosIface) Stdout() io.Writer { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Stdout") + ret0, _ := ret[0].(io.Writer) + return ret0 +} + +// Stdout indicates an expected call of Stdout +func (mr *MockosIfaceMockRecorder) Stdout() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stdout", reflect.TypeOf((*MockosIface)(nil).Stdout)) +} diff --git a/src/x/config/configflag/testdata/config1.yaml b/src/x/config/configflag/testdata/config1.yaml new file mode 100644 index 0000000000..1586bde778 --- /dev/null +++ b/src/x/config/configflag/testdata/config1.yaml @@ -0,0 +1 @@ +foo: 1 \ No newline at end of file diff --git a/src/x/config/configflag/testdata/config2.yaml b/src/x/config/configflag/testdata/config2.yaml new file mode 100644 index 0000000000..3907fd8dd7 --- /dev/null +++ b/src/x/config/configflag/testdata/config2.yaml @@ -0,0 +1 @@ +bar: "bar" \ No newline at end of file diff --git a/src/x/config/example_test.go b/src/x/config/example_test.go index 046d6ec79a..603db9bbc2 100644 --- a/src/x/config/example_test.go +++ b/src/x/config/example_test.go @@ -21,7 +21,6 @@ package config_test import ( - "flag" "fmt" "log" @@ -41,14 +40,3 @@ func ExampleLoadFile() { fmt.Printf("listenAddress: %s\n", cfg.ListenAddress) // Output: listenAddress: 0.0.0.0:8392 } - -// The FlagStringSlice allows for multiple values when used as a flag variable. -func ExampleFlagStringSlice() { - var configFiles config.FlagStringSlice - fs := flag.NewFlagSet("config", flag.PanicOnError) - fs.Var(&configFiles, "f", "config files") - fs.Parse([]string{"-f", "file1.yaml", "-f", "file2.yaml", "-f", "file3.yaml"}) - fmt.Println("Config files:", configFiles) - // Output: - // Config files: [file1.yaml file2.yaml file3.yaml] -} diff --git a/src/x/config/flag.go b/src/x/config/flag.go index e5af10472f..77b866ac10 100644 --- a/src/x/config/flag.go +++ b/src/x/config/flag.go @@ -19,33 +19,3 @@ // THE SOFTWARE. package config - -import ( - "flag" - "fmt" -) - -var _ flag.Value = &FlagStringSlice{} - -// FlagStringSlice represents a slice of strings. When used as a flag variable, -// it allows for multiple string values. For example, it can be used like this: -// var configFiles FlagStringSlice -// flag.Var(&configFiles, "f", "configuration file(s)") -// Then it can be invoked like this: -// ./app -f file1.yaml -f file2.yaml -f valueN.yaml -// Finally, when the flags are parsed, the variable contains all the values. -type FlagStringSlice []string - -// String() returns a string implmentation of the slice. -func (i *FlagStringSlice) String() string { - if i == nil { - return "" - } - return fmt.Sprintf("%v", ([]string)(*i)) -} - -// Set appends a string value to the slice. -func (i *FlagStringSlice) Set(value string) error { - *i = append(*i, value) - return nil -} diff --git a/src/x/config/flag_test.go b/src/x/config/flag_test.go deleted file mode 100644 index f248a8799b..0000000000 --- a/src/x/config/flag_test.go +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright (c) 2019 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 config_test - -import ( - "flag" - "testing" - - "github.com/m3db/m3/src/x/config" - - "github.com/stretchr/testify/assert" -) - -func TestFlagArray(t *testing.T) { - tests := []struct { - args []string - name string - want string - }{ - { - name: "single value", - args: []string{"-f", "./some/file/path/here.yaml"}, - want: "[./some/file/path/here.yaml]", - }, - { - name: "single empty value", - args: []string{"-f", ""}, - want: "[]", - }, - { - name: "two value", - args: []string{"-f", "file1.yaml", "-f", "file2.yaml"}, - want: "[file1.yaml file2.yaml]", - }, - { - name: "two value one of which empty", - args: []string{"-f", "", "-f", "file2.yaml"}, - want: "[ file2.yaml]", - }, - { - name: "three values", - args: []string{"-f", "file1.yaml", "-f", "file2.yaml", "-f", "file3.yaml"}, - want: "[file1.yaml file2.yaml file3.yaml]", - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - var values config.FlagStringSlice - fs := flag.NewFlagSet(tt.name, flag.PanicOnError) - fs.Var(&values, "f", "config files") - err := fs.Parse(tt.args) - assert.NoError(t, err, "error parsing flags") - assert.Equal(t, tt.want, values.String(), "unexpected output") - }) - } -} - -func TestFlagStringSliceNilFlag(t *testing.T) { - var s *config.FlagStringSlice - assert.Equal(t, "", s.String(), "nil string slice representation") -} - -func TestFlagStringSliceWithOtherFlags(t *testing.T) { - var values config.FlagStringSlice - var x string - fs := flag.NewFlagSet("app", flag.PanicOnError) - fs.StringVar(&x, "x", "", "some random var") - fs.Var(&values, "f", "config files") - fs.Parse([]string{"-f", "file1.yaml", "-x", "file2.yaml", "-f", "file3.yaml"}) - assert.Equal(t, "[file1.yaml file3.yaml]", values.String(), "flag string slice representation") - assert.Equal(t, "file2.yaml", x, "x value") -}