From 033ef9cf140404b4f1176f1c9c39d917c2a486dc Mon Sep 17 00:00:00 2001 From: Artem Date: Thu, 28 Jan 2021 13:32:50 -0500 Subject: [PATCH] Revert "[dbnode] Emit aggregate usage metrics (#3123)" eb5dfac3e468a0d243f08e3f26d39628eba2c5eb --- src/dbnode/storage/index.go | 15 +-- src/dbnode/storage/index/aggregate_results.go | 107 +--------------- .../storage/index/aggregate_results_test.go | 121 +++--------------- src/dbnode/storage/index/block.go | 12 +- src/dbnode/storage/index/block_test.go | 18 --- src/dbnode/storage/index/index_mock.go | 83 ------------ src/dbnode/storage/index/types.go | 18 --- 7 files changed, 36 insertions(+), 338 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 38dfa00428..581d39b03f 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1404,10 +1404,9 @@ func (i *nsIndex) AggregateQuery( query index.Query, opts index.AggregationOptions, ) (index.AggregateQueryResult, error) { - id := i.nsMetadata.ID() logFields := []opentracinglog.Field{ opentracinglog.String("query", query.String()), - opentracinglog.String("namespace", id.String()), + opentracinglog.String("namespace", i.nsMetadata.ID().String()), opentracinglog.Int("seriesLimit", opts.SeriesLimit), opentracinglog.Int("docsLimit", opts.DocsLimit), xopentracing.Time("queryStart", opts.StartInclusive), @@ -1418,15 +1417,13 @@ func (i *nsIndex) AggregateQuery( sp.LogFields(logFields...) defer sp.Finish() - metrics := index.NewAggregateUsageMetrics(id, i.opts.InstrumentOptions()) // Get results and set the filters, namespace ID and size limit. results := i.aggregateResultsPool.Get() aopts := index.AggregateResultsOptions{ - SizeLimit: opts.SeriesLimit, - DocsLimit: opts.DocsLimit, - FieldFilter: opts.FieldFilter, - Type: opts.Type, - AggregateUsageMetrics: metrics, + SizeLimit: opts.SeriesLimit, + DocsLimit: opts.DocsLimit, + FieldFilter: opts.FieldFilter, + Type: opts.Type, } ctx.RegisterFinalizer(results) // use appropriate fn to query underlying blocks. @@ -1445,7 +1442,7 @@ func (i *nsIndex) AggregateQuery( } } aopts.FieldFilter = aopts.FieldFilter.SortAndDedupe() - results.Reset(id, aopts) + results.Reset(i.nsMetadata.ID(), aopts) exhaustive, err := i.query(ctx, query, results, opts.QueryOptions, fn, logFields) if err != nil { return index.AggregateQueryResult{}, err diff --git a/src/dbnode/storage/index/aggregate_results.go b/src/dbnode/storage/index/aggregate_results.go index bcee7b81c3..fd2809818e 100644 --- a/src/dbnode/storage/index/aggregate_results.go +++ b/src/dbnode/storage/index/aggregate_results.go @@ -24,11 +24,8 @@ import ( "math" "sync" - "github.com/uber-go/tally" - "github.com/m3db/m3/src/m3ninx/index/segment/fst/encoding/docs" "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/pool" ) @@ -45,80 +42,10 @@ type aggregatedResults struct { idPool ident.Pool bytesPool pool.CheckedBytesPool - pool AggregateResultsPool - valuesPool AggregateValuesPool - encodedDocReader docs.EncodedDocumentReader - - iOpts instrument.Options -} - -var _ AggregateUsageMetrics = (*usageMetrics)(nil) - -type usageMetrics struct { - total tally.Counter - - totalTerms tally.Counter - dedupedTerms tally.Counter - - totalFields tally.Counter - dedupedFields tally.Counter -} - -func (m *usageMetrics) IncTotal(val int64) { - // NB: if metrics not set, to valid values, no-op. - if m.total != nil { - m.total.Inc(val) - } -} - -func (m *usageMetrics) IncTotalTerms(val int64) { - // NB: if metrics not set, to valid values, no-op. - if m.totalTerms != nil { - m.totalTerms.Inc(val) - } -} - -func (m *usageMetrics) IncDedupedTerms(val int64) { - // NB: if metrics not set, to valid values, no-op. - if m.dedupedTerms != nil { - m.dedupedTerms.Inc(val) - } -} - -func (m *usageMetrics) IncTotalFields(val int64) { - // NB: if metrics not set, to valid values, no-op. - if m.totalFields != nil { - m.totalFields.Inc(val) - } -} - -func (m *usageMetrics) IncDedupedFields(val int64) { - // NB: if metrics not set, to valid values, no-op. - if m.dedupedFields != nil { - m.dedupedFields.Inc(val) - } -} - -// NewAggregateUsageMetrics builds a new aggregated usage metrics. -func NewAggregateUsageMetrics(ns ident.ID, iOpts instrument.Options) AggregateUsageMetrics { - if ns == nil { - return &usageMetrics{} - } - - scope := iOpts.MetricsScope() - buildCounter := func(val string) tally.Counter { - return scope. - Tagged(map[string]string{"type": val, "namespace": ns.String()}). - Counter("aggregated-results") - } + pool AggregateResultsPool + valuesPool AggregateValuesPool - return &usageMetrics{ - total: buildCounter("total"), - totalTerms: buildCounter("total-terms"), - dedupedTerms: buildCounter("deduped-terms"), - totalFields: buildCounter("total-fields"), - dedupedFields: buildCounter("deduped-fields"), - } + encodedDocReader docs.EncodedDocumentReader } // NewAggregateResults returns a new AggregateResults object. @@ -127,14 +54,9 @@ func NewAggregateResults( aggregateOpts AggregateResultsOptions, opts Options, ) AggregateResults { - if aggregateOpts.AggregateUsageMetrics == nil { - aggregateOpts.AggregateUsageMetrics = &usageMetrics{} - } - return &aggregatedResults{ nsID: namespaceID, aggregateOpts: aggregateOpts, - iOpts: opts.InstrumentOptions(), resultsMap: newAggregateResultsMap(opts.IdentifierPool()), idPool: opts.IdentifierPool(), bytesPool: opts.CheckedBytesPool(), @@ -151,11 +73,8 @@ func (r *aggregatedResults) Reset( ) { r.Lock() - if aggregateOpts.AggregateUsageMetrics == nil { - aggregateOpts.AggregateUsageMetrics = NewAggregateUsageMetrics(nsID, r.iOpts) - } - r.aggregateOpts = aggregateOpts + // finalize existing held nsID if r.nsID != nil { r.nsID.Finalize() @@ -172,6 +91,7 @@ func (r *aggregatedResults) Reset( valueMap := entry.Value() valueMap.finalize() } + // reset all keys in the map next r.resultsMap.Reset() r.totalDocsCount = 0 @@ -190,14 +110,6 @@ func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, int) r.Lock() defer r.Unlock() - // NB: init total count with batch length, since each aggregated entry - // will have one field. - totalCount := len(batch) - for idx := 0; idx < len(batch); idx++ { - totalCount += len(batch[idx].Terms) - } - - r.aggregateOpts.AggregateUsageMetrics.IncTotal(int64(totalCount)) remainingDocs := math.MaxInt64 if r.aggregateOpts.DocsLimit != 0 { remainingDocs = r.aggregateOpts.DocsLimit - r.totalDocsCount @@ -207,9 +119,7 @@ func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, int) if remainingDocs <= 0 { for idx := 0; idx < len(batch); idx++ { batch[idx].Field.Finalize() - r.aggregateOpts.AggregateUsageMetrics.IncTotalFields(1) for _, term := range batch[idx].Terms { - r.aggregateOpts.AggregateUsageMetrics.IncTotalTerms(1) term.Finalize() } } @@ -228,12 +138,9 @@ func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, int) docs := 0 numInserts := 0 for _, entry := range batch { - r.aggregateOpts.AggregateUsageMetrics.IncTotalFields(1) - if docs >= remainingDocs || numInserts >= remainingInserts { entry.Field.Finalize() for _, term := range entry.Terms { - r.aggregateOpts.AggregateUsageMetrics.IncTotalTerms(1) term.Finalize() } @@ -247,8 +154,6 @@ func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, int) aggValues, ok := r.resultsMap.Get(f) if !ok { if remainingInserts > numInserts { - r.aggregateOpts.AggregateUsageMetrics.IncDedupedFields(1) - numInserts++ aggValues = r.valuesPool.Get() // we can avoid the copy because we assume ownership of the passed ident.ID, @@ -270,14 +175,12 @@ func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, int) valuesMap := aggValues.Map() for _, t := range entry.Terms { - r.aggregateOpts.AggregateUsageMetrics.IncTotalTerms(1) if remainingDocs > docs { docs++ if !valuesMap.Contains(t) { // we can avoid the copy because we assume ownership of the passed ident.ID, // but still need to finalize it. if remainingInserts > numInserts { - r.aggregateOpts.AggregateUsageMetrics.IncDedupedTerms(1) valuesMap.SetUnsafe(t, struct{}{}, AggregateValuesMapSetUnsafeOptions{ NoCopyKey: true, NoFinalizeKey: false, diff --git a/src/dbnode/storage/index/aggregate_results_test.go b/src/dbnode/storage/index/aggregate_results_test.go index 085350a023..bf6368ead4 100644 --- a/src/dbnode/storage/index/aggregate_results_test.go +++ b/src/dbnode/storage/index/aggregate_results_test.go @@ -26,10 +26,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/uber-go/tally" "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" xtest "github.com/m3db/m3/src/x/test" ) @@ -64,6 +62,22 @@ func toMap(res AggregateResults) map[string][]string { return resultMap } +func TestAggResultsInsertWithRepeatedFields(t *testing.T) { + res := NewAggregateResults(nil, AggregateResultsOptions{}, testOpts) + entries := entries(genResultsEntry("foo", "baz", "baz", "baz", "qux")) + size, docsCount := res.AddFields(entries) + require.Equal(t, 3, size) + require.Equal(t, 5, docsCount) + require.Equal(t, 3, res.Size()) + require.Equal(t, 5, res.TotalDocsCount()) + + expected := map[string][]string{ + "foo": {"baz", "qux"}, + } + + assert.Equal(t, expected, toMap(res)) +} + func TestWithLimits(t *testing.T) { tests := []struct { name string @@ -73,7 +87,6 @@ func TestWithLimits(t *testing.T) { exSeries int exDocs int expected map[string][]string - exMetrics map[string]int64 }{ { name: "single term", @@ -81,11 +94,6 @@ func TestWithLimits(t *testing.T) { exSeries: 1, exDocs: 1, expected: map[string][]string{"foo": {}}, - - exMetrics: map[string]int64{ - "total": 1, "total-fields": 1, "deduped-fields": 1, - "total-terms": 0, "deduped-terms": 0, - }, }, { name: "same term", @@ -93,10 +101,6 @@ func TestWithLimits(t *testing.T) { exSeries: 1, exDocs: 2, expected: map[string][]string{"foo": {}}, - exMetrics: map[string]int64{ - "total": 2, "total-fields": 2, "deduped-fields": 1, - "total-terms": 0, "deduped-terms": 0, - }, }, { name: "multiple terms", @@ -104,10 +108,6 @@ func TestWithLimits(t *testing.T) { exSeries: 2, exDocs: 2, expected: map[string][]string{"foo": {}, "bar": {}}, - exMetrics: map[string]int64{ - "total": 2, "total-fields": 2, "deduped-fields": 2, - "total-terms": 0, "deduped-terms": 0, - }, }, { name: "single entry", @@ -115,10 +115,6 @@ func TestWithLimits(t *testing.T) { exSeries: 2, exDocs: 2, expected: map[string][]string{"foo": {"bar"}}, - exMetrics: map[string]int64{ - "total": 2, "total-fields": 1, "deduped-fields": 1, - "total-terms": 1, "deduped-terms": 1, - }, }, { name: "single entry multiple fields", @@ -126,10 +122,6 @@ func TestWithLimits(t *testing.T) { exSeries: 4, exDocs: 6, expected: map[string][]string{"foo": {"bar", "baz", "qux"}}, - exMetrics: map[string]int64{ - "total": 6, "total-fields": 1, "deduped-fields": 1, - "total-terms": 5, "deduped-terms": 3, - }, }, { name: "multiple entry multiple fields", @@ -139,10 +131,6 @@ func TestWithLimits(t *testing.T) { exSeries: 4, exDocs: 7, expected: map[string][]string{"foo": {"bar", "baz", "qux"}}, - exMetrics: map[string]int64{ - "total": 7, "total-fields": 2, "deduped-fields": 1, - "total-terms": 5, "deduped-terms": 3, - }, }, { name: "multiple entries", @@ -150,10 +138,6 @@ func TestWithLimits(t *testing.T) { exSeries: 5, exDocs: 5, expected: map[string][]string{"foo": {"baz"}, "bar": {"baz", "qux"}}, - exMetrics: map[string]int64{ - "total": 5, "total-fields": 2, "deduped-fields": 2, - "total-terms": 3, "deduped-terms": 3, - }, }, { @@ -163,10 +147,6 @@ func TestWithLimits(t *testing.T) { exSeries: 4, exDocs: 5, expected: map[string][]string{"foo": {"bar", "baz", "qux"}}, - exMetrics: map[string]int64{ - "total": 5, "total-fields": 1, "deduped-fields": 1, - "total-terms": 4, "deduped-terms": 3, - }, }, { name: "single entry query at doc limit", @@ -175,10 +155,6 @@ func TestWithLimits(t *testing.T) { exSeries: 4, exDocs: 5, expected: map[string][]string{"foo": {"bar", "baz", "qux"}}, - exMetrics: map[string]int64{ - "total": 5, "total-fields": 1, "deduped-fields": 1, - "total-terms": 4, "deduped-terms": 3, - }, }, { @@ -188,10 +164,6 @@ func TestWithLimits(t *testing.T) { exSeries: 3, exDocs: 4, expected: map[string][]string{"foo": {"bar", "baz"}}, - exMetrics: map[string]int64{ - "total": 4, "total-fields": 1, "deduped-fields": 1, - "total-terms": 3, "deduped-terms": 2, - }, }, { name: "single entry query below doc limit", @@ -200,11 +172,8 @@ func TestWithLimits(t *testing.T) { exSeries: 2, exDocs: 3, expected: map[string][]string{"foo": {"bar"}}, - exMetrics: map[string]int64{ - "total": 5, "total-fields": 1, "deduped-fields": 1, - "total-terms": 4, "deduped-terms": 1, - }, }, + { name: "multiple entry query below series limit", entries: entries(genResultsEntry("foo", "bar"), genResultsEntry("baz", "qux")), @@ -212,10 +181,6 @@ func TestWithLimits(t *testing.T) { exSeries: 3, exDocs: 4, expected: map[string][]string{"foo": {"bar"}, "baz": {}}, - exMetrics: map[string]int64{ - "total": 4, "total-fields": 2, "deduped-fields": 2, - "total-terms": 2, "deduped-terms": 1, - }, }, { name: "multiple entry query below doc limit", @@ -224,11 +189,8 @@ func TestWithLimits(t *testing.T) { exSeries: 3, exDocs: 3, expected: map[string][]string{"foo": {"bar"}, "baz": {}}, - exMetrics: map[string]int64{ - "total": 4, "total-fields": 2, "deduped-fields": 2, - "total-terms": 2, "deduped-terms": 1, - }, }, + { name: "multiple entry query both limits", entries: entries(genResultsEntry("foo", "bar"), genResultsEntry("baz", "qux")), @@ -237,21 +199,14 @@ func TestWithLimits(t *testing.T) { exSeries: 3, exDocs: 3, expected: map[string][]string{"foo": {"bar"}, "baz": {}}, - exMetrics: map[string]int64{ - "total": 4, "total-fields": 2, "deduped-fields": 2, - "total-terms": 2, "deduped-terms": 1, - }, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - scope := tally.NewTestScope("", nil) - iOpts := instrument.NewOptions().SetMetricsScope(scope) - res := NewAggregateResults(ident.StringID("ns"), AggregateResultsOptions{ - SizeLimit: tt.sizeLimit, - DocsLimit: tt.docLimit, - AggregateUsageMetrics: NewAggregateUsageMetrics(ident.StringID("ns"), iOpts), + res := NewAggregateResults(nil, AggregateResultsOptions{ + SizeLimit: tt.sizeLimit, + DocsLimit: tt.docLimit, }, testOpts) size, docsCount := res.AddFields(tt.entries) @@ -261,14 +216,6 @@ func TestWithLimits(t *testing.T) { assert.Equal(t, tt.exDocs, res.TotalDocsCount()) assert.Equal(t, tt.expected, toMap(res)) - - counters := scope.Snapshot().Counters() - actualCounters := make(map[string]int64, len(counters)) - for _, v := range counters { - actualCounters[v.Tags()["type"]] = v.Value() - } - - assert.Equal(t, tt.exMetrics, actualCounters) }) } } @@ -348,29 +295,3 @@ func TestAggResultFinalize(t *testing.T) { require.False(t, id.IsNoFinalize()) } } - -func TestResetUpdatesMetics(t *testing.T) { - scope := tally.NewTestScope("", nil) - iOpts := instrument.NewOptions().SetMetricsScope(scope) - testOpts = testOpts.SetInstrumentOptions(iOpts) - res := NewAggregateResults(nil, AggregateResultsOptions{ - AggregateUsageMetrics: NewAggregateUsageMetrics(ident.StringID("ns1"), iOpts), - }, testOpts) - res.AddFields(entries(genResultsEntry("foo"))) - res.Reset(ident.StringID("ns2"), AggregateResultsOptions{}) - res.AddFields(entries(genResultsEntry("bar"))) - - counters := scope.Snapshot().Counters() - seenNamespaces := make(map[string]struct{}) - for _, v := range counters { - ns := v.Tags()["namespace"] - seenNamespaces[ns] = struct{}{} - } - - assert.Equal(t, map[string]struct{}{ - "ns1": {}, - "ns2": {}, - }, seenNamespaces) - - res.Finalize() -} diff --git a/src/dbnode/storage/index/block.go b/src/dbnode/storage/index/block.go index 09180c28fc..86061f3a9c 100644 --- a/src/dbnode/storage/index/block.go +++ b/src/dbnode/storage/index/block.go @@ -142,7 +142,6 @@ type block struct { nsMD namespace.Metadata namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager docsLimit limits.LookbackLimit - aggregatedAddedCounter tally.Counter metrics blockMetrics logger *zap.Logger @@ -226,7 +225,6 @@ func NewBlock( iopts, ) - aggAdded := opts.InstrumentOptions().MetricsScope().Counter("aggregate-added-counter") // NB(bodu): The length of coldMutableSegments is always at least 1. coldSegs := []*mutableSegments{ newMutableSegments( @@ -253,7 +251,6 @@ func NewBlock( metrics: newBlockMetrics(scope), logger: iopts.Logger(), docsLimit: opts.QueryLimits().DocsLimit(), - aggregatedAddedCounter: aggAdded, } b.newFieldsAndTermsIteratorFn = newFieldsAndTermsIterator b.newExecutorWithRLockFn = b.executorWithRLock @@ -710,7 +707,7 @@ func (b *block) aggregateWithSpan( continue } - batch, size, resultCount, err = b.addAggregateResults(cancellable, results, batch, source, numAdded) + batch, size, resultCount, err = b.addAggregateResults(cancellable, results, batch, source, currBatchSize) if err != nil { return false, err } @@ -730,7 +727,7 @@ func (b *block) aggregateWithSpan( // Add last batch to results if remaining. for len(batch) > 0 { - batch, size, resultCount, err = b.addAggregateResults(cancellable, results, batch, source, numAdded) + batch, size, resultCount, err = b.addAggregateResults(cancellable, results, batch, source, currBatchSize) if err != nil { return false, err } @@ -822,12 +819,11 @@ func (b *block) addAggregateResults( results AggregateResults, batch []AggregateResultsEntry, source []byte, - numAdded int, + currBatchSize int, ) ([]AggregateResultsEntry, int, int, error) { // update recently queried docs to monitor memory. if results.EnforceLimits() { - b.aggregatedAddedCounter.Inc(int64(numAdded)) - if err := b.docsLimit.Inc(len(batch), source); err != nil { + if err := b.docsLimit.Inc(currBatchSize, source); err != nil { return batch, 0, 0, err } } diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index d8065a3c89..a45687da68 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/index/compaction" - "github.com/m3db/m3/src/dbnode/storage/limits" "github.com/m3db/m3/src/dbnode/test" "github.com/m3db/m3/src/dbnode/tracepoint" "github.com/m3db/m3/src/m3ninx/doc" @@ -41,10 +40,8 @@ import ( "github.com/m3db/m3/src/m3ninx/search" "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" xresource "github.com/m3db/m3/src/x/resource" - "github.com/m3db/m3/src/x/tallytest" xtime "github.com/m3db/m3/src/x/time" "github.com/golang/mock/gomock" @@ -52,7 +49,6 @@ import ( opentracinglog "github.com/opentracing/opentracing-go/log" "github.com/opentracing/opentracing-go/mocktracer" "github.com/stretchr/testify/require" - "github.com/uber-go/tally" "go.uber.org/zap" ) @@ -1870,16 +1866,6 @@ func TestBlockAggregate(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - scope := tally.NewTestScope("", nil) - iOpts := instrument.NewOptions().SetMetricsScope(scope) - limitOpts := limits.NewOptions(). - SetInstrumentOptions(iOpts). - SetDocsLimitOpts(limits.LookbackLimitOptions{Limit: 50, Lookback: time.Minute}). - SetBytesReadLimitOpts(limits.LookbackLimitOptions{Lookback: time.Minute}) - queryLimits, err := limits.NewQueryLimits((limitOpts)) - require.NoError(t, err) - testOpts = testOpts.SetInstrumentOptions(iOpts).SetQueryLimits(queryLimits) - // NB: seriesLimit must be higher than the number of fields to be exhaustive. seriesLimit := 10 testMD := newTestNSMetadata(t) @@ -1947,10 +1933,6 @@ func TestBlockAggregate(t *testing.T) { spans := mtr.FinishedSpans() require.Len(t, spans, 2) require.Equal(t, tracepoint.BlockAggregate, spans[0].OperationName) - - snap := scope.Snapshot() - tallytest.AssertCounterValue(t, 4, snap, "query-limit.total-docs-matched", nil) - tallytest.AssertCounterValue(t, 8, snap, "aggregate-added-counter", nil) } func TestBlockAggregateNotExhaustive(t *testing.T) { diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index a4d097f18a..ab6af6aded 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -587,89 +587,6 @@ func (mr *MockAggregateResultsMockRecorder) Map() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Map", reflect.TypeOf((*MockAggregateResults)(nil).Map)) } -// MockAggregateUsageMetrics is a mock of AggregateUsageMetrics interface -type MockAggregateUsageMetrics struct { - ctrl *gomock.Controller - recorder *MockAggregateUsageMetricsMockRecorder -} - -// MockAggregateUsageMetricsMockRecorder is the mock recorder for MockAggregateUsageMetrics -type MockAggregateUsageMetricsMockRecorder struct { - mock *MockAggregateUsageMetrics -} - -// NewMockAggregateUsageMetrics creates a new mock instance -func NewMockAggregateUsageMetrics(ctrl *gomock.Controller) *MockAggregateUsageMetrics { - mock := &MockAggregateUsageMetrics{ctrl: ctrl} - mock.recorder = &MockAggregateUsageMetricsMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockAggregateUsageMetrics) EXPECT() *MockAggregateUsageMetricsMockRecorder { - return m.recorder -} - -// IncTotal mocks base method -func (m *MockAggregateUsageMetrics) IncTotal(val int64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "IncTotal", val) -} - -// IncTotal indicates an expected call of IncTotal -func (mr *MockAggregateUsageMetricsMockRecorder) IncTotal(val interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IncTotal", reflect.TypeOf((*MockAggregateUsageMetrics)(nil).IncTotal), val) -} - -// IncTotalTerms mocks base method -func (m *MockAggregateUsageMetrics) IncTotalTerms(val int64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "IncTotalTerms", val) -} - -// IncTotalTerms indicates an expected call of IncTotalTerms -func (mr *MockAggregateUsageMetricsMockRecorder) IncTotalTerms(val interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IncTotalTerms", reflect.TypeOf((*MockAggregateUsageMetrics)(nil).IncTotalTerms), val) -} - -// IncDedupedTerms mocks base method -func (m *MockAggregateUsageMetrics) IncDedupedTerms(val int64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "IncDedupedTerms", val) -} - -// IncDedupedTerms indicates an expected call of IncDedupedTerms -func (mr *MockAggregateUsageMetricsMockRecorder) IncDedupedTerms(val interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IncDedupedTerms", reflect.TypeOf((*MockAggregateUsageMetrics)(nil).IncDedupedTerms), val) -} - -// IncTotalFields mocks base method -func (m *MockAggregateUsageMetrics) IncTotalFields(val int64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "IncTotalFields", val) -} - -// IncTotalFields indicates an expected call of IncTotalFields -func (mr *MockAggregateUsageMetricsMockRecorder) IncTotalFields(val interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IncTotalFields", reflect.TypeOf((*MockAggregateUsageMetrics)(nil).IncTotalFields), val) -} - -// IncDedupedFields mocks base method -func (m *MockAggregateUsageMetrics) IncDedupedFields(val int64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "IncDedupedFields", val) -} - -// IncDedupedFields indicates an expected call of IncDedupedFields -func (mr *MockAggregateUsageMetricsMockRecorder) IncDedupedFields(val interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IncDedupedFields", reflect.TypeOf((*MockAggregateUsageMetrics)(nil).IncDedupedFields), val) -} - // MockAggregateResultsPool is a mock of AggregateResultsPool interface type MockAggregateResultsPool struct { ctrl *gomock.Controller diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index 01fa78f609..e54acefc2c 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -278,24 +278,6 @@ type AggregateResultsOptions struct { // RestrictByQuery is a query to restrict the set of documents that must // be present for an aggregated term to be returned. RestrictByQuery *Query - - // AggregateUsageMetrics are aggregate usage metrics that track field - // and term counts for aggregate queries. - AggregateUsageMetrics AggregateUsageMetrics -} - -// AggregateUsageMetrics are metrics for aggregate query usage. -type AggregateUsageMetrics interface { - // IncTotal increments the total metric count. - IncTotal(val int64) - // IncTotalTerms increments the totalTerms metric count. - IncTotalTerms(val int64) - // IncDedupedTerms increments the dedupedTerms metric count. - IncDedupedTerms(val int64) - // IncTotalFields increments the totalFields metric count. - IncTotalFields(val int64) - // IncDedupedFields increments the dedupedFields metric count. - IncDedupedFields(val int64) } // AggregateResultsAllocator allocates AggregateResults types.