diff --git a/.ci b/.ci index 15209040a6..b045911f33 160000 --- a/.ci +++ b/.ci @@ -1 +1 @@ -Subproject commit 15209040a6432a3280c1e2ed2d55ebd520ebe723 +Subproject commit b045911f33a3396918b4120a59bd83c54e2d9034 diff --git a/kube/bundle.yaml b/kube/bundle.yaml index bc2bd61b36..60f9eb57fd 100644 --- a/kube/bundle.yaml +++ b/kube/bundle.yaml @@ -117,11 +117,6 @@ data: m3dbnode.yml: |+ coordinator: listenAddress: 0.0.0.0:7201 - local: - namespaces: - - namespace: default - type: unaggregated - retention: 48h logging: level: info metrics: diff --git a/kube/m3dbnode-configmap.yaml b/kube/m3dbnode-configmap.yaml index 0536215c4a..42135e5841 100644 --- a/kube/m3dbnode-configmap.yaml +++ b/kube/m3dbnode-configmap.yaml @@ -7,11 +7,6 @@ data: m3dbnode.yml: |+ coordinator: listenAddress: 0.0.0.0:7201 - local: - namespaces: - - namespace: default - type: unaggregated - retention: 48h logging: level: info metrics: diff --git a/kube/terraform/main.tf b/kube/terraform/main.tf index 6391658abb..a6b76bbbfe 100755 --- a/kube/terraform/main.tf +++ b/kube/terraform/main.tf @@ -133,7 +133,7 @@ resource "kubernetes_config_map" "m3dbnode_config" { namespace = "m3db" } data { - m3dbnode.yml = "coordinator:\n listenAddress: \"0.0.0.0:7201\"\n local:\n namespaces:\n - namespace: default\n type: unaggregated\n retention: 48h\n metrics:\n scope:\n prefix: \"coordinator\"\n prometheus:\n handlerPath: /metrics\n listenAddress: 0.0.0.0:7203\n sanitization: prometheus\n samplingRate: 1.0\n extended: none\n tagOptions:\n idScheme: quoted\n\ndb:\n logging:\n level: info\n\n metrics:\n prometheus:\n handlerPath: /metrics\n sanitization: prometheus\n samplingRate: 1.0\n extended: detailed\n\n listenAddress: 0.0.0.0:9000\n clusterListenAddress: 0.0.0.0:9001\n httpNodeListenAddress: 0.0.0.0:9002\n httpClusterListenAddress: 0.0.0.0:9003\n debugListenAddress: 0.0.0.0:9004\n\n hostID:\n resolver: hostname\n\n client:\n writeConsistencyLevel: majority\n readConsistencyLevel: unstrict_majority\n\n gcPercentage: 100\n\n writeNewSeriesAsync: true\n writeNewSeriesBackoffDuration: 2ms\n\n commitlog:\n flushMaxBytes: 524288\n flushEvery: 1s\n queue:\n calculationType: fixed\n size: 2097152\n\n filesystem:\n filePathPrefix: /var/lib/m3db\n\n config:\n service:\n env: default_env\n zone: embedded\n service: m3db\n cacheDir: /var/lib/m3kv\n etcdClusters:\n - zone: embedded\n endpoints:\n - http://etcd-0.etcd:2379\n - http://etcd-1.etcd:2379\n - http://etcd-2.etcd:2379\n" + m3dbnode.yml = "coordinator:\n listenAddress: \"0.0.0.0:7201\"\n metrics:\n scope:\n prefix: \"coordinator\"\n prometheus:\n handlerPath: /metrics\n listenAddress: 0.0.0.0:7203\n sanitization: prometheus\n samplingRate: 1.0\n extended: none\n tagOptions:\n idScheme: quoted\n\ndb:\n logging:\n level: info\n\n metrics:\n prometheus:\n handlerPath: /metrics\n sanitization: prometheus\n samplingRate: 1.0\n extended: detailed\n\n listenAddress: 0.0.0.0:9000\n clusterListenAddress: 0.0.0.0:9001\n httpNodeListenAddress: 0.0.0.0:9002\n httpClusterListenAddress: 0.0.0.0:9003\n debugListenAddress: 0.0.0.0:9004\n\n hostID:\n resolver: hostname\n\n client:\n writeConsistencyLevel: majority\n readConsistencyLevel: unstrict_majority\n\n gcPercentage: 100\n\n writeNewSeriesAsync: true\n writeNewSeriesBackoffDuration: 2ms\n\n commitlog:\n flushMaxBytes: 524288\n flushEvery: 1s\n queue:\n calculationType: fixed\n size: 2097152\n\n filesystem:\n filePathPrefix: /var/lib/m3db\n\n config:\n service:\n env: default_env\n zone: embedded\n service: m3db\n cacheDir: /var/lib/m3kv\n etcdClusters:\n - zone: embedded\n endpoints:\n - http://etcd-0.etcd:2379\n - http://etcd-1.etcd:2379\n - http://etcd-2.etcd:2379\n" } } diff --git a/scripts/docker-integration-tests/m3aggregator.Dockerfile b/scripts/docker-integration-tests/m3aggregator.Dockerfile index d0dce08d1a..e64339e041 100644 --- a/scripts/docker-integration-tests/m3aggregator.Dockerfile +++ b/scripts/docker-integration-tests/m3aggregator.Dockerfile @@ -8,5 +8,7 @@ ADD ./m3aggregator.yml /etc/m3aggregator/m3aggregator.yml EXPOSE 6000-6001/tcp +ENV PANIC_ON_INVARIANT_VIOLATED=true + ENTRYPOINT [ "/bin/m3aggregator" ] CMD [ "-f", "/etc/m3aggregator/m3aggregator.yml" ] diff --git a/scripts/docker-integration-tests/m3coordinator.Dockerfile b/scripts/docker-integration-tests/m3coordinator.Dockerfile index afaae86723..0319613977 100644 --- a/scripts/docker-integration-tests/m3coordinator.Dockerfile +++ b/scripts/docker-integration-tests/m3coordinator.Dockerfile @@ -8,5 +8,7 @@ ADD ./m3coordinator-local-etcd.yml /etc/m3coordinator/m3coordinator.yml EXPOSE 7201/tcp 7203/tcp +ENV PANIC_ON_INVARIANT_VIOLATED=true + ENTRYPOINT [ "/bin/m3coordinator" ] CMD [ "-f", "/etc/m3coordinator/m3coordinator.yml" ] diff --git a/scripts/docker-integration-tests/m3dbnode.Dockerfile b/scripts/docker-integration-tests/m3dbnode.Dockerfile index d928221b94..a352ad4bf5 100644 --- a/scripts/docker-integration-tests/m3dbnode.Dockerfile +++ b/scripts/docker-integration-tests/m3dbnode.Dockerfile @@ -8,5 +8,7 @@ ADD ./m3dbnode-local-etcd.yml /etc/m3dbnode/m3dbnode.yml EXPOSE 2379/tcp 2380/tcp 7201/tcp 7203/tcp 9000-9004/tcp +ENV PANIC_ON_INVARIANT_VIOLATED=true + ENTRYPOINT [ "/bin/m3dbnode" ] CMD [ "-f", "/etc/m3dbnode/m3dbnode.yml" ] diff --git a/scripts/dtest/docker-compose.yml b/scripts/dtest/docker-compose.yml index ba8502c42e..921055276d 100755 --- a/scripts/dtest/docker-compose.yml +++ b/scripts/dtest/docker-compose.yml @@ -10,6 +10,8 @@ services: - "0.0.0.0:9000:9000" volumes: - "./m3dbnode.yml:/etc/m3dbnode/m3dbnode.yml" + env: + - PANIC_ON_INVARIANT_VIOLATED=true coord01: networks: - dtest @@ -20,5 +22,7 @@ services: - "0.0.0.0:7204:7204" volumes: - "./m3coordinator.yml:/etc/m3coordinator/m3coordinator.yml" + env: + - PANIC_ON_INVARIANT_VIOLATED=true networks: dtest: diff --git a/scripts/dtest/m3dbnode.yml b/scripts/dtest/m3dbnode.yml index 28627c78f7..676c586933 100644 --- a/scripts/dtest/m3dbnode.yml +++ b/scripts/dtest/m3dbnode.yml @@ -1,12 +1,6 @@ coordinator: listenAddress: 0.0.0.0:7201 - local: - namespaces: - - namespace: default - type: unaggregated - retention: 48h - logging: level: info diff --git a/src/aggregator/aggregation/quantile/cm/heap.go b/src/aggregator/aggregation/quantile/cm/heap.go index bcd8fc968b..027ac59a03 100644 --- a/src/aggregator/aggregation/quantile/cm/heap.go +++ b/src/aggregator/aggregation/quantile/cm/heap.go @@ -43,9 +43,9 @@ func (h *minHeap) Push(value float64) { heap := *h n := len(heap) i := n - 1 - for { + for i < n && i >= 0 { parent := (i - 1) / 2 - if parent == i || heap[parent] <= heap[i] { + if parent == i || parent >= n || parent < 0 || heap[parent] <= heap[i] { break } heap[parent], heap[i] = heap[i], heap[parent] @@ -53,17 +53,6 @@ func (h *minHeap) Push(value float64) { } } -func (h *minHeap) ensureSize() { - heap := *h - targetCap := cap(heap) * 2 - newHeap := sharedHeapPool.Get(targetCap) - (*newHeap) = append(*newHeap, heap...) - if cap(heap) >= _initialHeapBucketSize { - sharedHeapPool.Put(heap) - } - (*h) = *newHeap -} - func (h *minHeap) Reset() { if heap := *h; cap(heap) >= _initialHeapBucketSize { sharedHeapPool.Put(heap) @@ -99,3 +88,47 @@ func (h *minHeap) Pop() float64 { *h = old[0:n] return val } + +func (h minHeap) SortDesc() { + heap := h + // this is equivalent to Pop() in a loop (heapsort) + // all the redundant-looking conditions are there to eliminate bounds-checks + for n := len(heap) - 1; n > 0 && n < len(heap); n = len(heap) - 1 { + var ( + i int + smallest int + ) + heap[0], heap[n] = heap[n], heap[0] + for smallest >= 0 && smallest <= n { + var ( + left = smallest*2 + 1 + right = left + 1 + ) + if left < n && left >= 0 && heap[left] < heap[smallest] { + smallest = left + } + if right < n && right >= 0 && heap[right] < heap[smallest] { + smallest = right + } + if smallest == i { + break + } + heap[i], heap[smallest] = heap[smallest], heap[i] + i = smallest + } + heap = heap[0:n] + } +} + +func (h *minHeap) ensureSize() { + var ( + heap = *h + targetCap = cap(heap) * 2 + newHeap = sharedHeapPool.Get(targetCap) + ) + (*newHeap) = append(*newHeap, heap...) + if cap(heap) >= _initialHeapBucketSize { + sharedHeapPool.Put(heap) + } + (*h) = *newHeap +} diff --git a/src/aggregator/aggregation/quantile/cm/heap_test.go b/src/aggregator/aggregation/quantile/cm/heap_test.go index 8c02f8240a..befbaa1ebc 100644 --- a/src/aggregator/aggregation/quantile/cm/heap_test.go +++ b/src/aggregator/aggregation/quantile/cm/heap_test.go @@ -21,6 +21,8 @@ package cm import ( + "math/rand" + "sort" "testing" "github.com/stretchr/testify/require" @@ -33,6 +35,7 @@ func TestMinHeapPushInDecreasingOrder(t *testing.T) { h.Push(float64(i)) require.Equal(t, iter-i, h.Len()) } + validateSort(t, *h) for i := 0; i < iter; i++ { require.Equal(t, float64(i), h.Min()) require.Equal(t, float64(i), h.Pop()) @@ -47,6 +50,7 @@ func TestMinHeapPushInIncreasingOrder(t *testing.T) { h.Push(float64(i)) require.Equal(t, i+1, h.Len()) } + validateSort(t, *h) for i := 0; i < iter; i++ { require.Equal(t, float64(i), h.Min()) require.Equal(t, float64(i), h.Pop()) @@ -54,6 +58,29 @@ func TestMinHeapPushInIncreasingOrder(t *testing.T) { } } +func TestMinHeapPushInRandomOrderAndSort(t *testing.T) { + h := &minHeap{} + iter := 42 + for i := 0; i < iter; i++ { + h.Push(rand.ExpFloat64()) + } + validateSort(t, *h) +} + +func validateSort(t *testing.T, h minHeap) { + t.Helper() + // copy heap before applying reference sort and minheap-sort + a := make([]float64, h.Len()) + b := make([]float64, h.Len()) + for i := 0; i < len(h); i++ { + a[i], b[i] = h[i], h[i] + } + sort.Sort(sort.Reverse(sort.Float64Slice(a))) + heap := (*minHeap)(&b) + heap.SortDesc() + require.Equal(t, a, b) +} + func validateInvariant(t *testing.T, h minHeap, i int) { var ( n = h.Len() diff --git a/src/aggregator/aggregation/quantile/cm/stream.go b/src/aggregator/aggregation/quantile/cm/stream.go index cc9e20e4b5..179274dd39 100644 --- a/src/aggregator/aggregation/quantile/cm/stream.go +++ b/src/aggregator/aggregation/quantile/cm/stream.go @@ -253,10 +253,7 @@ func (s *Stream) calcQuantiles() { ) } - for curr != nil { - if idx == len(s.quantiles) { - break - } + for curr != nil && idx < len(s.computedQuantiles) { maxRank = minRank + curr.numRanks + curr.delta rank, threshold := s.thresholdBuf[idx].rank, s.thresholdBuf[idx].threshold @@ -271,7 +268,7 @@ func (s *Stream) calcQuantiles() { } // check if the last sample value should satisfy unprocessed quantiles - for i := idx; i < len(s.quantiles); i++ { + for i := idx; i < len(s.thresholdBuf); i++ { rank, threshold := s.thresholdBuf[i].rank, s.thresholdBuf[i].threshold if maxRank >= rank+threshold || minRank > rank { s.computedQuantiles[i] = prev.value @@ -284,6 +281,7 @@ func (s *Stream) insert() { var ( compCur = s.compressCursor compValue = math.NaN() + samples = &s.samples insertPointValue float64 sample *Sample ) @@ -292,15 +290,22 @@ func (s *Stream) insert() { compValue = compCur.value } - samples := &s.samples + // break heap invariant and just sort all the times, as we'll consume all of them in one go + s.bufMore.SortDesc() - for s.insertCursor != nil { + var ( + vals = []float64(s.bufMore) + idx = len(vals) - 1 + ) + + for s.insertCursor != nil && idx < len(vals) { curr := s.insertCursor insertPointValue = curr.value - for s.bufMore.Len() > 0 && s.bufMore.Min() <= insertPointValue { + for idx >= 0 && vals[idx] <= insertPointValue { + val := vals[idx] + idx-- sample = s.samples.Acquire() - val := s.bufMore.Pop() sample.value = val sample.numRanks = 1 sample.delta = curr.numRanks + curr.delta - 1 @@ -316,19 +321,20 @@ func (s *Stream) insert() { s.insertCursor = s.insertCursor.next } - if s.insertCursor != nil { - return - } - - for s.bufMore.Len() > 0 && s.bufMore.Min() >= samples.Back().value { - sample = s.samples.Acquire() - sample.value = s.bufMore.Pop() - sample.numRanks = 1 - sample.delta = 0 - samples.PushBack(sample) - s.numValues++ + if s.insertCursor == nil && idx < len(vals) { + for idx >= 0 && vals[idx] >= samples.Back().value { + val := vals[idx] + idx-- + sample = s.samples.Acquire() + sample.value = val + sample.numRanks = 1 + sample.delta = 0 + samples.PushBack(sample) + s.numValues++ + } } + s.bufMore = s.bufMore[:0] s.resetInsertCursor() } @@ -345,27 +351,48 @@ func (s *Stream) compress() { s.compressCursor = s.compressCursor.prev } + var ( + numVals = s.numValues + eps = 2.0 * s.eps + ) + for s.compressCursor != s.samples.Front() { - next := s.compressCursor.next - maxRank := s.compressMinRank + s.compressCursor.numRanks + s.compressCursor.delta - threshold := s.threshold(maxRank) - s.compressMinRank -= s.compressCursor.numRanks - testVal := s.compressCursor.numRanks + next.numRanks + next.delta + var ( + curr = s.compressCursor + next = curr.next + prev = curr.prev + + maxRank = s.compressMinRank + curr.numRanks + curr.delta + + threshold = int64(math.MaxInt64) + quantileMin int64 + ) + + for i := range s.quantiles { + if maxRank >= int64(s.quantiles[i]*float64(numVals)) { + quantileMin = int64(eps * float64(maxRank) / s.quantiles[i]) + } else { + quantileMin = int64(eps * float64(numVals-maxRank) / (1.0 - s.quantiles[i])) + } + if quantileMin < threshold { + threshold = quantileMin + } + } + + s.compressMinRank -= curr.numRanks + testVal := curr.numRanks + next.numRanks + next.delta if testVal <= threshold { - if s.insertCursor == s.compressCursor { + if s.insertCursor == curr { s.insertCursor = next } - next.numRanks += s.compressCursor.numRanks + next.numRanks += curr.numRanks - prev := s.compressCursor.prev // no need to release sample here - s.samples.Remove(s.compressCursor) - s.compressCursor = prev - } else { - s.compressCursor = s.compressCursor.prev + s.samples.Remove(curr) } + s.compressCursor = prev } if s.compressCursor == s.samples.Front() { diff --git a/src/aggregator/aggregator/entry.go b/src/aggregator/aggregator/entry.go index 1070be662e..0a6e3f1dc2 100644 --- a/src/aggregator/aggregator/entry.go +++ b/src/aggregator/aggregator/entry.go @@ -58,10 +58,16 @@ var ( errEmptyMetadatas = errors.New("empty metadata list") errNoApplicableMetadata = errors.New("no applicable metadata") errNoPipelinesInMetadata = errors.New("no pipelines in metadata") - errTooFarInTheFuture = xerrors.NewInvalidParamsError(errors.New("too far in the future")) - errTooFarInThePast = xerrors.NewInvalidParamsError(errors.New("too far in the past")) - errArrivedTooLate = xerrors.NewInvalidParamsError(errors.New("arrived too late")) - errTimestampFormat = time.RFC822Z + errOnlyDefaultStagedMetadata = xerrors.NewInvalidParamsError( + errors.New("only default staged metadata provided"), + ) + errOnlyDropPolicyStagedMetadata = xerrors.NewInvalidParamsError( + errors.New("only drop policy staged metadata provided"), + ) + errTooFarInTheFuture = xerrors.NewInvalidParamsError(errors.New("too far in the future")) + errTooFarInThePast = xerrors.NewInvalidParamsError(errors.New("too far in the past")) + errArrivedTooLate = xerrors.NewInvalidParamsError(errors.New("arrived too late")) + errTimestampFormat = time.RFC822Z ) type rateLimitEntryMetrics struct { @@ -285,6 +291,10 @@ func (e *Entry) AddTimedWithStagedMetadatas( if err := e.applyValueRateLimit(1, e.metrics.timed.rateLimit); err != nil { return err } + // Must have at least one metadata. addTimed further confirms that this metadata isn't the default metadata. + if len(metas) == 0 { + return errEmptyMetadatas + } return e.addTimed(metric, metadata.TimedMetadata{}, metas) } @@ -687,9 +697,20 @@ func (e *Entry) addTimed( } // Only apply processing of staged metadatas if has sent staged metadatas - // that isn't the default staged metadatas. + // that isn't the default staged metadatas. The default staged metadata + // would not produce a meaningful aggregation, so we error out in that case. hasDefaultMetadatas := stagedMetadatas.IsDefault() - if len(stagedMetadatas) > 0 && !hasDefaultMetadatas { + if len(stagedMetadatas) > 0 { + if hasDefaultMetadatas { + e.RUnlock() + timeLock.RUnlock() + return errOnlyDefaultStagedMetadata + } else if stagedMetadatas.IsDropPolicySet() { + e.RUnlock() + timeLock.RUnlock() + return errOnlyDropPolicyStagedMetadata + } + sm, err := e.activeStagedMetadataWithLock(currTime, stagedMetadatas) if err != nil { e.RUnlock() @@ -780,7 +801,7 @@ func (e *Entry) addTimed( return err } - // Update metatadata if not exists, and add metric. + // Update metadata if not exists, and add metric. if err := e.updateTimedMetadataWithLock(metric, metadata); err != nil { e.Unlock() timeLock.RUnlock() @@ -809,9 +830,9 @@ func (e *Entry) checkTimestampForTimedMetric( timestamp := time.Unix(0, metricTimeNanos) futureLimit := time.Unix(0, currNanos+timedBufferFuture.Nanoseconds()) err := fmt.Errorf("datapoint for aggregation too far in future: "+ - "id=%s, off_by=%s, timestamp=%s, future_limit=%s, "+ + "off_by=%s, timestamp=%s, future_limit=%s, "+ "timestamp_unix_nanos=%d, future_limit_unix_nanos=%d", - metric.ID, timestamp.Sub(futureLimit).String(), + timestamp.Sub(futureLimit).String(), timestamp.Format(errTimestampFormat), futureLimit.Format(errTimestampFormat), timestamp.UnixNano(), futureLimit.UnixNano()) @@ -828,9 +849,9 @@ func (e *Entry) checkTimestampForTimedMetric( timestamp := time.Unix(0, metricTimeNanos) pastLimit := time.Unix(0, currNanos-timedBufferPast.Nanoseconds()) err := fmt.Errorf("datapoint for aggregation too far in past: "+ - "id=%s, off_by=%s, timestamp=%s, past_limit=%s, "+ + "off_by=%s, timestamp=%s, past_limit=%s, "+ "timestamp_unix_nanos=%d, past_limit_unix_nanos=%d", - metric.ID, pastLimit.Sub(timestamp).String(), + pastLimit.Sub(timestamp).String(), timestamp.Format(errTimestampFormat), pastLimit.Format(errTimestampFormat), timestamp.UnixNano(), pastLimit.UnixNano()) diff --git a/src/aggregator/aggregator/entry_test.go b/src/aggregator/aggregator/entry_test.go index 8932400210..eafc89d269 100644 --- a/src/aggregator/aggregator/entry_test.go +++ b/src/aggregator/aggregator/entry_test.go @@ -1271,7 +1271,6 @@ func TestEntryAddTimedMetricTooLate(t *testing.T) { require.True(t, xerrors.IsInvalidParams(err)) require.Equal(t, errTooFarInThePast, xerrors.InnerError(err)) require.True(t, strings.Contains(err.Error(), "datapoint for aggregation too far in past")) - require.True(t, strings.Contains(err.Error(), "id="+string(metric.ID))) require.True(t, strings.Contains(err.Error(), "timestamp=")) require.True(t, strings.Contains(err.Error(), "past_limit=")) } @@ -1307,7 +1306,6 @@ func TestEntryAddTimedMetricTooEarly(t *testing.T) { require.True(t, xerrors.IsInvalidParams(err)) require.Equal(t, errTooFarInTheFuture, xerrors.InnerError(err)) require.True(t, strings.Contains(err.Error(), "datapoint for aggregation too far in future")) - require.True(t, strings.Contains(err.Error(), "id="+string(metric.ID))) require.True(t, strings.Contains(err.Error(), "timestamp=")) require.True(t, strings.Contains(err.Error(), "future_limit=")) } @@ -1425,6 +1423,47 @@ func TestEntryAddTimed(t *testing.T) { require.Equal(t, testTimedMetric.ID, counterElem.ID()) } +func TestEntryAddTimedWithStagedMetadatasEmptyList(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + e, _, _ := testEntry(ctrl, testEntryOptions{ + options: testOptions(ctrl).SetVerboseErrors(true), + }) + + require.Equal(t, errEmptyMetadatas, e.AddTimedWithStagedMetadatas(testTimedMetric, nil)) +} + +func TestEntryAddTimedWithStagedMetadatasDefaultMetadata(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + e, _, _ := testEntry(ctrl, testEntryOptions{ + options: testOptions(ctrl).SetVerboseErrors(true), + }) + + require.Equal( + t, + errOnlyDefaultStagedMetadata, + e.AddTimedWithStagedMetadatas(testTimedMetric, metadata.DefaultStagedMetadatas), + ) +} + +func TestEntryAddTimedWithStagedMetadatasDropPolicyMetadata(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + e, _, _ := testEntry(ctrl, testEntryOptions{ + options: testOptions(ctrl).SetVerboseErrors(true), + }) + + require.Equal( + t, + errOnlyDropPolicyStagedMetadata, + e.AddTimedWithStagedMetadatas(testTimedMetric, metadata.DropStagedMetadatas), + ) +} + func TestEntryForwardedRateLimiting(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() diff --git a/src/aggregator/aggregator/flush_mgr_options.go b/src/aggregator/aggregator/flush_mgr_options.go index 431443cd74..93fe0a1d12 100644 --- a/src/aggregator/aggregator/flush_mgr_options.go +++ b/src/aggregator/aggregator/flush_mgr_options.go @@ -122,6 +122,12 @@ type FlushManagerOptions interface { // ForcedFlushWindowSize returns the window size for a forced flush. ForcedFlushWindowSize() time.Duration + + // SetBufferForPastTimedMetric sets the size of the buffer for timed metrics in the past. + SetBufferForPastTimedMetric(value time.Duration) FlushManagerOptions + + // BufferForPastTimedMetric returns the size of the buffer for timed metrics in the past. + BufferForPastTimedMetric() time.Duration } type flushManagerOptions struct { @@ -137,6 +143,8 @@ type flushManagerOptions struct { flushTimesPersistEvery time.Duration maxBufferSize time.Duration forcedFlushWindowSize time.Duration + + bufferForPastTimedMetric time.Duration } // NewFlushManagerOptions create a new set of flush manager options. @@ -152,6 +160,8 @@ func NewFlushManagerOptions() FlushManagerOptions { flushTimesPersistEvery: defaultFlushTimesPersistEvery, maxBufferSize: defaultMaxBufferSize, forcedFlushWindowSize: defaultForcedFlushWindowSize, + + bufferForPastTimedMetric: defaultTimedMetricBuffer, } } @@ -274,3 +284,13 @@ func (o *flushManagerOptions) SetForcedFlushWindowSize(value time.Duration) Flus func (o *flushManagerOptions) ForcedFlushWindowSize() time.Duration { return o.forcedFlushWindowSize } + +func (o *flushManagerOptions) SetBufferForPastTimedMetric(value time.Duration) FlushManagerOptions { + opts := *o + opts.bufferForPastTimedMetric = value + return &opts +} + +func (o *flushManagerOptions) BufferForPastTimedMetric() time.Duration { + return o.bufferForPastTimedMetric +} diff --git a/src/aggregator/aggregator/follower_flush_mgr.go b/src/aggregator/aggregator/follower_flush_mgr.go index ac69dfb4c3..70e285fe27 100644 --- a/src/aggregator/aggregator/follower_flush_mgr.go +++ b/src/aggregator/aggregator/follower_flush_mgr.go @@ -123,6 +123,8 @@ type followerFlushManager struct { flushTask *followerFlushTask sleepFn sleepFn metrics followerFlushManagerMetrics + + bufferForPastTimedMetric time.Duration } func newFollowerFlushManager( @@ -133,22 +135,23 @@ func newFollowerFlushManager( instrumentOpts := opts.InstrumentOptions() scope := instrumentOpts.MetricsScope() mgr := &followerFlushManager{ - nowFn: nowFn, - checkEvery: opts.CheckEvery(), - workers: opts.WorkerPool(), - placementManager: opts.PlacementManager(), - electionManager: opts.ElectionManager(), - flushTimesManager: opts.FlushTimesManager(), - maxBufferSize: opts.MaxBufferSize(), - forcedFlushWindowSize: opts.ForcedFlushWindowSize(), - logger: instrumentOpts.Logger(), - scope: scope, - doneCh: doneCh, - flushTimesState: flushTimesUninitialized, - flushMode: unknownFollowerFlush, - lastFlushed: nowFn(), - sleepFn: time.Sleep, - metrics: newFollowerFlushManagerMetrics(scope), + nowFn: nowFn, + checkEvery: opts.CheckEvery(), + workers: opts.WorkerPool(), + placementManager: opts.PlacementManager(), + electionManager: opts.ElectionManager(), + flushTimesManager: opts.FlushTimesManager(), + maxBufferSize: opts.MaxBufferSize(), + forcedFlushWindowSize: opts.ForcedFlushWindowSize(), + bufferForPastTimedMetric: opts.BufferForPastTimedMetric(), + logger: instrumentOpts.Logger(), + scope: scope, + doneCh: doneCh, + flushTimesState: flushTimesUninitialized, + flushMode: unknownFollowerFlush, + lastFlushed: nowFn(), + sleepFn: time.Sleep, + metrics: newFollowerFlushManagerMetrics(scope), } mgr.flushTask = &followerFlushTask{mgr: mgr} return mgr @@ -215,12 +218,7 @@ func (mgr *followerFlushManager) OnBucketAdded(int, *flushBucket) {} // NB(xichen): The follower flush manager flushes data based on the flush times // stored in kv and does not need to take extra actions when a new flusher is added. -func (mgr *followerFlushManager) OnFlusherAdded( - bucketIdx int, - bucket *flushBucket, - flusher flushingMetricList, -) { -} +func (mgr *followerFlushManager) OnFlusherAdded(int, *flushBucket, flushingMetricList) {} // The follower flush manager may only lead if and only if all the following conditions // are met: @@ -271,7 +269,6 @@ func (mgr *followerFlushManager) CanLead() bool { now, int(shardID), shardFlushTimes.ForwardedByResolution, - mgr.metrics.forwarded, ) { return false } @@ -294,10 +291,10 @@ func (mgr *followerFlushManager) canLead( zap.Time("now", now), zap.Stringer("windowSize", windowSize), zap.Stringer("flusherType", flusherType), - zap.Int("shardID", int(shardID))) + zap.Int("shardID", shardID)) mgr.metrics.forwarded.windowNeverFlushed.Inc(1) - if mgr.canLeadNotFlushed(now, windowSize) { + if mgr.canLeadNotFlushed(now, windowSize, flusherType) { continue } else { return false @@ -320,7 +317,6 @@ func (mgr *followerFlushManager) canLeadForwarded( now time.Time, shardID int, flushTimes map[int64]*schema.ForwardedFlushTimesForResolution, - metrics forwardedFollowerFlusherMetrics, ) bool { // Check that the forwarded metrics have been flushed past the process start // time, meaning the forwarded metrics that didn't make to the process have been @@ -328,7 +324,7 @@ func (mgr *followerFlushManager) canLeadForwarded( for windowNanos, fbr := range flushTimes { if fbr == nil { mgr.logger.Warn("ForwardedByResolution is nil", - zap.Int("shardID", int(shardID))) + zap.Int("shardID", shardID)) mgr.metrics.forwarded.nilForwardedTimes.Inc(1) return false } @@ -348,11 +344,11 @@ func (mgr *followerFlushManager) canLeadForwarded( zap.Time("now", now), zap.Stringer("windowSize", windowSize), zap.Stringer("flusherType", forwardedMetricListType), - zap.Int("shardID", int(shardID)), + zap.Int("shardID", shardID), zap.Int32("numForwardedTimes", numForwardedTimes)) mgr.metrics.forwarded.windowNeverFlushed.Inc(1) - if mgr.canLeadNotFlushed(now, windowSize) { + if mgr.canLeadNotFlushed(now, windowSize, forwardedMetricListType) { continue } else { return false @@ -371,10 +367,20 @@ func (mgr *followerFlushManager) canLeadForwarded( // canLeadNotFlushed determines whether the follower can takeover leadership // for the window that was not (yet) flushed by leader. -// This is case is possible when leader encounters a metric at some resolution +// This case is possible when leader encounters a metric at some resolution // window for the first time in the shard it owns. -func (mgr *followerFlushManager) canLeadNotFlushed(now time.Time, windowSize time.Duration) bool { - windowStartAt := now.Truncate(windowSize) +func (mgr *followerFlushManager) canLeadNotFlushed( + now time.Time, + windowSize time.Duration, + flusherType metricListType, +) bool { + adjustedNow := now + if flusherType == timedMetricListType { + adjustedNow = adjustedNow.Add(-mgr.bufferForPastTimedMetric) + } + + windowStartAt := adjustedNow.Truncate(windowSize) + return mgr.openedAt.Before(windowStartAt) } diff --git a/src/aggregator/aggregator/follower_flush_mgr_test.go b/src/aggregator/aggregator/follower_flush_mgr_test.go index 265bf4a004..32edb07108 100644 --- a/src/aggregator/aggregator/follower_flush_mgr_test.go +++ b/src/aggregator/aggregator/follower_flush_mgr_test.go @@ -48,7 +48,8 @@ func TestFollowerFlushManagerOpen(t *testing.T) { mgr := newFollowerFlushManager(doneCh, opts).(*followerFlushManager) mgr.Open() - watchable.Update(testFlushTimes) + require.NoError(t, watchable.Update(testFlushTimes)) + for { mgr.RLock() state := mgr.flushTimesState @@ -139,12 +140,12 @@ func TestFollowerFlushManagerCanLeadNotFlushed(t *testing.T) { window1m := time.Minute testFlushTimes := &schema.ShardSetFlushTimes{ ByShard: map[uint32]*schema.ShardFlushTimes{ - 123: &schema.ShardFlushTimes{ + 123: { StandardByResolution: map[int64]int64{ window10m.Nanoseconds(): 0, }, ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - window10m.Nanoseconds(): &schema.ForwardedFlushTimesForResolution{ + window10m.Nanoseconds(): { ByNumForwardedTimes: map[int32]int64{ 1: 0, }, @@ -154,53 +155,25 @@ func TestFollowerFlushManagerCanLeadNotFlushed(t *testing.T) { }, } - runTestFn := func( - t *testing.T, - flushTimes *schema.ShardSetFlushTimes, - followerOpenedAt time.Time, - expectedCanLead bool, - ) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - doneCh := make(chan struct{}) - electionManager := NewMockElectionManager(ctrl) - electionManager.EXPECT().IsCampaigning().Return(true).AnyTimes() - clockOpts := clock.NewOptions().SetNowFn(func() time.Time { - return now - }) - opts := NewFlushManagerOptions(). - SetElectionManager(electionManager). - SetClockOptions(clockOpts) - mgr := newFollowerFlushManager(doneCh, opts).(*followerFlushManager) - - mgr.processed = flushTimes - mgr.openedAt = followerOpenedAt - require.Equal(t, expectedCanLead, mgr.CanLead()) - } - t.Run("opened_on_the_window_start", func(t *testing.T) { followerOpenedAt := now.Truncate(window10m) - expectedCanLead := false - runTestFn(t, testFlushTimes, followerOpenedAt, expectedCanLead) + testCanLeadNotFlushed(t, testFlushTimes, now, followerOpenedAt, 0, false) }) t.Run("opened_after_the_window_start", func(t *testing.T) { followerOpenedAt := now.Truncate(window10m).Add(1 * time.Second) - expectedCanLead := false - runTestFn(t, testFlushTimes, followerOpenedAt, expectedCanLead) + testCanLeadNotFlushed(t, testFlushTimes, now, followerOpenedAt, 0, false) }) t.Run("opened_before_the_window_start", func(t *testing.T) { followerOpenedAt := now.Truncate(window10m).Add(-1 * time.Second) - expectedCanLead := true - runTestFn(t, testFlushTimes, followerOpenedAt, expectedCanLead) + testCanLeadNotFlushed(t, testFlushTimes, now, followerOpenedAt, 0, true) }) t.Run("standard_flushed_ok_and_unflushed_bad", func(t *testing.T) { flushedAndUnflushedTimes := &schema.ShardSetFlushTimes{ ByShard: map[uint32]*schema.ShardFlushTimes{ - 123: &schema.ShardFlushTimes{ + 123: { StandardByResolution: map[int64]int64{ window1m.Nanoseconds(): now.Add(1 * time.Second).UnixNano(), window10m.Nanoseconds(): 0, @@ -210,21 +183,20 @@ func TestFollowerFlushManagerCanLeadNotFlushed(t *testing.T) { } followerOpenedAt := now - expectedCanLead := false - runTestFn(t, flushedAndUnflushedTimes, followerOpenedAt, expectedCanLead) + testCanLeadNotFlushed(t, flushedAndUnflushedTimes, now, followerOpenedAt, 0, false) }) t.Run("forwarded_flushed_ok_and_unflushed_bad", func(t *testing.T) { flushedAndUnflushedTimes := &schema.ShardSetFlushTimes{ ByShard: map[uint32]*schema.ShardFlushTimes{ - 123: &schema.ShardFlushTimes{ + 123: { ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - window1m.Nanoseconds(): &schema.ForwardedFlushTimesForResolution{ + window1m.Nanoseconds(): { ByNumForwardedTimes: map[int32]int64{ 1: now.Truncate(window1m).Add(1 * time.Second).UnixNano(), }, }, - window10m.Nanoseconds(): &schema.ForwardedFlushTimesForResolution{ + window10m.Nanoseconds(): { ByNumForwardedTimes: map[int32]int64{ 1: 0, }, @@ -235,9 +207,75 @@ func TestFollowerFlushManagerCanLeadNotFlushed(t *testing.T) { } followerOpenedAt := now - expectedCanLead := false - runTestFn(t, flushedAndUnflushedTimes, followerOpenedAt, expectedCanLead) + testCanLeadNotFlushed(t, flushedAndUnflushedTimes, now, followerOpenedAt, 0, false) + }) +} + +func TestFollowerFlushManagerCanLeadTimedNotFlushed(t *testing.T) { + var ( + now = time.Unix(24*60*60, 0) + window10m = 10 * time.Minute + bufferPast = 3 * time.Minute + ) + + flushTimes := &schema.ShardSetFlushTimes{ + ByShard: map[uint32]*schema.ShardFlushTimes{ + 123: { + TimedByResolution: map[int64]int64{ + window10m.Nanoseconds(): 0, + }, + }, + }, + } + + t.Run("opened_on_the_window_start", func(t *testing.T) { + followerOpenedAt := now.Add(-bufferPast).Truncate(window10m) + testCanLeadNotFlushed(t, flushTimes, now, followerOpenedAt, bufferPast, false) + }) + + t.Run("opened_after_the_window_start", func(t *testing.T) { + followerOpenedAt := now.Add(-bufferPast).Truncate(window10m).Add(time.Second) + testCanLeadNotFlushed(t, flushTimes, now, followerOpenedAt, bufferPast, false) }) + + t.Run("opened_before_the_window_start", func(t *testing.T) { + followerOpenedAt := now.Add(-bufferPast).Truncate(window10m).Add(-time.Second) + testCanLeadNotFlushed(t, flushTimes, now, followerOpenedAt, bufferPast, true) + }) +} + +func testCanLeadNotFlushed( + t *testing.T, + flushTimes *schema.ShardSetFlushTimes, + now time.Time, + followerOpenedAt time.Time, + bufferPast time.Duration, + expectedCanLead bool, +) { + t.Helper() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + doneCh := make(chan struct{}) + electionManager := NewMockElectionManager(ctrl) + electionManager.EXPECT().IsCampaigning().Return(true).AnyTimes() + clockOpts := clock.NewOptions().SetNowFn(func() time.Time { + return now + }) + opts := NewFlushManagerOptions(). + SetElectionManager(electionManager). + SetClockOptions(clockOpts) + + if bufferPast > 0 { + opts = opts.SetBufferForPastTimedMetric(bufferPast) + } + + mgr := newFollowerFlushManager(doneCh, opts).(*followerFlushManager) + + mgr.processed = flushTimes + mgr.openedAt = followerOpenedAt + require.Equal(t, expectedCanLead, mgr.CanLead()) } func TestFollowerFlushManagerCanLeadNoTombstonedShards(t *testing.T) { diff --git a/src/aggregator/aggregator/leader_flush_mgr_test.go b/src/aggregator/aggregator/leader_flush_mgr_test.go index 1ed81ec748..37057bef46 100644 --- a/src/aggregator/aggregator/leader_flush_mgr_test.go +++ b/src/aggregator/aggregator/leader_flush_mgr_test.go @@ -37,18 +37,18 @@ import ( var ( testFlushTimes = &schema.ShardSetFlushTimes{ ByShard: map[uint32]*schema.ShardFlushTimes{ - 0: &schema.ShardFlushTimes{ + 0: { StandardByResolution: map[int64]int64{ 1000000000: 3663000000000, 60000000000: 3660000000000, }, ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - 1000000000: &schema.ForwardedFlushTimesForResolution{ + 1000000000: { ByNumForwardedTimes: map[int32]int64{ 1: 3663000000000, }, }, - 60000000000: &schema.ForwardedFlushTimesForResolution{ + 60000000000: { ByNumForwardedTimes: map[int32]int64{ 2: 3660000000000, 3: 3600000000000, @@ -56,19 +56,19 @@ var ( }, }, }, - 1: &schema.ShardFlushTimes{ + 1: { StandardByResolution: map[int64]int64{ 1000000000: 3658000000000, }, ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - 1000000000: &schema.ForwardedFlushTimesForResolution{ + 1000000000: { ByNumForwardedTimes: map[int32]int64{ 1: 3658000000000, }, }, }, }, - 2: &schema.ShardFlushTimes{ + 2: { StandardByResolution: map[int64]int64{ 3600000000000: 3600000000000, }, @@ -78,7 +78,7 @@ var ( testFlushTimes2 = &schema.ShardSetFlushTimes{ ByShard: map[uint32]*schema.ShardFlushTimes{ - 0: &schema.ShardFlushTimes{ + 0: { StandardByResolution: map[int64]int64{ 1000000000: 3669000000000, 60000000000: 3660000000000, @@ -87,12 +87,12 @@ var ( 1000000000: 3600000000000, }, ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - 1000000000: &schema.ForwardedFlushTimesForResolution{ + 1000000000: { ByNumForwardedTimes: map[int32]int64{ 1: 3681000000000, }, }, - 60000000000: &schema.ForwardedFlushTimesForResolution{ + 60000000000: { ByNumForwardedTimes: map[int32]int64{ 2: 3660000000000, 3: 3600000000000, @@ -101,12 +101,12 @@ var ( }, Tombstoned: false, }, - 1: &schema.ShardFlushTimes{ + 1: { StandardByResolution: map[int64]int64{ 1000000000: 3658000000000, }, ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - 1000000000: &schema.ForwardedFlushTimesForResolution{ + 1000000000: { ByNumForwardedTimes: map[int32]int64{ 1: 3658000000000, }, @@ -114,21 +114,21 @@ var ( }, Tombstoned: true, }, - 2: &schema.ShardFlushTimes{ + 2: { StandardByResolution: map[int64]int64{ 3600000000000: 3600000000000, }, Tombstoned: true, }, - 3: &schema.ShardFlushTimes{ + 3: { StandardByResolution: map[int64]int64{ 3600000000000: 7200000000000, }, Tombstoned: false, }, - 4: &schema.ShardFlushTimes{ + 4: { ForwardedByResolution: map[int64]*schema.ForwardedFlushTimesForResolution{ - 60000000000: &schema.ForwardedFlushTimesForResolution{ + 60000000000: { ByNumForwardedTimes: map[int32]int64{ 2: 3658000000000, }, @@ -655,38 +655,38 @@ func testFlushBuckets(ctrl *gomock.Controller) []*flushBucket { return []*flushBucket{ // Standard flushing metric lists. - &flushBucket{ + { bucketID: standardMetricListID{resolution: time.Second}.toMetricListID(), interval: time.Second, offset: 250 * time.Millisecond, flushers: []flushingMetricList{standardFlusher1, standardFlusher2}, }, - &flushBucket{ + { bucketID: standardMetricListID{resolution: time.Minute}.toMetricListID(), interval: time.Minute, offset: 12 * time.Second, flushers: []flushingMetricList{standardFlusher3}, }, - &flushBucket{ + { bucketID: standardMetricListID{resolution: time.Hour}.toMetricListID(), interval: time.Hour, offset: time.Minute, flushers: []flushingMetricList{standardFlusher4}, }, // Forwarded flushing metric lists. - &flushBucket{ + { bucketID: forwardedMetricListID{resolution: time.Second, numForwardedTimes: 1}.toMetricListID(), interval: time.Second, offset: 100 * time.Millisecond, flushers: []flushingMetricList{forwardedFlusher1, forwardedFlusher2}, }, - &flushBucket{ + { bucketID: forwardedMetricListID{resolution: time.Minute, numForwardedTimes: 2}.toMetricListID(), interval: time.Minute, offset: time.Second, flushers: []flushingMetricList{forwardedFlusher3}, }, - &flushBucket{ + { bucketID: forwardedMetricListID{resolution: time.Minute, numForwardedTimes: 3}.toMetricListID(), interval: time.Minute, offset: 0, @@ -722,31 +722,31 @@ func testFlushBuckets2(ctrl *gomock.Controller) []*flushBucket { forwardedFlusher2.EXPECT().LastFlushedNanos().Return(int64(3658000000000)).AnyTimes() return []*flushBucket{ - &flushBucket{ + { bucketID: standardMetricListID{resolution: time.Second}.toMetricListID(), interval: time.Second, offset: 250 * time.Millisecond, flushers: []flushingMetricList{standardFlusher1}, }, - &flushBucket{ + { bucketID: standardMetricListID{resolution: time.Hour}.toMetricListID(), interval: time.Hour, offset: time.Minute, flushers: []flushingMetricList{standardFlusher2}, }, - &flushBucket{ + { bucketID: timedMetricListID{resolution: time.Second}.toMetricListID(), interval: time.Second, offset: 250 * time.Millisecond, flushers: []flushingMetricList{timedFlusher1}, }, - &flushBucket{ + { bucketID: forwardedMetricListID{resolution: time.Second, numForwardedTimes: 1}.toMetricListID(), interval: time.Second, offset: 100 * time.Millisecond, flushers: []flushingMetricList{forwardedFlusher1}, }, - &flushBucket{ + { bucketID: forwardedMetricListID{resolution: time.Minute, numForwardedTimes: 2}.toMetricListID(), interval: time.Minute, offset: time.Second, diff --git a/src/aggregator/aggregator/options.go b/src/aggregator/aggregator/options.go index ad58cd1cb6..2957ae6143 100644 --- a/src/aggregator/aggregator/options.go +++ b/src/aggregator/aggregator/options.go @@ -250,10 +250,16 @@ type Options interface { // delay for given metric resolution and number of times the metric has been forwarded. MaxAllowedForwardingDelayFn() MaxAllowedForwardingDelayFn - // SetBufferForPastTimedMetricFn sets the size of the buffer for timed metrics in the past. + // SetBufferForPastTimedMetric sets the size of the buffer for timed metrics in the past. + SetBufferForPastTimedMetric(value time.Duration) Options + + // BufferForPastTimedMetric returns the size of the buffer for timed metrics in the past. + BufferForPastTimedMetric() time.Duration + + // SetBufferForPastTimedMetricFn sets the size fn of the buffer for timed metrics in the past. SetBufferForPastTimedMetricFn(value BufferForPastTimedMetricFn) Options - // BufferForPastTimedMetricFn returns the size of the buffer for timed metrics in the past. + // BufferForPastTimedMetricFn returns the size fn of the buffer for timed metrics in the past. BufferForPastTimedMetricFn() BufferForPastTimedMetricFn // SetBufferForFutureTimedMetric sets the size of the buffer for timed metrics in the future. @@ -353,6 +359,7 @@ type options struct { electionManager ElectionManager resignTimeout time.Duration maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn + bufferForPastTimedMetric time.Duration bufferForPastTimedMetricFn BufferForPastTimedMetricFn bufferForFutureTimedMetric time.Duration maxNumCachedSourceSets int @@ -399,6 +406,7 @@ func NewOptions() Options { defaultStoragePolicies: defaultDefaultStoragePolicies, resignTimeout: defaultResignTimeout, maxAllowedForwardingDelayFn: defaultMaxAllowedForwardingDelayFn, + bufferForPastTimedMetric: defaultTimedMetricBuffer, bufferForPastTimedMetricFn: defaultBufferForPastTimedMetricFn, bufferForFutureTimedMetric: defaultTimedMetricBuffer, maxNumCachedSourceSets: defaultMaxNumCachedSourceSets, @@ -689,6 +697,16 @@ func (o *options) MaxAllowedForwardingDelayFn() MaxAllowedForwardingDelayFn { return o.maxAllowedForwardingDelayFn } +func (o *options) SetBufferForPastTimedMetric(value time.Duration) Options { + opts := *o + opts.bufferForPastTimedMetric = value + return &opts +} + +func (o *options) BufferForPastTimedMetric() time.Duration { + return o.bufferForPastTimedMetric +} + func (o *options) SetBufferForPastTimedMetricFn(value BufferForPastTimedMetricFn) Options { opts := *o opts.bufferForPastTimedMetricFn = value diff --git a/src/aggregator/integration/setup.go b/src/aggregator/integration/setup.go index d80afed7fa..095ddb2aab 100644 --- a/src/aggregator/integration/setup.go +++ b/src/aggregator/integration/setup.go @@ -161,7 +161,8 @@ func newTestServerSetup(t *testing.T, opts testServerOptions) *testServerSetup { SetFlushTimesManager(flushTimesManager). SetElectionManager(electionManager). SetJitterEnabled(opts.JitterEnabled()). - SetMaxJitterFn(opts.MaxJitterFn()) + SetMaxJitterFn(opts.MaxJitterFn()). + SetBufferForPastTimedMetric(aggregatorOpts.BufferForPastTimedMetric()) flushManager := aggregator.NewFlushManager(flushManagerOpts) aggregatorOpts = aggregatorOpts.SetFlushManager(flushManager) diff --git a/src/aggregator/server/m3msg/server.go b/src/aggregator/server/m3msg/server.go index e856d9c5fc..3164449fa9 100644 --- a/src/aggregator/server/m3msg/server.go +++ b/src/aggregator/server/m3msg/server.go @@ -71,9 +71,11 @@ func (s *server) Consume(c consumer.Consumer) { break } + // Reset and reuse the protobuf message for unpacking. + protobuf.ReuseMetricWithMetadatasProto(pb) err := s.handleMessage(pb, union, msg) if err != nil { - s.logger.Error("could not process message", zap.Error(err)) + s.logger.Error("could not process message", zap.Error(err), zap.String("proto", pb.String())) } } if msgErr != nil && msgErr != io.EOF { @@ -89,9 +91,6 @@ func (s *server) handleMessage( ) error { defer msg.Ack() - // Reset and reuse the protobuf message for unpacking. - protobuf.ReuseMetricWithMetadatasProto(pb) - // Unmarshal the message. if err := pb.Unmarshal(msg.Bytes()); err != nil { return err diff --git a/src/cmd/services/m3aggregator/config/aggregator.go b/src/cmd/services/m3aggregator/config/aggregator.go index 93db250d88..cf18fe6080 100644 --- a/src/cmd/services/m3aggregator/config/aggregator.go +++ b/src/cmd/services/m3aggregator/config/aggregator.go @@ -332,7 +332,8 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( opts = opts.SetBufferDurationAfterShardCutoff(c.BufferDurationAfterShardCutoff) } if c.BufferDurationForPastTimedMetric != 0 { - opts = opts.SetBufferForPastTimedMetricFn(bufferForPastTimedMetricFn(c.BufferDurationForPastTimedMetric)) + opts = opts.SetBufferForPastTimedMetric(c.BufferDurationForPastTimedMetric). + SetBufferForPastTimedMetricFn(bufferForPastTimedMetricFn(c.BufferDurationForPastTimedMetric)) } if c.BufferDurationForFutureTimedMetric != 0 { opts = opts.SetBufferForFutureTimedMetric(c.BufferDurationForFutureTimedMetric) @@ -374,6 +375,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( electionManager, flushTimesManager, iOpts, + opts.BufferForPastTimedMetric(), ) if err != nil { return nil, err @@ -779,12 +781,14 @@ func (c flushManagerConfiguration) NewFlushManagerOptions( electionManager aggregator.ElectionManager, flushTimesManager aggregator.FlushTimesManager, instrumentOpts instrument.Options, + bufferForPastTimedMetric time.Duration, ) (aggregator.FlushManagerOptions, error) { opts := aggregator.NewFlushManagerOptions(). SetInstrumentOptions(instrumentOpts). SetPlacementManager(placementManager). SetElectionManager(electionManager). - SetFlushTimesManager(flushTimesManager) + SetFlushTimesManager(flushTimesManager). + SetBufferForPastTimedMetric(bufferForPastTimedMetric) if c.CheckEvery != 0 { opts = opts.SetCheckEvery(c.CheckEvery) } diff --git a/src/cmd/services/m3coordinator/downsample/downsampler.go b/src/cmd/services/m3coordinator/downsample/downsampler.go index 6e19e39b2a..df8e4da8d7 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler.go @@ -66,9 +66,9 @@ type SamplesAppenderResult struct { // SampleAppenderOptions defines the options being used when constructing // the samples appender for a metric. type SampleAppenderOptions struct { - Override bool - OverrideRules SamplesAppenderOverrideRules - MetricType ts.M3MetricType + Override bool + OverrideRules SamplesAppenderOverrideRules + SeriesAttributes ts.SeriesAttributes } // SamplesAppenderOverrideRules provides override rules to diff --git a/src/cmd/services/m3coordinator/downsample/downsampler_test.go b/src/cmd/services/m3coordinator/downsample/downsampler_test.go index 3777b5abc7..979ef48cae 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler_test.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler_test.go @@ -447,6 +447,88 @@ func TestDownsamplerAggregationWithRulesConfigMappingRules(t *testing.T) { testDownsamplerAggregation(t, testDownsampler) } +func TestDownsamplerAggregationWithAutoMappingRulesAndRulesConfigMappingRulesAndDropRule(t *testing.T) { + t.Parallel() + + gaugeMetric := testGaugeMetric{ + tags: map[string]string{ + nameTag: "foo_metric", + "app": "nginx_edge", + "env": "staging", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 15}, {value: 10}, {value: 30}, {value: 5}, {value: 0}, + }, + expectDropPolicyApplied: true, + } + testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + autoMappingRules: []m3.ClusterNamespaceOptions{ + m3.NewClusterNamespaceOptions( + storagemetadata.Attributes{ + MetricsType: storagemetadata.AggregatedMetricsType, + Retention: 2 * time.Hour, + Resolution: 1 * time.Second, + }, + nil, + ), + m3.NewClusterNamespaceOptions( + storagemetadata.Attributes{ + MetricsType: storagemetadata.AggregatedMetricsType, + Retention: 12 * time.Hour, + Resolution: 5 * time.Second, + }, + nil, + ), + }, + rulesConfig: &RulesConfiguration{ + MappingRules: []MappingRuleConfiguration{ + { + Filter: "env:staging", + Drop: true, + }, + { + Filter: "app:nginx*", + Aggregations: []aggregation.Type{aggregation.Max}, + StoragePolicies: []StoragePolicyConfiguration{ + { + Resolution: 10 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + }, + }, + ingest: &testDownsamplerOptionsIngest{ + gaugeMetrics: []testGaugeMetric{gaugeMetric}, + }, + expect: &testDownsamplerOptionsExpect{ + allowFilter: &testDownsamplerOptionsExpectAllowFilter{ + attributes: []storagemetadata.Attributes{ + { + MetricsType: storagemetadata.AggregatedMetricsType, + Resolution: 10 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + writes: []testExpectedWrite{ + { + tags: gaugeMetric.tags, + values: []expectedValue{{value: 30}}, + attributes: &storagemetadata.Attributes{ + MetricsType: storagemetadata.AggregatedMetricsType, + Resolution: 10 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + }, + }) + + // Test expected output + testDownsamplerAggregation(t, testDownsampler) +} + func TestDownsamplerAggregationWithRulesConfigMappingRulesPartialReplaceAutoMappingRuleFromNamespacesWatcher(t *testing.T) { t.Parallel() @@ -677,7 +759,64 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesTypeFilter(t *testing. }, }, sampleAppenderOpts: &SampleAppenderOptions{ - MetricType: ts.M3MetricTypeCounter, + SeriesAttributes: ts.SeriesAttributes{M3Type: ts.M3MetricTypeCounter}, + }, + ingest: &testDownsamplerOptionsIngest{ + gaugeMetrics: []testGaugeMetric{gaugeMetric}, + }, + expect: &testDownsamplerOptionsExpect{ + writes: []testExpectedWrite{ + { + tags: map[string]string{ + "app": "nginx_edge", + "endpoint": "health", + }, + values: []expectedValue{{value: 30}}, + attributes: &storagemetadata.Attributes{ + MetricsType: storagemetadata.AggregatedMetricsType, + Resolution: 1 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + }, + }) + + // Test expected output + testDownsamplerAggregation(t, testDownsampler) +} + +//nolint:dupl +func TestDownsamplerAggregationWithRulesConfigMappingRulesTypePromFilter(t *testing.T) { + t.Parallel() + + gaugeMetric := testGaugeMetric{ + tags: map[string]string{ + "app": "nginx_edge", + "endpoint": "health", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 15}, {value: 10}, {value: 30}, {value: 5}, {value: 0}, + }, + } + testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + identTag: "endpoint", + rulesConfig: &RulesConfiguration{ + MappingRules: []MappingRuleConfiguration{ + { + Filter: "__m3_prom_type__:counter", + Aggregations: []aggregation.Type{aggregation.Max}, + StoragePolicies: []StoragePolicyConfiguration{ + { + Resolution: 1 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + }, + }, + sampleAppenderOpts: &SampleAppenderOptions{ + SeriesAttributes: ts.SeriesAttributes{PromType: ts.PromMetricTypeCounter}, }, ingest: &testDownsamplerOptionsIngest{ gaugeMetrics: []testGaugeMetric{gaugeMetric}, @@ -733,7 +872,51 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesTypeFilterNoMatch(t *t }, }, sampleAppenderOpts: &SampleAppenderOptions{ - MetricType: ts.M3MetricTypeGauge, + SeriesAttributes: ts.SeriesAttributes{M3Type: ts.M3MetricTypeGauge}, + }, + ingest: &testDownsamplerOptionsIngest{ + gaugeMetrics: []testGaugeMetric{gaugeMetric}, + }, + expect: &testDownsamplerOptionsExpect{ + writes: []testExpectedWrite{}, + }, + }) + + // Test expected output + testDownsamplerAggregation(t, testDownsampler) +} + +//nolint:dupl +func TestDownsamplerAggregationWithRulesConfigMappingRulesPromTypeFilterNoMatch(t *testing.T) { + t.Parallel() + + gaugeMetric := testGaugeMetric{ + tags: map[string]string{ + "app": "nginx_edge", + "endpoint": "health", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 15}, {value: 10}, {value: 30}, {value: 5}, {value: 0}, + }, + } + testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + identTag: "endpoint", + rulesConfig: &RulesConfiguration{ + MappingRules: []MappingRuleConfiguration{ + { + Filter: "__m3_prom_type__:counter", + Aggregations: []aggregation.Type{aggregation.Max}, + StoragePolicies: []StoragePolicyConfiguration{ + { + Resolution: 1 * time.Second, + Retention: 30 * 24 * time.Hour, + }, + }, + }, + }, + }, + sampleAppenderOpts: &SampleAppenderOptions{ + SeriesAttributes: ts.SeriesAttributes{PromType: ts.PromMetricTypeGauge}, }, ingest: &testDownsamplerOptionsIngest{ gaugeMetrics: []testGaugeMetric{gaugeMetric}, @@ -1362,6 +1545,7 @@ func TestDownsamplerAggregationWithRulesConfigRollupRulesAggregateTransformNoRol func TestDownsamplerAggregationWithRulesConfigRollupRulesIncreaseAdd(t *testing.T) { t.Parallel() + // nolint:dupl gaugeMetrics := []testGaugeMetric{ { tags: map[string]string{ @@ -1544,7 +1728,7 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropPolicy(t *testing testDownsamplerAggregation(t, testDownsampler) } -func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropTimestamp(t *testing.T) { +func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropPolicyAndDropTimestamp(t *testing.T) { t.Parallel() gaugeMetrics := []testGaugeMetric{ @@ -1561,7 +1745,8 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropTimestamp(t *test // Explicit no value. {value: 12, offset: 2 * time.Second}, // +12 - simulate a reset (should not be accounted) }, - expectDropTimestamp: true, + expectDropTimestamp: true, + expectDropPolicyApplied: true, }, { tags: map[string]string{ @@ -1575,7 +1760,8 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropTimestamp(t *test {value: 13}, {value: 27, offset: 2 * time.Second}, // +14 }, - expectDropTimestamp: true, + expectDropTimestamp: true, + expectDropPolicyApplied: true, }, } tags := []Tag{ @@ -1588,15 +1774,9 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropTimestamp(t *test rulesConfig: &RulesConfiguration{ MappingRules: []MappingRuleConfiguration{ { - Filter: filter, - Tags: tags, - Aggregations: []aggregation.Type{testAggregationType}, - StoragePolicies: []StoragePolicyConfiguration{ - { - Resolution: res, - Retention: ret, - }, - }, + Filter: filter, + Drop: true, + Tags: tags, }, }, RollupRules: []RollupRuleConfiguration{ @@ -1648,68 +1828,160 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropTimestamp(t *test testDownsamplerAggregation(t, testDownsampler) } -func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropPolicyAndDropTimestamp(t *testing.T) { +func TestDownsamplerAggregationWithRulesConfigRollupRulesExcludeByLastMean(t *testing.T) { t.Parallel() gaugeMetrics := []testGaugeMetric{ { tags: map[string]string{ - nameTag: "http_requests", - "app": "nginx_edge", - "status_code": "500", - "endpoint": "/foo/bar", - "not_rolled_up": "not_rolled_up_value_1", + nameTag: "http_request_latency_max_gauge", + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + "instance": "not_rolled_up_instance_1", }, timedSamples: []testGaugeMetricTimedSample{ - {value: 42}, // +42 (should not be accounted since is a reset) - // Explicit no value. - {value: 12, offset: 2 * time.Second}, // +12 - simulate a reset (should not be accounted) + {value: 42}, }, - expectDropTimestamp: true, - expectDropPolicyApplied: true, }, { tags: map[string]string{ - nameTag: "http_requests", - "app": "nginx_edge", - "status_code": "500", - "endpoint": "/foo/bar", - "not_rolled_up": "not_rolled_up_value_2", + nameTag: "http_request_latency_max_gauge", + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + "instance": "not_rolled_up_instance_2", }, timedSamples: []testGaugeMetricTimedSample{ {value: 13}, - {value: 27, offset: 2 * time.Second}, // +14 }, - expectDropTimestamp: true, - expectDropPolicyApplied: true, }, } - tags := []Tag{ - {Name: "__m3_drop_timestamp__", Value: ""}, - } res := 1 * time.Second ret := 30 * 24 * time.Hour - filter := fmt.Sprintf("%s:http_requests app:* status_code:* endpoint:*", nameTag) testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ rulesConfig: &RulesConfiguration{ - MappingRules: []MappingRuleConfiguration{ + RollupRules: []RollupRuleConfiguration{ { - Filter: filter, - Drop: true, - Tags: tags, + Filter: fmt.Sprintf( + "%s:http_request_latency_max_gauge app:* status_code:* endpoint:*", + nameTag), + Transforms: []TransformConfiguration{ + { + Aggregate: &AggregateOperationConfiguration{ + Type: aggregation.Last, + }, + }, + { + Rollup: &RollupOperationConfiguration{ + MetricName: "{{ .MetricName }}:mean_without_instance", + ExcludeBy: []string{"instance"}, + Aggregations: []aggregation.Type{aggregation.Mean}, + }, + }, + }, + StoragePolicies: []StoragePolicyConfiguration{ + { + Resolution: res, + Retention: ret, + }, + }, + }, + }, + }, + ingest: &testDownsamplerOptionsIngest{ + gaugeMetrics: gaugeMetrics, + }, + expect: &testDownsamplerOptionsExpect{ + writes: []testExpectedWrite{ + { + tags: map[string]string{ + nameTag: "http_request_latency_max_gauge:mean_without_instance", + string(rollupTagName): string(rollupTagValue), + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + }, + values: []expectedValue{ + {value: 27.5}, + }, + attributes: &storagemetadata.Attributes{ + MetricsType: storagemetadata.AggregatedMetricsType, + Resolution: res, + Retention: ret, + }, }, }, + }, + }) + + // Test expected output + testDownsamplerAggregation(t, testDownsampler) +} + +func TestDownsamplerAggregationWithRulesConfigRollupRulesExcludeByIncreaseSumAdd(t *testing.T) { + t.Parallel() + + // nolint:dupl + gaugeMetrics := []testGaugeMetric{ + { + tags: map[string]string{ + nameTag: "http_requests", + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + "instance": "not_rolled_up_instance_1", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 42, offset: 1 * time.Second}, // +42 (should not be accounted since is a reset) + // Explicit no value. + {value: 12, offset: 2 * time.Second}, // +12 - simulate a reset (should not be accounted) + {value: 33, offset: 3 * time.Second}, // +21 + }, + }, + { + tags: map[string]string{ + nameTag: "http_requests", + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + "instance": "not_rolled_up_instance_2", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 13, offset: 1 * time.Second}, // +13 (should not be accounted since is a reset) + {value: 27, offset: 2 * time.Second}, // +14 + // Explicit no value. + {value: 42, offset: 3 * time.Second}, // +15 + }, + }, + } + res := 1 * time.Second + ret := 30 * 24 * time.Hour + testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + rulesConfig: &RulesConfiguration{ RollupRules: []RollupRuleConfiguration{ { - Filter: filter, + Filter: fmt.Sprintf( + "%s:http_requests app:* status_code:* endpoint:*", + nameTag), Transforms: []TransformConfiguration{ + { + Transform: &TransformOperationConfiguration{ + Type: transformation.Increase, + }, + }, { Rollup: &RollupOperationConfiguration{ - MetricName: "http_requests_by_status_code", - GroupBy: []string{"app", "status_code", "endpoint"}, + MetricName: "{{ .MetricName }}:sum_without_instance", + ExcludeBy: []string{"instance"}, Aggregations: []aggregation.Type{aggregation.Sum}, }, }, + { + Transform: &TransformOperationConfiguration{ + Type: transformation.Add, + }, + }, }, StoragePolicies: []StoragePolicyConfiguration{ { @@ -1727,13 +1999,16 @@ func TestDownsamplerAggregationWithRulesConfigRollupRuleAndDropPolicyAndDropTime writes: []testExpectedWrite{ { tags: map[string]string{ - nameTag: "http_requests_by_status_code", + nameTag: "http_requests:sum_without_instance", string(rollupTagName): string(rollupTagValue), "app": "nginx_edge", "status_code": "500", "endpoint": "/foo/bar", }, - values: []expectedValue{{value: 55}, {value: 39}}, + values: []expectedValue{ + {value: 14}, + {value: 50, offset: 1 * time.Second}, + }, attributes: &storagemetadata.Attributes{ MetricsType: storagemetadata.AggregatedMetricsType, Resolution: res, @@ -2077,12 +2352,14 @@ func testDownsamplerAggregation( expectedWrites := append(counterMetricsExpect, gaugeMetricsExpect...) // Allow overrides + var allowFilter *testDownsamplerOptionsExpectAllowFilter if ingest := testOpts.ingest; ingest != nil { counterMetrics = ingest.counterMetrics gaugeMetrics = ingest.gaugeMetrics } if expect := testOpts.expect; expect != nil { expectedWrites = expect.writes + allowFilter = expect.allowFilter } // Ingest points @@ -2200,6 +2477,24 @@ CheckAllWritesArrivedLoop: } } } + + if allowFilter == nil { + return // No allow filter checking required. + } + + for _, write := range testDownsampler.storage.Writes() { + attrs := write.Attributes() + foundMatchingAttribute := false + for _, allowed := range allowFilter.attributes { + if allowed == attrs { + foundMatchingAttribute = true + break + } + } + assert.True(t, foundMatchingAttribute, + fmt.Sprintf("attribute not allowed: allowed=%v, actual=%v", + allowFilter.attributes, attrs)) + } } func testDownsamplerRemoteAggregation( @@ -2425,7 +2720,7 @@ type testDownsamplerOptions struct { identTag string // Options for the test - autoMappingRules []AutoMappingRule + autoMappingRules []m3.ClusterNamespaceOptions sampleAppenderOpts *SampleAppenderOptions remoteClientMock *client.MockClient rulesConfig *RulesConfiguration @@ -2442,7 +2737,12 @@ type testDownsamplerOptionsIngest struct { } type testDownsamplerOptionsExpect struct { - writes []testExpectedWrite + writes []testExpectedWrite + allowFilter *testDownsamplerOptionsExpectAllowFilter +} + +type testDownsamplerOptionsExpectAllowFilter struct { + attributes []storagemetadata.Attributes } func newTestDownsampler(t *testing.T, opts testDownsamplerOptions) testDownsampler { @@ -2526,6 +2826,23 @@ func newTestDownsampler(t *testing.T, opts testDownsamplerOptions) testDownsampl }) require.NoError(t, err) + if len(opts.autoMappingRules) > 0 { + // Simulate the automapping rules being injected into the downsampler. + ctrl := gomock.NewController(t) + + var mockNamespaces m3.ClusterNamespaces + for _, r := range opts.autoMappingRules { + n := m3.NewMockClusterNamespace(ctrl) + n.EXPECT(). + Options(). + Return(r). + AnyTimes() + mockNamespaces = append(mockNamespaces, n) + } + + instance.(*downsampler).OnUpdate(mockNamespaces) + } + downcast, ok := instance.(*downsampler) require.True(t, ok) diff --git a/src/cmd/services/m3coordinator/downsample/metrics_appender.go b/src/cmd/services/m3coordinator/downsample/metrics_appender.go index 95420bb70d..428272e3ab 100644 --- a/src/cmd/services/m3coordinator/downsample/metrics_appender.go +++ b/src/cmd/services/m3coordinator/downsample/metrics_appender.go @@ -151,7 +151,7 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp // NB (@shreyas): Add the metric type tag. The tag has the prefix // __m3_. All tags with that prefix are only used for the purpose of // filter match and then stripped off before we actually send to the aggregator. - switch opts.MetricType { + switch opts.SeriesAttributes.M3Type { case ts.M3MetricTypeCounter: tags.append(metric.M3TypeTag, metric.M3CounterValue) case ts.M3MetricTypeGauge: @@ -159,6 +159,24 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp case ts.M3MetricTypeTimer: tags.append(metric.M3TypeTag, metric.M3TimerValue) } + switch opts.SeriesAttributes.PromType { + case ts.PromMetricTypeUnknown: + tags.append(metric.M3PromTypeTag, metric.PromUnknownValue) + case ts.PromMetricTypeCounter: + tags.append(metric.M3PromTypeTag, metric.PromCounterValue) + case ts.PromMetricTypeGauge: + tags.append(metric.M3PromTypeTag, metric.PromGaugeValue) + case ts.PromMetricTypeHistogram: + tags.append(metric.M3PromTypeTag, metric.PromHistogramValue) + case ts.PromMetricTypeGaugeHistogram: + tags.append(metric.M3PromTypeTag, metric.PromGaugeHistogramValue) + case ts.PromMetricTypeSummary: + tags.append(metric.M3PromTypeTag, metric.PromSummaryValue) + case ts.PromMetricTypeInfo: + tags.append(metric.M3PromTypeTag, metric.PromInfoValue) + case ts.PromMetricTypeStateSet: + tags.append(metric.M3PromTypeTag, metric.PromStateSetValue) + } // Sort tags sort.Sort(tags) @@ -186,17 +204,16 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp matchResult := a.matcher.ForwardMatch(id, fromNanos, toNanos) id.Close() - // filter out augmented metrics tags - tags.filterPrefix(metric.M3MetricsPrefix) + // Filter out augmented metrics tags we added for matching. + for _, filter := range defaultFilterOutTagPrefixes { + tags.filterPrefix(filter) + } - var ( - dropApplyResult metadata.ApplyOrRemoveDropPoliciesResult - dropTimestamp bool - ) - if opts.Override { - // Reuse a slice to keep the current staged metadatas we will apply. - a.curr.Pipelines = a.curr.Pipelines[:0] + // Reuse a slice to keep the current staged metadatas we will apply. + a.curr.Pipelines = a.curr.Pipelines[:0] + if opts.Override { + // Process an override explicitly provided as part of request. for _, rule := range opts.OverrideRules.MappingRules { stagedMetadatas, err := rule.StagedMetadatas() if err != nil { @@ -214,56 +231,68 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp if err := a.addSamplesAppenders(tags, a.curr); err != nil { return SamplesAppenderResult{}, err } - } else { - // Reuse a slice to keep the current staged metadatas we will apply. - a.curr.Pipelines = a.curr.Pipelines[:0] - - // NB(r): First apply mapping rules to see which storage policies - // have been applied, any that have been applied as part of - // mapping rules that exact match a default storage policy will be - // skipped when applying default rules, so as to avoid storing - // the same metrics in the same namespace with the same metric - // name and tags (i.e. overwriting each other). - a.mappingRuleStoragePolicies = a.mappingRuleStoragePolicies[:0] - - ruleStagedMetadatas := matchResult.ForExistingIDAt(nowNanos) - if !ruleStagedMetadatas.IsDefault() && len(ruleStagedMetadatas) != 0 { - a.debugLogMatch("downsampler applying matched rule", - debugLogMatchOptions{Meta: ruleStagedMetadatas}) - - // Collect storage policies for all the current active mapping rules. - // TODO: we should convert this to iterate over pointers - // nolint:gocritic - for _, stagedMetadata := range ruleStagedMetadatas { - for _, pipe := range stagedMetadata.Pipelines { - // Skip rollup rules unless configured otherwise. - // We only want to consider mapping rules here, - // as we still want to apply default mapping rules to - // metrics that are rolled up to ensure the underlying metric - // gets written to aggregated namespaces. - if pipe.IsMappingRule() { - for _, sp := range pipe.StoragePolicies { - a.mappingRuleStoragePolicies = - append(a.mappingRuleStoragePolicies, sp) - } - } else { - a.debugLogMatch( - "skipping rollup rule in populating active mapping rule policies", - debugLogMatchOptions{}, - ) + + return SamplesAppenderResult{ + SamplesAppender: a.multiSamplesAppender, + IsDropPolicyApplied: false, + ShouldDropTimestamp: false, + }, nil + } + + // NB(r): First apply mapping rules to see which storage policies + // have been applied, any that have been applied as part of + // mapping rules that exact match a default storage policy will be + // skipped when applying default rules, so as to avoid storing + // the same metrics in the same namespace with the same metric + // name and tags (i.e. overwriting each other). + var ( + ruleStagedMetadatas = matchResult.ForExistingIDAt(nowNanos) + dropApplyResult metadata.ApplyOrRemoveDropPoliciesResult + dropTimestamp bool + ) + a.mappingRuleStoragePolicies = a.mappingRuleStoragePolicies[:0] + if !ruleStagedMetadatas.IsDefault() && len(ruleStagedMetadatas) != 0 { + a.debugLogMatch("downsampler applying matched rule", + debugLogMatchOptions{Meta: ruleStagedMetadatas}) + + // Collect storage policies for all the current active mapping rules. + // TODO: we should convert this to iterate over pointers + // nolint:gocritic + for _, stagedMetadata := range ruleStagedMetadatas { + for _, pipe := range stagedMetadata.Pipelines { + // Skip rollup rules unless configured otherwise. + // We only want to consider mapping rules here, + // as we still want to apply default mapping rules to + // metrics that are rolled up to ensure the underlying metric + // gets written to aggregated namespaces. + if pipe.IsMappingRule() { + for _, sp := range pipe.StoragePolicies { + a.mappingRuleStoragePolicies = + append(a.mappingRuleStoragePolicies, sp) } + } else { + a.debugLogMatch( + "skipping rollup rule in populating active mapping rule policies", + debugLogMatchOptions{}, + ) } } - - // Only sample if going to actually aggregate - pipelines := ruleStagedMetadatas[len(ruleStagedMetadatas)-1] - a.curr.Pipelines = - append(a.curr.Pipelines, pipelines.Pipelines...) } - // Always aggregate any default staged metadatas (unless - // mapping rule has provided an override for a storage policy, - // if so then skip aggregating for that storage policy). + // Only sample if going to actually aggregate + pipelines := ruleStagedMetadatas[len(ruleStagedMetadatas)-1] + a.curr.Pipelines = append(a.curr.Pipelines, pipelines.Pipelines...) + } + + // Always aggregate any default staged metadatas with a few exceptions. + // Exceptions are: + // 1. A mapping rule has provided an override for a storage policy, + // if so then skip aggregating for that storage policy). + // 2. Any type of drop rule has been set, since they should only + // impact mapping rules, not default staged metadatas provided from + // auto-mapping rules (i.e. default namespace aggregation). + if !a.curr.Pipelines.IsDropPolicySet() { + // No drop rule has been set as part of rule matching. for idx, stagedMetadatasProto := range a.defaultStagedMetadatasProtos { // NB(r): Need to take copy of default staged metadatas as we // sometimes mutate it. @@ -342,40 +371,39 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp debugLogMatchOptions{Meta: stagedMetadatas}) pipelines := stagedMetadatas[len(stagedMetadatas)-1] - a.curr.Pipelines = - append(a.curr.Pipelines, pipelines.Pipelines...) + a.curr.Pipelines = append(a.curr.Pipelines, pipelines.Pipelines...) } + } - // Apply the custom tags first so that they apply even if mapping - // rules drop the metric. - dropTimestamp = a.curr.Pipelines.ApplyCustomTags() + // Apply the custom tags first so that they apply even if mapping + // rules drop the metric. + dropTimestamp = a.curr.Pipelines.ApplyCustomTags() - // Apply drop policies results - a.curr.Pipelines, dropApplyResult = a.curr.Pipelines.ApplyOrRemoveDropPolicies() + // Apply drop policies results + a.curr.Pipelines, dropApplyResult = a.curr.Pipelines.ApplyOrRemoveDropPolicies() - if len(a.curr.Pipelines) > 0 && !a.curr.IsDropPolicyApplied() { - // Send to downsampler if we have something in the pipeline. - a.debugLogMatch("downsampler using built mapping staged metadatas", - debugLogMatchOptions{Meta: []metadata.StagedMetadata{a.curr}}) + if len(a.curr.Pipelines) > 0 && !a.curr.IsDropPolicyApplied() { + // Send to downsampler if we have something in the pipeline. + a.debugLogMatch("downsampler using built mapping staged metadatas", + debugLogMatchOptions{Meta: []metadata.StagedMetadata{a.curr}}) - if err := a.addSamplesAppenders(tags, a.curr); err != nil { - return SamplesAppenderResult{}, err - } + if err := a.addSamplesAppenders(tags, a.curr); err != nil { + return SamplesAppenderResult{}, err } + } - numRollups := matchResult.NumNewRollupIDs() - for i := 0; i < numRollups; i++ { - rollup := matchResult.ForNewRollupIDsAt(i, nowNanos) - - a.debugLogMatch("downsampler applying matched rollup rule", - debugLogMatchOptions{Meta: rollup.Metadatas, RollupID: rollup.ID}) - a.multiSamplesAppender.addSamplesAppender(samplesAppender{ - agg: a.agg, - clientRemote: a.clientRemote, - unownedID: rollup.ID, - stagedMetadatas: rollup.Metadatas, - }) - } + numRollups := matchResult.NumNewRollupIDs() + for i := 0; i < numRollups; i++ { + rollup := matchResult.ForNewRollupIDsAt(i, nowNanos) + + a.debugLogMatch("downsampler applying matched rollup rule", + debugLogMatchOptions{Meta: rollup.Metadatas, RollupID: rollup.ID}) + a.multiSamplesAppender.addSamplesAppender(samplesAppender{ + agg: a.agg, + clientRemote: a.clientRemote, + unownedID: rollup.ID, + stagedMetadatas: rollup.Metadatas, + }) } dropPolicyApplied := dropApplyResult != metadata.NoDropPolicyPresentResult diff --git a/src/cmd/services/m3coordinator/downsample/metrics_appender_test.go b/src/cmd/services/m3coordinator/downsample/metrics_appender_test.go index 368dcc519e..68320ab9a7 100644 --- a/src/cmd/services/m3coordinator/downsample/metrics_appender_test.go +++ b/src/cmd/services/m3coordinator/downsample/metrics_appender_test.go @@ -121,7 +121,7 @@ func TestSamplesAppenderPoolResetsTagsAcrossSamples(t *testing.T) { // NB: expected ID is generated into human-readable form // from tags in ForwardMatch mock above. Also include the m3 type, which is included when matching. // nolint:scopelint - expected := fmt.Sprintf("__m3_type__-gauge,foo%d-bar%d", i, i) + expected := fmt.Sprintf("__m3_prom_type__-unknown,__m3_type__-gauge,foo%d-bar%d", i, i) if expected != u.ID.String() { // NB: if this fails, appender is holding state after Finalize. return fmt.Errorf("expected ID %s, got %s", expected, u.ID.String()) diff --git a/src/cmd/services/m3coordinator/downsample/options.go b/src/cmd/services/m3coordinator/downsample/options.go index 21035420cf..88a1d1fa48 100644 --- a/src/cmd/services/m3coordinator/downsample/options.go +++ b/src/cmd/services/m3coordinator/downsample/options.go @@ -21,6 +21,7 @@ package downsample import ( + "bytes" "errors" "fmt" "runtime" @@ -91,8 +92,11 @@ const ( ) var ( - numShards = runtime.NumCPU() - defaultNamespaceTag = metric.M3MetricsPrefixString + "_namespace__" + numShards = runtime.NumCPU() + defaultNamespaceTag = metric.M3MetricsPrefixString + "_namespace__" + defaultFilterOutTagPrefixes = [][]byte{ + metric.M3MetricsPrefix, + } errNoStorage = errors.New("downsampling enabled with storage not set") errNoClusterClient = errors.New("downsampling enabled with cluster client not set") @@ -478,15 +482,30 @@ func (r RollupRuleConfiguration) Rule() (view.RollupRule, error) { switch { case elem.Rollup != nil: cfg := elem.Rollup + if len(cfg.GroupBy) > 0 && len(cfg.ExcludeBy) > 0 { + return view.RollupRule{}, fmt.Errorf( + "must specify group by or exclude by tags for rollup operation not both: "+ + "groupBy=%d, excludeBy=%d", len(cfg.GroupBy), len(cfg.ExcludeBy)) + } + + rollupType := pipelinepb.RollupOp_GROUP_BY + tags := cfg.GroupBy + if len(cfg.ExcludeBy) > 0 { + rollupType = pipelinepb.RollupOp_EXCLUDE_BY + tags = cfg.ExcludeBy + } + aggregationTypes, err := AggregationTypes(cfg.Aggregations).Proto() if err != nil { return view.RollupRule{}, err } + op, err := pipeline.NewOpUnionFromProto(pipelinepb.PipelineOp{ Type: pipelinepb.PipelineOp_ROLLUP, Rollup: &pipelinepb.RollupOp{ + Type: rollupType, NewName: cfg.MetricName, - Tags: cfg.GroupBy, + Tags: tags, AggregationTypes: aggregationTypes, }, }) @@ -567,8 +586,16 @@ type RollupOperationConfiguration struct { // GroupBy is a set of labels to group by, only these remain on the // new metric name produced by the rollup operation. + // Note: Can only use either groupBy or excludeBy, not both, use the + // rollup operation "type" to specify which is used. GroupBy []string `yaml:"groupBy"` + // ExcludeBy is a set of labels to exclude by, only these tags are removed + // from the resulting rolled up metric. + // Note: Can only use either groupBy or excludeBy, not both, use the + // rollup operation "type" to specify which is used. + ExcludeBy []string `yaml:"excludeBy"` + // Aggregations is a set of aggregate operations to perform. Aggregations []aggregation.Type `yaml:"aggregations"` } @@ -1016,8 +1043,23 @@ func (o DownsamplerOptions) newAggregatorRulesOptions(pools aggPools) rules.Opti newRollupIDProviderPool.Init() newRollupIDFn := func(newName []byte, tagPairs []id.TagPair) []byte { + // First filter out any tags that have a prefix that + // are not included in output metric IDs (such as metric + // type tags that are just used for filtering like __m3_type__). + filtered := tagPairs[:0] + TagPairsFilterLoop: + for i := range tagPairs { + for _, filter := range defaultFilterOutTagPrefixes { + if bytes.HasPrefix(tagPairs[i].Name, filter) { + continue TagPairsFilterLoop + } + } + filtered = append(filtered, tagPairs[i]) + } + + // Create the rollup using filtered tag pairs. rollupIDProvider := newRollupIDProviderPool.Get() - id, err := rollupIDProvider.provide(newName, tagPairs) + id, err := rollupIDProvider.provide(newName, filtered) if err != nil { panic(err) // Encoding should never fail } diff --git a/src/cmd/services/m3coordinator/ingest/write.go b/src/cmd/services/m3coordinator/ingest/write.go index cc94744243..0e14a567d7 100644 --- a/src/cmd/services/m3coordinator/ingest/write.go +++ b/src/cmd/services/m3coordinator/ingest/write.go @@ -259,6 +259,8 @@ func (d *downsamplerAndWriter) writeToDownsampler( downsample.GraphiteIDSchemeTagValue) } + // NB: we don't set series attributes on the sample appender options here. + // In practice this isn't needed because only the carbon ingest path comes through here. var appenderOpts downsample.SampleAppenderOptions if downsampleMappingRuleOverrides, ok := d.downsampleOverrideRules(overrides); ok { appenderOpts = downsample.SampleAppenderOptions{ @@ -478,7 +480,7 @@ func (d *downsamplerAndWriter) writeAggregatedBatch( } opts := downsample.SampleAppenderOptions{ - MetricType: value.Attributes.M3Type, + SeriesAttributes: value.Attributes, } if downsampleMappingRuleOverrides, ok := d.downsampleOverrideRules(overrides); ok { opts = downsample.SampleAppenderOptions{ diff --git a/src/cmd/services/m3coordinator/ingest/write_test.go b/src/cmd/services/m3coordinator/ingest/write_test.go index b3281fabe6..e50c757dbe 100644 --- a/src/cmd/services/m3coordinator/ingest/write_test.go +++ b/src/cmd/services/m3coordinator/ingest/write_test.go @@ -564,12 +564,16 @@ func TestDownsampleAndWriteBatchDifferentTypes(t *testing.T) { mockMetricsAppender. EXPECT(). - SamplesAppender(downsample.SampleAppenderOptions{MetricType: ts.M3MetricTypeCounter}). - Return(downsample.SamplesAppenderResult{SamplesAppender: mockSamplesAppender}, nil).Times(1) + SamplesAppender(downsample.SampleAppenderOptions{ + SeriesAttributes: ts.SeriesAttributes{M3Type: ts.M3MetricTypeCounter}, + }).Return(downsample.SamplesAppenderResult{SamplesAppender: mockSamplesAppender}, nil). + Times(1) mockMetricsAppender. EXPECT(). - SamplesAppender(downsample.SampleAppenderOptions{MetricType: ts.M3MetricTypeTimer}). - Return(downsample.SamplesAppenderResult{SamplesAppender: mockSamplesAppender}, nil).Times(1) + SamplesAppender(downsample.SampleAppenderOptions{ + SeriesAttributes: ts.SeriesAttributes{M3Type: ts.M3MetricTypeTimer}, + }).Return(downsample.SamplesAppenderResult{SamplesAppender: mockSamplesAppender}, nil). + Times(1) for _, tag := range testTags1.Tags { mockMetricsAppender.EXPECT().AddTag(tag.Name, tag.Value) } diff --git a/src/cmd/services/m3dbnode/config/config.go b/src/cmd/services/m3dbnode/config/config.go index e2a4db455d..d5f3cc797f 100644 --- a/src/cmd/services/m3dbnode/config/config.go +++ b/src/cmd/services/m3dbnode/config/config.go @@ -197,9 +197,6 @@ type DBConfiguration struct { // of applying back-pressure or protecting the db nodes. Limits LimitsConfiguration `yaml:"limits"` - // FetchTagged contains configuration related to the FetchTagged API endpoint. - FetchTagged FetchTaggedConfiguration `yaml:"fetchTagged"` - // WideConfig contains some limits for wide operations. These operations // differ from regular paths by optimizing for query completeness across // arbitary query ranges rather than speed. diff --git a/src/cmd/services/m3dbnode/config/config_test.go b/src/cmd/services/m3dbnode/config/config_test.go index 3d03f31f06..2ce0ad573f 100644 --- a/src/cmd/services/m3dbnode/config/config_test.go +++ b/src/cmd/services/m3dbnode/config/config_test.go @@ -312,8 +312,6 @@ db: writeNewSeriesBackoffDuration: 2ms tracing: backend: jaeger - fetchTagged: - seriesBlocksPerBatch: 100 ` func TestConfiguration(t *testing.T) { @@ -741,8 +739,6 @@ func TestConfiguration(t *testing.T) { maxOutstandingRepairedBytes: 0 maxEncodersPerBlock: 0 writeNewSeriesPerSecond: 0 - fetchTagged: - seriesBlocksPerBatch: 100 wide: null tchannel: null debug: diff --git a/src/cmd/services/m3dbnode/config/fetch_tagged.go b/src/cmd/services/m3dbnode/config/fetch_tagged.go deleted file mode 100644 index af0497613c..0000000000 --- a/src/cmd/services/m3dbnode/config/fetch_tagged.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright (c) 2021 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 contains static configuration for the dbnode service. -package config - -// FetchTaggedConfiguration contains configuration related to the FetchTagged API endpoint. -type FetchTaggedConfiguration struct { - // SeriesBlocksPerBatch specifies how many series blocks are allowed to be retrieved - // per permit acquired. Defaults to 1 (i.e. a permit is acquired per series block). - SeriesBlocksPerBatch *int `yaml:"seriesBlocksPerBatch"` -} - -const defaultSeriesBlocksPerBatch = 1 - -// SeriesBlocksPerBatchOrDefault returns the SeriesBlocksPerBatch value provided or -// a sane default. -func (f *FetchTaggedConfiguration) SeriesBlocksPerBatchOrDefault() int { - if f.SeriesBlocksPerBatch == nil || *f.SeriesBlocksPerBatch < 1 { - return defaultSeriesBlocksPerBatch - } - - return *f.SeriesBlocksPerBatch -} diff --git a/src/cmd/services/m3dbnode/config/pooling.go b/src/cmd/services/m3dbnode/config/pooling.go index 584397a86b..5f92617446 100644 --- a/src/cmd/services/m3dbnode/config/pooling.go +++ b/src/cmd/services/m3dbnode/config/pooling.go @@ -20,7 +20,11 @@ package config -import "fmt" +import ( + "fmt" + + "github.com/m3db/m3/src/x/pool" +) // PoolingType is a type of pooling, using runtime or mmap'd bytes pooling. type PoolingType string @@ -39,7 +43,7 @@ const ( ) type poolPolicyDefault struct { - size int + size pool.Size refillLowWaterMark float64 refillHighWaterMark float64 @@ -214,7 +218,7 @@ var ( { Capacity: intPtr(16), PoolPolicy: PoolPolicy{ - Size: intPtr(524288), + Size: poolSizePtr(524288), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -222,7 +226,7 @@ var ( { Capacity: intPtr(32), PoolPolicy: PoolPolicy{ - Size: intPtr(262144), + Size: poolSizePtr(262144), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -230,7 +234,7 @@ var ( { Capacity: intPtr(64), PoolPolicy: PoolPolicy{ - Size: intPtr(131072), + Size: poolSizePtr(131072), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -238,7 +242,7 @@ var ( { Capacity: intPtr(128), PoolPolicy: PoolPolicy{ - Size: intPtr(65536), + Size: poolSizePtr(65536), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -246,7 +250,7 @@ var ( { Capacity: intPtr(256), PoolPolicy: PoolPolicy{ - Size: intPtr(65536), + Size: poolSizePtr(65536), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -254,7 +258,7 @@ var ( { Capacity: intPtr(1440), PoolPolicy: PoolPolicy{ - Size: intPtr(16384), + Size: poolSizePtr(16384), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -262,7 +266,7 @@ var ( { Capacity: intPtr(4096), PoolPolicy: PoolPolicy{ - Size: intPtr(8192), + Size: poolSizePtr(8192), RefillLowWaterMark: &defaultRefillLowWaterMark, RefillHighWaterMark: &defaultRefillHighWaterMark, }, @@ -486,7 +490,7 @@ func (p *PoolingPolicy) TypeOrDefault() PoolingType { // PoolPolicy specifies a single pool policy. type PoolPolicy struct { // The size of the pool. - Size *int `yaml:"size"` + Size *pool.Size `yaml:"size"` // The low watermark to start refilling the pool, if zero none. RefillLowWaterMark *float64 `yaml:"lowWatermark"` @@ -525,7 +529,7 @@ func (p *PoolPolicy) initDefaultsAndValidate(poolName string) error { } // SizeOrDefault returns the configured size if present, or a default value otherwise. -func (p *PoolPolicy) SizeOrDefault() int { +func (p *PoolPolicy) SizeOrDefault() pool.Size { return *p.Size } @@ -668,3 +672,8 @@ type WriteBatchPoolPolicy struct { func intPtr(x int) *int { return &x } + +func poolSizePtr(x int) *pool.Size { + sz := pool.Size(x) + return &sz +} diff --git a/src/cmd/services/r2ctl/config/config.go b/src/cmd/services/r2ctl/config/config.go index e169b1bb2f..9141e6cb0a 100644 --- a/src/cmd/services/r2ctl/config/config.go +++ b/src/cmd/services/r2ctl/config/config.go @@ -71,7 +71,7 @@ type r2StoreConfiguration struct { // NewR2Store creates a new R2 store. func (c r2StoreConfiguration) NewR2Store(instrumentOpts instrument.Options) (r2store.Store, error) { if c.Stub { - return stub.NewStore(instrumentOpts), nil + return stub.NewStore(instrumentOpts) } if c.KV == nil { diff --git a/src/ctl/service/r2/store/kv/store_test.go b/src/ctl/service/r2/store/kv/store_test.go index 8de4b8e75f..d1742d5c96 100644 --- a/src/ctl/service/r2/store/kv/store_test.go +++ b/src/ctl/service/r2/store/kv/store_test.go @@ -291,11 +291,21 @@ func newTestRuleSetChanges(mrs view.MappingRules, rrs view.RollupRules) changes. // nolint: unparam func testRuleSet(version int, meta rules.UpdateMetadata) (rules.RuleSet, error) { mutable := rules.NewEmptyRuleSet("testNamespace", meta) - err := mutable.ApplyRuleSetChanges( + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min), + ) + if err != nil { + return nil, err + } + + err = mutable.ApplyRuleSetChanges( changes.RuleSetChanges{ Namespace: "testNamespace", RollupRuleChanges: []changes.RollupRuleChange{ - changes.RollupRuleChange{ + { Op: changes.AddOp, RuleData: &view.RollupRule{ Name: "rollupRule3", @@ -303,12 +313,8 @@ func testRuleSet(version int, meta rules.UpdateMetadata) (rules.RuleSet, error) { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testTarget"), - Tags: [][]byte{[]byte("tag1"), []byte("tag2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Min), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -320,7 +326,7 @@ func testRuleSet(version int, meta rules.UpdateMetadata) (rules.RuleSet, error) }, }, MappingRuleChanges: []changes.MappingRuleChange{ - changes.MappingRuleChange{ + { Op: changes.AddOp, RuleData: &view.MappingRule{ Name: "mappingRule3", diff --git a/src/ctl/service/r2/store/stub/store.go b/src/ctl/service/r2/store/stub/store.go index 5464ef7ba9..f3f88c29a0 100644 --- a/src/ctl/service/r2/store/stub/store.go +++ b/src/ctl/service/r2/store/stub/store.go @@ -53,7 +53,107 @@ type stubData struct { var ( errNotImplemented = errors.New("not implemented") cutoverMillis = time.Now().UnixNano() / int64(time.Millisecond/time.Nanosecond) - dummyData = stubData{ +) + +// Operator contains the data necessary to implement stubbed out implementations for various r2 operations. +type store struct { + data *stubData + iOpts instrument.Options +} + +// NewStore creates a new stub +func NewStore(iOpts instrument.Options) (r2store.Store, error) { + dummyData, err := buildDummyData() + if err != nil { + return nil, err + } + + return &store{data: &dummyData, iOpts: iOpts}, err +} + +func buildDummyData() (stubData, error) { + rollup1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min), + ) + if err != nil { + return stubData{}, err + } + rollup2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min), + ) + if err != nil { + return stubData{}, err + } + rollup3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + if err != nil { + return stubData{}, err + } + rollup4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.P999), + ) + if err != nil { + return stubData{}, err + } + rollup5, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + if err != nil { + return stubData{}, err + } + rollup6, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.P999), + ) + if err != nil { + return stubData{}, err + } + rollup7, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + if err != nil { + return stubData{}, err + } + rollup8, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + if err != nil { + return stubData{}, err + } + rollup9, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testTarget", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.P999), + ) + if err != nil { + return stubData{}, err + } + return stubData{ ErrorNamespace: "errNs", ConflictNamespace: "conflictNs", Namespaces: view.Namespaces{ @@ -107,12 +207,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min), - }, + Type: pipeline.RollupOpType, + Rollup: rollup1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -130,12 +226,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min), - }, + Type: pipeline.RollupOpType, + Rollup: rollup2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -161,12 +253,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rollup3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -176,12 +264,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.P999), - }, + Type: pipeline.RollupOpType, + Rollup: rollup4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -194,7 +278,7 @@ var ( }, }, MappingHistory: map[string]mappingRuleHistories{ - "ns1": mappingRuleHistories{ + "ns1": { "mr_id1": []view.MappingRule{ { ID: "mr_id1", @@ -223,7 +307,7 @@ var ( "ns2": nil, }, RollupHistory: map[string]rollupRuleHistories{ - "ns1": rollupRuleHistories{ + "ns1": { "rr_id1": []view.RollupRule{ { ID: "rr_id1", @@ -234,12 +318,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rollup5, }, }), StoragePolicies: policy.StoragePolicies{ @@ -249,12 +329,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.P999), - }, + Type: pipeline.RollupOpType, + Rollup: rollup6, }, }), StoragePolicies: policy.StoragePolicies{ @@ -272,12 +348,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rollup7, }, }), StoragePolicies: policy.StoragePolicies{ @@ -288,7 +360,7 @@ var ( }, }, }, - "ns2": rollupRuleHistories{ + "ns2": { "rr_id3": []view.RollupRule{ { ID: "rr_id1", @@ -299,12 +371,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rollup8, }, }), StoragePolicies: policy.StoragePolicies{ @@ -314,12 +382,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testTarget"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.P999), - }, + Type: pipeline.RollupOpType, + Rollup: rollup9, }, }), StoragePolicies: policy.StoragePolicies{ @@ -331,18 +395,7 @@ var ( }, }, }, - } -) - -// Operator contains the data necessary to implement stubbed out implementations for various r2 operations. -type store struct { - data *stubData - iOpts instrument.Options -} - -// NewStore creates a new stub -func NewStore(iOpts instrument.Options) r2store.Store { - return &store{data: &dummyData, iOpts: iOpts} + }, nil } func (s *store) FetchNamespaces() (view.Namespaces, error) { diff --git a/src/dbnode/client/session.go b/src/dbnode/client/session.go index 6ffa119ad6..fc3bbf1708 100644 --- a/src/dbnode/client/session.go +++ b/src/dbnode/client/session.go @@ -1005,7 +1005,7 @@ func (s *session) setTopologyWithLock(topoMap topology.Map, queues []hostQueue, s.pools.multiReaderIteratorArray = encoding.NewMultiReaderIteratorArrayPool([]pool.Bucket{ { Capacity: replicas, - Count: s.opts.SeriesIteratorPoolSize(), + Count: pool.Size(s.opts.SeriesIteratorPoolSize()), }, }) s.pools.multiReaderIteratorArray.Init() diff --git a/src/dbnode/client/session_aggregate_test.go b/src/dbnode/client/session_aggregate_test.go index 2b1e5b67a9..aff8de1bc2 100644 --- a/src/dbnode/client/session_aggregate_test.go +++ b/src/dbnode/client/session_aggregate_test.go @@ -32,6 +32,7 @@ import ( "github.com/m3db/m3/src/m3ninx/idx" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xretry "github.com/m3db/m3/src/x/retry" xtest "github.com/m3db/m3/src/x/test" @@ -282,10 +283,11 @@ func TestSessionAggregateIDsEnqueueErr(t *testing.T) { assert.NoError(t, session.Open()) - _, _, err = session.Aggregate(testContext(), ident.StringID("namespace"), - testSessionAggregateQuery, testSessionAggregateQueryOpts(start, end)) - assert.Error(t, err) - assert.NoError(t, session.Close()) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { + _, _, _ = session.Aggregate(testContext(), ident.StringID("namespace"), + testSessionAggregateQuery, testSessionAggregateQueryOpts(start, end)) + }) } func TestSessionAggregateMergeTest(t *testing.T) { diff --git a/src/dbnode/client/session_fetch_tagged_test.go b/src/dbnode/client/session_fetch_tagged_test.go index 3505f19514..5c8e607baa 100644 --- a/src/dbnode/client/session_fetch_tagged_test.go +++ b/src/dbnode/client/session_fetch_tagged_test.go @@ -34,6 +34,7 @@ import ( "github.com/m3db/m3/src/m3ninx/idx" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xretry "github.com/m3db/m3/src/x/retry" xtest "github.com/m3db/m3/src/x/test" @@ -288,10 +289,11 @@ func TestSessionFetchTaggedIDsEnqueueErr(t *testing.T) { assert.NoError(t, session.Open()) - _, _, err = session.FetchTaggedIDs(testContext(), ident.StringID("namespace"), - testSessionFetchTaggedQuery, testSessionFetchTaggedQueryOpts(start, end)) - assert.Error(t, err) - assert.NoError(t, session.Close()) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { + _, _, _ = session.FetchTaggedIDs(testContext(), ident.StringID("namespace"), + testSessionFetchTaggedQuery, testSessionFetchTaggedQueryOpts(start, end)) + }) } func TestSessionFetchTaggedMergeTest(t *testing.T) { @@ -588,6 +590,15 @@ func mockExtendedHostQueues( require.Fail(t, "unable to find host idx: %v", host.ID()) return -1 } + expectClose := true + for _, hq := range opsByHost { + for _, e := range hq.enqueues { + if e.enqueueErr != nil { + expectClose = false + break + } + } + } s.newHostQueueFn = func( host topology.Host, opts hostQueueOpts, @@ -600,6 +611,7 @@ func mockExtendedHostQueues( hostQueue.EXPECT().Open() hostQueue.EXPECT().Host().Return(host).AnyTimes() hostQueue.EXPECT().ConnectionCount().Return(opts.opts.MinConnectionCount()).AnyTimes() + var expectNextEnqueueFn func(fns []testEnqueue) expectNextEnqueueFn = func(fns []testEnqueue) { fn := fns[0] @@ -619,7 +631,9 @@ func mockExtendedHostQueues( if len(hostEnqueues.enqueues) > 0 { expectNextEnqueueFn(hostEnqueues.enqueues) } - hostQueue.EXPECT().Close() + if expectClose { + hostQueue.EXPECT().Close() + } return hostQueue, nil } } diff --git a/src/dbnode/config/m3dbnode-all-config.yml b/src/dbnode/config/m3dbnode-all-config.yml index da6caf8692..1d3724b862 100644 --- a/src/dbnode/config/m3dbnode-all-config.yml +++ b/src/dbnode/config/m3dbnode-all-config.yml @@ -3,14 +3,6 @@ coordinator: # Address for M3Coordinator to listen for traffic. listenAddress: 0.0.0.0:7201 - # All configured M3DB namespaces must be listed in this config if running an - # embedded M3Coordinator instance. - local: - namespaces: - - namespace: default - type: unaggregated - retention: 48h - # Configuratino for M3Coordinator logging. logging: level: info diff --git a/src/dbnode/encoding/multi_reader_iterator_array_pool.go b/src/dbnode/encoding/multi_reader_iterator_array_pool.go index cd0330962c..c5ac88c36d 100644 --- a/src/dbnode/encoding/multi_reader_iterator_array_pool.go +++ b/src/dbnode/encoding/multi_reader_iterator_array_pool.go @@ -66,7 +66,7 @@ func (p *multiReaderIteratorArrayPool) Init() { for i := range p.sizesAsc { buckets[i].capacity = p.sizesAsc[i].Capacity buckets[i].values = make(chan []MultiReaderIterator, p.sizesAsc[i].Count) - for j := 0; j < p.sizesAsc[i].Count; j++ { + for j := 0; pool.Size(j) < p.sizesAsc[i].Count; j++ { buckets[i].values <- p.alloc(p.sizesAsc[i].Capacity) } } diff --git a/src/dbnode/encoding/mutable_series_iterators_pool.go b/src/dbnode/encoding/mutable_series_iterators_pool.go index 914b2c5b0e..0b9a2f5bc8 100644 --- a/src/dbnode/encoding/mutable_series_iterators_pool.go +++ b/src/dbnode/encoding/mutable_series_iterators_pool.go @@ -59,7 +59,7 @@ func (p *seriesIteratorsPool) Init() { for i := range p.sizesAsc { buckets[i].capacity = p.sizesAsc[i].Capacity buckets[i].values = make(chan MutableSeriesIterators, p.sizesAsc[i].Count) - for j := 0; j < p.sizesAsc[i].Count; j++ { + for j := 0; pool.Size(j) < p.sizesAsc[i].Count; j++ { buckets[i].values <- p.alloc(p.sizesAsc[i].Capacity) } } 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 1b7a42bbb4..e1fbdd8bf7 100644 --- a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go +++ b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go @@ -178,10 +178,8 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { setup.SetNowFn(now) setup.SleepFor10xTickMinimumInterval() - // Twice because the test bootstrapper will need to run two times, once to fulfill - // all historical blocks and once to fulfill the active block. - signalCh <- struct{}{} - signalCh <- struct{}{} + // Close signalCh to unblock bootstrapper and run the bootstrap till the end + close(signalCh) }() require.NoError(t, setup.StartServer()) // Blocks until bootstrap is complete 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 e4cf63d508..d5c8a143b2 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 @@ -192,10 +192,8 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { // tick. time.Sleep(existingOptions.TickMinimumInterval() * 10) - // Twice because the test bootstrapper will need to run two times, once to fulfill - // all historical blocks and once to fulfill the active block. - signalCh <- struct{}{} - signalCh <- struct{}{} + // Close signalCh to unblock bootstrapper and run the bootstrap till the end + close(signalCh) }() require.NoError(t, setup.StartServer()) // Blocks until bootstrap is complete diff --git a/src/dbnode/integration/bootstrap_retries_test.go b/src/dbnode/integration/bootstrap_retries_test.go new file mode 100644 index 0000000000..de8c9d2046 --- /dev/null +++ b/src/dbnode/integration/bootstrap_retries_test.go @@ -0,0 +1,264 @@ +// +build integration + +// Copyright (c) 2021 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 integration + +import ( + "errors" + "strings" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/uber-go/tally" + + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" + "github.com/m3db/m3/src/dbnode/storage" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" + "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" + "github.com/m3db/m3/src/x/context" +) + +func TestBootstrapRetriesDueToError(t *testing.T) { + // Setup the test bootstrapper to only proceed when a signal is sent. + signalCh := make(chan bool) + + setup, testScope := bootstrapRetryTestSetup(t, func( + ctx context.Context, + namespaces bootstrap.Namespaces, + cache bootstrap.Cache, + ) (bootstrap.NamespaceResults, error) { + shouldError := <-signalCh + if shouldError { + return bootstrap.NamespaceResults{}, errors.New("error in bootstrapper") + } + // Mark all as fulfilled + bs, err := bootstrapper.NewNoOpAllBootstrapperProvider().Provide() + require.NoError(t, err) + return bs.Bootstrap(ctx, namespaces, cache) + }) + + go func() { + // Wait for server to get started by the main test method. + setup.WaitUntilServerIsUp() + + // First bootstrap pass. Bootstrapper produces an error. Check if DB is not marked bootstrapped. + signalCh <- true + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + + // Bootstrap retry. Bootstrapper completes persist range without errors. Check if DB isn't + // marked as bootstrapped on the second pass. + signalCh <- false + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + + // Still bootstrap retry. Bootstrapper completes in-memory range without errors. DB finishes bootstrapping. + signalCh <- false + }() + + require.NoError(t, setup.StartServer()) // Blocks until bootstrap is complete + defer func() { + require.NoError(t, setup.StopServer()) + }() + + assert.True(t, setup.DB().IsBootstrapped(), "database should be bootstrapped") + assertRetryMetric(t, testScope, "other") +} + +func TestBootstrapRetriesDueToObsoleteRanges(t *testing.T) { + // Setup the test bootstrapper to only proceed when a signal is sent. + signalCh := make(chan struct{}) + + setup, testScope := bootstrapRetryTestSetup(t, func( + ctx context.Context, + namespaces bootstrap.Namespaces, + cache bootstrap.Cache, + ) (bootstrap.NamespaceResults, error) { + // read from signalCh twice so we could advance the clock exactly in between of those signals + <-signalCh + <-signalCh + bs, err := bootstrapper.NewNoOpAllBootstrapperProvider().Provide() + require.NoError(t, err) + return bs.Bootstrap(ctx, namespaces, cache) + }) + + go func() { + // Wait for server to get started by the main test method. + setup.WaitUntilServerIsUp() + + // First bootstrap pass, persist ranges. Check if DB is not marked bootstrapped and advance clock. + signalCh <- struct{}{} + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + setup.SetNowFn(setup.NowFn()().Add(2 * time.Hour)) + signalCh <- struct{}{} + + // Still first bootstrap pass, in-memory ranges. Due to advanced clock previously calculated + // ranges are obsolete. Check if DB is not marked bootstrapped. + signalCh <- struct{}{} + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + signalCh <- struct{}{} + + // Bootstrap retry, persist ranges. Check if DB isn't marked as bootstrapped on the second pass. + signalCh <- struct{}{} + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + signalCh <- struct{}{} + + // Still bootstrap retry, in-memory ranges. DB finishes bootstrapping. + signalCh <- struct{}{} + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + signalCh <- struct{}{} + }() + + require.NoError(t, setup.StartServer()) // Blocks until bootstrap is complete + defer func() { + require.NoError(t, setup.StopServer()) + }() + + assert.True(t, setup.DB().IsBootstrapped(), "database should be bootstrapped") + assertRetryMetric(t, testScope, "obsolete-ranges") +} + +func TestBootstrapRetriesDueToUnfulfilledRanges(t *testing.T) { + // Setup the test bootstrapper to only proceed when a signal is sent. + signalCh := make(chan bool) + + setup, testScope := bootstrapRetryTestSetup(t, func( + ctx context.Context, + namespaces bootstrap.Namespaces, + cache bootstrap.Cache, + ) (bootstrap.NamespaceResults, error) { + var provider bootstrap.BootstrapperProvider + shouldUnfulfill := <-signalCh + if shouldUnfulfill { + provider = bootstrapper.NewNoOpNoneBootstrapperProvider() + } else { + provider = bootstrapper.NewNoOpAllBootstrapperProvider() + } + bs, err := provider.Provide() + require.NoError(t, err) + return bs.Bootstrap(ctx, namespaces, cache) + }) + + go func() { + // Wait for server to get started by the main test method. + setup.WaitUntilServerIsUp() + + // First bootstrap pass. Bootstrap produces unfulfilled ranges for persist range. + // Check if DB is not marked bootstrapped. + signalCh <- true + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + // Still first bootstrap pass. Bootstrap produces unfulfilled ranges for in-memory range. + // Check if DB is not marked bootstrapped. + signalCh <- true + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + + // Bootstrap retry. Bootstrapper completes persist range fulfilling everything. + // Check if DB isn't marked as bootstrapped on the second pass. + signalCh <- false + assert.False(t, setup.DB().IsBootstrapped(), "database should not yet be bootstrapped") + + // Still bootstrap retry. Bootstrapper completes in-memory range fulfilling everything. + // DB finishes bootstrapping. + signalCh <- false + }() + + require.NoError(t, setup.StartServer()) // Blocks until bootstrap is complete + defer func() { + require.NoError(t, setup.StopServer()) + }() + + assert.True(t, setup.DB().IsBootstrapped(), "database should be bootstrapped") + + assertRetryMetric(t, testScope, "other") +} + +type bootstrapFn = func( + ctx context.Context, + namespaces bootstrap.Namespaces, + cache bootstrap.Cache, +) (bootstrap.NamespaceResults, error) + +func bootstrapRetryTestSetup(t *testing.T, bootstrapFn bootstrapFn) (TestSetup, tally.TestScope) { + testScope := tally.NewTestScope("testScope", map[string]string{}) + + rOpts := retention.NewOptions(). + SetRetentionPeriod(12 * time.Hour). + SetBufferPast(5 * time.Minute). + SetBufferFuture(5 * time.Minute) + + ns1, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(rOpts)) + require.NoError(t, err) + opts := NewTestOptions(t). + SetNamespaces([]namespace.Metadata{ns1}) + + setup, err := NewTestSetup(t, opts, nil, func(storageOpts storage.Options) storage.Options { + return storageOpts.SetInstrumentOptions(storageOpts.InstrumentOptions().SetMetricsScope(testScope)) + }) + require.NoError(t, err) + defer setup.Close() + + var ( + fsOpts = setup.StorageOpts().CommitLogOptions().FilesystemOptions() + + bootstrapOpts = newDefaulTestResultOptions(setup.StorageOpts()) + bootstrapperSourceOpts = testBootstrapperSourceOptions{read: bootstrapFn} + processOpts = bootstrap.NewProcessOptions(). + SetTopologyMapProvider(setup). + SetOrigin(setup.Origin()) + ) + bootstrapOpts.SetInstrumentOptions(bootstrapOpts.InstrumentOptions().SetMetricsScope(testScope)) + boostrapper := newTestBootstrapperSource(bootstrapperSourceOpts, bootstrapOpts, nil) + + processProvider, err := bootstrap.NewProcessProvider( + boostrapper, processOpts, bootstrapOpts, fsOpts) + require.NoError(t, err) + setup.SetStorageOpts(setup.StorageOpts().SetBootstrapProcessProvider(processProvider)) + return setup, testScope +} + +func assertRetryMetric(t *testing.T, testScope tally.TestScope, expectedReason string) { + const ( + metricName = "bootstrap-retries" + reasonTag = "reason" + ) + valuesByReason := make(map[string]int) + for _, counter := range testScope.Snapshot().Counters() { + if strings.Contains(counter.Name(), metricName) { + reason := "" + if r, ok := counter.Tags()[reasonTag]; ok { + reason = r + } + valuesByReason[reason] = int(counter.Value()) + } + } + + val, ok := valuesByReason[expectedReason] + if assert.True(t, ok, "missing metric for expected reason") { + assert.Equal(t, 1, val) + } + for r, val := range valuesByReason { + if r != expectedReason { + assert.Equal(t, 0, val) + } + } +} diff --git a/src/dbnode/integration/options.go b/src/dbnode/integration/options.go index 36193deeeb..5040ea627c 100644 --- a/src/dbnode/integration/options.go +++ b/src/dbnode/integration/options.go @@ -61,6 +61,9 @@ const ( // defaultTickMinimumInterval is the default minimum tick interval. defaultTickMinimumInterval = 1 * time.Second + // defaultTickCancellationCheckInterval is the default minimum tick cancellation check interval. + defaultTickCancellationCheckInterval = 1 * time.Second + // defaultUseTChannelClientForReading determines whether we use the tchannel client for reading by default. defaultUseTChannelClientForReading = false @@ -119,6 +122,12 @@ type TestOptions interface { // TickMinimumInterval returns the tick interval. TickMinimumInterval() time.Duration + // SetTickCancellationCheckInterval sets the tick cancellation check interval. + SetTickCancellationCheckInterval(value time.Duration) TestOptions + + // TickCancellationCheckInterval returns the tick cancellation check interval. + TickCancellationCheckInterval() time.Duration + // SetHTTPClusterAddr sets the http cluster address. SetHTTPClusterAddr(value string) TestOptions @@ -319,6 +328,7 @@ type options struct { nsInitializer namespace.Initializer id string tickMinimumInterval time.Duration + tickCancellationCheckInterval time.Duration httpClusterAddr string tchannelClusterAddr string httpNodeAddr string @@ -368,6 +378,7 @@ func NewTestOptions(t *testing.T) TestOptions { namespaces: namespaces, id: defaultID, tickMinimumInterval: defaultTickMinimumInterval, + tickCancellationCheckInterval: defaultTickCancellationCheckInterval, serverStateChangeTimeout: defaultServerStateChangeTimeout, clusterConnectionTimeout: defaultClusterConnectionTimeout, readRequestTimeout: defaultReadRequestTimeout, @@ -427,6 +438,16 @@ func (o *options) TickMinimumInterval() time.Duration { return o.tickMinimumInterval } +func (o *options) SetTickCancellationCheckInterval(value time.Duration) TestOptions { + opts := *o + opts.tickCancellationCheckInterval = value + return &opts +} + +func (o *options) TickCancellationCheckInterval() time.Duration { + return o.tickCancellationCheckInterval +} + func (o *options) SetHTTPClusterAddr(value string) TestOptions { opts := *o opts.httpClusterAddr = value diff --git a/src/dbnode/integration/series_wired_list_panic_test.go b/src/dbnode/integration/series_wired_list_panic_test.go new file mode 100644 index 0000000000..f3a26ad7eb --- /dev/null +++ b/src/dbnode/integration/series_wired_list_panic_test.go @@ -0,0 +1,216 @@ +// +build integration + +// Copyright (c) 2021 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 integration + +import ( + "fmt" + "testing" + "time" + + "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" + "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/persist/fs" + "github.com/m3db/m3/src/dbnode/sharding" + "github.com/m3db/m3/src/dbnode/storage" + "github.com/m3db/m3/src/dbnode/storage/block" + "github.com/m3db/m3/src/x/ident" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/stretchr/testify/require" +) + +const ( + numSeries = 10 +) + +var ( + nsID = ident.StringID("ns0") +) + +func TestWiredListPanic(t *testing.T) { + // This test is used to repro https://github.com/m3db/m3/issues/2573. + // Unfortunately, this bug is due to a race condition and this test does not + // consistently reproduce it reliably in short period of time. As such, the + // test is configured to run for a very long duration to see if the repro + // occurs. Comment out the below SkipNow() to actually run this. + t.SkipNow() + + // Small increment to make race condition more likely. + tickInterval := 5 * time.Millisecond + + nsOpts := namespace.NewOptions(). + SetRepairEnabled(false). + SetRetentionOptions(DefaultIntegrationTestRetentionOpts). + SetCacheBlocksOnRetrieve(true) + ns, err := namespace.NewMetadata(nsID, nsOpts) + require.NoError(t, err) + testOpts := NewTestOptions(t). + SetTickMinimumInterval(tickInterval). + SetTickCancellationCheckInterval(tickInterval). + SetNamespaces([]namespace.Metadata{ns}). + // Wired list size of one means that if we query for two different IDs + // alternating between each one, we'll evict from the wired list on + // every query. + SetMaxWiredBlocks(1) + + testSetup, err := NewTestSetup(t, testOpts, nil, + func(opts storage.Options) storage.Options { + return opts.SetMediatorTickInterval(tickInterval) + }, + func(opts storage.Options) storage.Options { + blockRetrieverMgr := block.NewDatabaseBlockRetrieverManager( + func( + md namespace.Metadata, + shardSet sharding.ShardSet, + ) (block.DatabaseBlockRetriever, error) { + retrieverOpts := fs.NewBlockRetrieverOptions(). + SetBlockLeaseManager(opts.BlockLeaseManager()). + SetCacheBlocksOnRetrieve(true) + retriever, err := fs.NewBlockRetriever(retrieverOpts, + opts.CommitLogOptions().FilesystemOptions()) + if err != nil { + return nil, err + } + + if err := retriever.Open(md, shardSet); err != nil { + return nil, err + } + return retriever, nil + }) + return opts.SetDatabaseBlockRetrieverManager(blockRetrieverMgr) + }, + ) + + require.NoError(t, err) + defer testSetup.Close() + + // Start the server. + log := testSetup.StorageOpts().InstrumentOptions().Logger() + require.NoError(t, testSetup.StartServer()) + log.Info("server is now up") + + // Stop the server. + defer func() { + require.NoError(t, testSetup.StopServer()) + log.Info("server is now down") + }() + + md := testSetup.NamespaceMetadataOrFail(nsID) + ropts := md.Options().RetentionOptions() + blockSize := ropts.BlockSize() + filePathPrefix := testSetup.StorageOpts().CommitLogOptions().FilesystemOptions().FilePathPrefix() + + seriesStrs := make([]string, 0, numSeries) + for i := 0; i < numSeries; i++ { + seriesStrs = append(seriesStrs, fmt.Sprintf("series-%d", i)) + } + + start := testSetup.NowFn()() + go func() { + for i := 0; true; i++ { + write(t, testSetup, blockSize, start, filePathPrefix, i, seriesStrs) + time.Sleep(5 * time.Millisecond) + } + }() + + doneCh := make(chan struct{}) + go func() { + for { + select { + case <-doneCh: + return + default: + read(t, testSetup, blockSize, seriesStrs) + time.Sleep(5 * time.Millisecond) + } + } + }() + + time.Sleep(time.Hour) + // Stop reads before tearing down testSetup. + doneCh <- struct{}{} +} + +func write( + t *testing.T, + testSetup TestSetup, + blockSize time.Duration, + start time.Time, + filePathPrefix string, + i int, + seriesStrs []string, +) { + blockStart := start.Add(time.Duration(2*i) * blockSize) + testSetup.SetNowFn(blockStart) + + input := generate.BlockConfig{ + IDs: seriesStrs, NumPoints: 1, Start: blockStart, + } + testData := generate.Block(input) + require.NoError(t, testSetup.WriteBatch(nsID, testData)) + + // Progress well past the block boundary so that the series gets flushed to + // disk. This allows the next tick to purge the series from memory, closing + // the series and thus making the id nil. + testSetup.SetNowFn(blockStart.Add(blockSize * 3 / 2)) + require.NoError(t, waitUntilFileSetFilesExist( + filePathPrefix, + []fs.FileSetFileIdentifier{ + { + Namespace: nsID, + Shard: 1, + BlockStart: blockStart, + VolumeIndex: 0, + }, + }, + time.Second, + )) +} + +func read( + t *testing.T, + testSetup TestSetup, + blockSize time.Duration, + seriesStrs []string, +) { + // After every write, "now" would be progressed into the future so that the + // will be flushed to disk. This makes "now" a suitable RangeEnd for the + // fetch request. The precise range does not matter so long as it overlaps + // with the current retention. + now := testSetup.NowFn()() + + req := rpc.NewFetchRequest() + req.NameSpace = nsID.String() + req.RangeStart = xtime.ToNormalizedTime(now.Add(-4*blockSize), time.Second) + req.RangeEnd = xtime.ToNormalizedTime(now, time.Second) + req.ResultTimeType = rpc.TimeType_UNIX_SECONDS + + // Fetching the series sequentially ensures that the wired list will have + // evictions assuming that the list is configured with a size of 1. + for _, seriesStr := range seriesStrs { + req.ID = seriesStr + _, err := testSetup.Fetch(req) + require.NoError(t, err) + } +} diff --git a/src/dbnode/integration/setup.go b/src/dbnode/integration/setup.go index aeb07747db..470ed068e0 100644 --- a/src/dbnode/integration/setup.go +++ b/src/dbnode/integration/setup.go @@ -294,6 +294,7 @@ func NewTestSetup( runtimeOptsMgr := storageOpts.RuntimeOptionsManager() runtimeOpts := runtimeOptsMgr.Get(). SetTickMinimumInterval(opts.TickMinimumInterval()). + SetTickCancellationCheckInterval(opts.TickCancellationCheckInterval()). SetMaxWiredBlocks(opts.MaxWiredBlocks()). SetWriteNewSeriesAsync(opts.WriteNewSeriesAsync()) if err := runtimeOptsMgr.Update(runtimeOpts); err != nil { diff --git a/src/dbnode/network/server/tchannelthrift/node/fetch_result_iter_test.go b/src/dbnode/network/server/tchannelthrift/node/fetch_result_iter_test.go index 78ef5e03ba..9445c8988a 100644 --- a/src/dbnode/network/server/tchannelthrift/node/fetch_result_iter_test.go +++ b/src/dbnode/network/server/tchannelthrift/node/fetch_result_iter_test.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/m3ninx/doc" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" ) func TestFetchResultIterTest(t *testing.T) { @@ -46,7 +47,7 @@ func TestFetchResultIterTest(t *testing.T) { scope, ctx, nsID, resMap, start, end, db := setup(mocks) - blockPermits := &fakePermits{available: 5} + blockPermits := &fakePermits{available: 5, quotaPerPermit: 5} iter := newFetchTaggedResultsIter(fetchTaggedResultsIterOpts{ queryResult: index.QueryResult{ Results: resMap, @@ -59,7 +60,6 @@ func TestFetchResultIterTest(t *testing.T) { db: db, nsID: nsID, blockPermits: blockPermits, - blocksPerBatch: 5, nowFn: time.Now, dataReadMetrics: index.NewQueryMetrics("", scope), totalMetrics: index.NewQueryMetrics("", scope), @@ -76,65 +76,36 @@ func TestFetchResultIterTest(t *testing.T) { iter.Close(nil) require.Equal(t, 10, total) - require.Equal(t, 5, blockPermits.acquired) - require.Equal(t, 5, blockPermits.released) + // 20 permits are not acquired because the accounting is not 100% accurate. permits are not acquired until + // after the block is processed, so a block might be eagerly processed and then permit acquisition fails. + require.Equal(t, 19, blockPermits.acquired) + require.Equal(t, 19, blockPermits.released) requireSeriesBlockMetric(t, scope) } -func TestFetchResultIterTestUnsetBlocksPerBatch(t *testing.T) { - mocks := gomock.NewController(t) - defer mocks.Finish() - - scope, ctx, nsID, resMap, start, end, db := setup(mocks) - - blockPermits := &fakePermits{available: 10} +func TestFetchResultIterTestNoReleaseWithoutAcquire(t *testing.T) { + blockPermits := &fakePermits{available: 10, quotaPerPermit: 1000} + emptyMap := index.NewQueryResults(ident.StringID("testNs"), index.QueryResultsOptions{}, testIndexOptions) + scope := tally.NewTestScope("", map[string]string{}) iter := newFetchTaggedResultsIter(fetchTaggedResultsIterOpts{ queryResult: index.QueryResult{ - Results: resMap, - }, - queryOpts: index.QueryOptions{ - StartInclusive: start, - EndExclusive: end, + Results: emptyMap, }, - fetchData: true, - db: db, - nsID: nsID, blockPermits: blockPermits, nowFn: time.Now, + instrumentClose: func(err error) {}, dataReadMetrics: index.NewQueryMetrics("", scope), totalMetrics: index.NewQueryMetrics("", scope), seriesBlocks: scope.Histogram("series-blocks", tally.MustMakeExponentialValueBuckets(10, 2, 5)), - instrumentClose: func(err error) {}, }) - total := 0 + ctx := context.NewBackground() for iter.Next(ctx) { - total++ - require.NotNil(t, iter.Current()) - require.Len(t, iter.Current().(*idResult).blockReaders, 10) } require.NoError(t, iter.Err()) iter.Close(nil) - require.Equal(t, 10, total) - require.Equal(t, 10, blockPermits.acquired) - require.Equal(t, 10, blockPermits.released) - requireSeriesBlockMetric(t, scope) -} - -func TestFetchResultIterTestForceBlocksPerBatch(t *testing.T) { - blockPermits := &permits.LookbackLimitPermit{} - resMap := index.NewQueryResults(ident.StringID("testNs"), index.QueryResultsOptions{}, testIndexOptions) - iter := newFetchTaggedResultsIter(fetchTaggedResultsIterOpts{ - queryResult: index.QueryResult{ - Results: resMap, - }, - blockPermits: blockPermits, - blocksPerBatch: 1000, - nowFn: time.Now, - }) - downcast, ok := iter.(*fetchTaggedResultsIter) - require.True(t, ok) - require.Equal(t, 1, downcast.blocksPerBatch) + require.Equal(t, 0, blockPermits.acquired) + require.Equal(t, 0, blockPermits.released) } func requireSeriesBlockMetric(t *testing.T, scope tally.TestScope) { @@ -178,30 +149,31 @@ func setup(mocks *gomock.Controller) ( } type fakePermits struct { - acquired int - released int - available int + acquired int + released int + available int + quotaPerPermit int64 } -func (p *fakePermits) Acquire(_ context.Context) error { +func (p *fakePermits) Acquire(_ context.Context) (permits.Permit, error) { if p.available == 0 { - return errors.New("available should never be 0") + return nil, errors.New("available should never be 0") } p.available-- p.acquired++ - return nil + return permits.NewPermit(p.quotaPerPermit, instrument.NewOptions()), nil } -func (p *fakePermits) TryAcquire(_ context.Context) (bool, error) { +func (p *fakePermits) TryAcquire(_ context.Context) (permits.Permit, error) { if p.available == 0 { - return false, nil + return nil, nil } p.available-- p.acquired++ - return true, nil + return permits.NewPermit(p.quotaPerPermit, instrument.NewOptions()), nil } -func (p *fakePermits) Release(_ int64) { +func (p *fakePermits) Release(_ permits.Permit) { p.released++ p.available++ } diff --git a/src/dbnode/network/server/tchannelthrift/node/service.go b/src/dbnode/network/server/tchannelthrift/node/service.go index 54ec72f068..d6ee6e4562 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service.go +++ b/src/dbnode/network/server/tchannelthrift/node/service.go @@ -871,7 +871,6 @@ func (s *service) fetchTaggedIter( fetchStart: startTime, dataReadMetrics: s.metrics.queryTimingDataRead, totalMetrics: s.metrics.queryTimingFetchTagged, - blocksPerBatch: s.opts.FetchTaggedSeriesBlocksPerBatch(), seriesBlocks: s.metrics.fetchTaggedSeriesBlocks, }), nil } @@ -912,8 +911,8 @@ type fetchTaggedResultsIter struct { blockReadIdx int cur IDResult err error - batchesAcquired int - blocksAvailable int + permits []permits.Permit + unreleasedQuota int64 dataReadStart time.Time totalSeriesBlocks int } @@ -929,7 +928,6 @@ type fetchTaggedResultsIterOpts struct { iOpts instrument.Options instrumentClose func(error) blockPermits permits.Permits - blocksPerBatch int nowFn clock.NowFn fetchStart time.Time totalDocsCount int @@ -939,17 +937,11 @@ type fetchTaggedResultsIterOpts struct { } func newFetchTaggedResultsIter(opts fetchTaggedResultsIterOpts) FetchTaggedResultsIter { //nolint: gocritic - _, limitBased := opts.blockPermits.(*permits.LookbackLimitPermit) - if opts.blocksPerBatch == 0 || limitBased { - // NB(nate): if blocksPerBatch is unset, set blocksPerBatch to 1 (i.e. acquire a permit - // for each block as opposed to acquiring in bulk). Additionally, limit-based permits - // are required to use a blocksPerBatch size of 1 so as to not throw off limit accounting. - opts.blocksPerBatch = 1 - } return &fetchTaggedResultsIter{ fetchTaggedResultsIterOpts: opts, idResults: make([]idResult, 0, opts.queryResult.Results.Map().Len()), dataReadStart: opts.nowFn(), + permits: make([]permits.Permit, 0), } } @@ -993,7 +985,7 @@ func (i *fetchTaggedResultsIter) Next(ctx context.Context) bool { } } else { // release the permits and memory from the previous block readers. - i.releaseAll(i.idx - 1) + i.releaseQuotaUsed(i.idx - 1) i.idResults[i.idx-1].blockReaders = nil } @@ -1038,37 +1030,45 @@ func (i *fetchTaggedResultsIter) Next(ctx context.Context) bool { // acquire a block permit for a series ID. returns true if a permit is available. func (i *fetchTaggedResultsIter) acquire(ctx context.Context, idx int) (bool, error) { - if i.blocksAvailable > 0 { - i.blocksAvailable-- - } else { + var curPermit permits.Permit + if len(i.permits) > 0 { + curPermit = i.permits[len(i.permits)-1] + } + if curPermit == nil || curPermit.QuotaRemaining() <= 0 { if i.idx == idx { // block acquiring if we need the block readers to fulfill the current fetch. - if err := i.blockPermits.Acquire(ctx); err != nil { + permit, err := i.blockPermits.Acquire(ctx) + if err != nil { return false, err } + i.permits = append(i.permits, permit) + curPermit = permit } else { // don't block if we are prefetching for a future seriesID. - acquired, err := i.blockPermits.TryAcquire(ctx) + permit, err := i.blockPermits.TryAcquire(ctx) if err != nil { return false, err } - if !acquired { + if permit == nil { return false, nil } + i.permits = append(i.permits, permit) + curPermit = permit } - i.batchesAcquired++ - i.blocksAvailable = i.blocksPerBatch - 1 } - i.idResults[idx].blocksAcquired++ + curPermit.Use(1) + i.idResults[idx].quotaUsed++ return true, nil } // release all the block permits acquired by a series ID that has been processed. -func (i *fetchTaggedResultsIter) releaseAll(idx int) { - // Note: the actual batch permits are not released until the query completely finishes and the iterator - // closes. - for n := 0; n < i.idResults[idx].blocksAcquired; n++ { - i.blocksAvailable++ +func (i *fetchTaggedResultsIter) releaseQuotaUsed(idx int) { + i.unreleasedQuota += i.idResults[idx].quotaUsed + for i.unreleasedQuota > 0 && i.unreleasedQuota >= i.permits[0].AllowedQuota() { + p := i.permits[0] + i.blockPermits.Release(p) + i.unreleasedQuota -= p.AllowedQuota() + i.permits = i.permits[1:] } } @@ -1092,10 +1092,9 @@ func (i *fetchTaggedResultsIter) Close(err error) { i.seriesBlocks.RecordValue(float64(i.totalSeriesBlocks)) - for n := 0; n < i.batchesAcquired-1; n++ { - i.blockPermits.Release(int64(i.blocksPerBatch)) + for _, p := range i.permits { + i.blockPermits.Release(p) } - i.blockPermits.Release(int64(i.blocksPerBatch - i.blocksAvailable)) } // IDResult is the FetchTagged result for a series ID. @@ -1119,7 +1118,7 @@ type idResult struct { tagEncoder serialize.TagEncoder blockReadersIter series.BlockReaderIter blockReaders [][]xio.BlockReader - blocksAcquired int + quotaUsed int64 iOpts instrument.Options } diff --git a/src/dbnode/network/server/tchannelthrift/types.go b/src/dbnode/network/server/tchannelthrift/types.go index 8e1d73ea0e..e7802c44e5 100644 --- a/src/dbnode/network/server/tchannelthrift/types.go +++ b/src/dbnode/network/server/tchannelthrift/types.go @@ -115,10 +115,6 @@ type Options interface { // SetPermitsOptions sets the permits options. SetPermitsOptions(value permits.Options) Options - // FetchTaggedSeriesBlocksPerBatch gets the series blocks allowed to be read - // per permit acquired. - FetchTaggedSeriesBlocksPerBatch() int - // SetFetchTaggedSeriesBlocksPerBatch sets the series blocks allowed to be read // per permit acquired. SetFetchTaggedSeriesBlocksPerBatch(value int) Options diff --git a/src/dbnode/persist/fs/files_test.go b/src/dbnode/persist/fs/files_test.go index 556bf9efb2..153be9c101 100644 --- a/src/dbnode/persist/fs/files_test.go +++ b/src/dbnode/persist/fs/files_test.go @@ -415,8 +415,10 @@ func TestFileExists(t *testing.T) { require.NoError(t, err) require.True(t, exists) - _, err = FileExists(checkpointFilePath) - require.Error(t, err) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { + _, _ = FileExists(checkpointFilePath) + }) os.Remove(infoFilePath) require.False(t, mustFileExists(t, infoFilePath)) @@ -447,9 +449,8 @@ func TestCompleteCheckpointFileExists(t *testing.T) { require.NoError(t, err) require.True(t, exists) - exists, err = CompleteCheckpointFileExists("some-arbitrary-file") - require.Contains(t, err.Error(), instrument.InvariantViolatedMetricName) - require.False(t, exists) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { _, _ = CompleteCheckpointFileExists("some-arbitrary-file") }) } func TestShardDirPath(t *testing.T) { diff --git a/src/dbnode/persist/fs/index_claims_manager_test.go b/src/dbnode/persist/fs/index_claims_manager_test.go index e32d3dc70a..951761c2de 100644 --- a/src/dbnode/persist/fs/index_claims_manager_test.go +++ b/src/dbnode/persist/fs/index_claims_manager_test.go @@ -29,6 +29,7 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" ) @@ -49,9 +50,8 @@ func TestIndexClaimsManagerSingleGlobalManager(t *testing.T) { require.NoError(t, err) // Second should cause an error. - _, err = NewIndexClaimsManager(testDefaultOpts) - require.Error(t, err) - require.Equal(t, errMustUseSingleClaimsManager, err) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { _, _ = NewIndexClaimsManager(testDefaultOpts) }) } func TestIndexClaimsManagerConcurrentClaims(t *testing.T) { diff --git a/src/dbnode/persist/fs/persist_manager_test.go b/src/dbnode/persist/fs/persist_manager_test.go index 362db4b5eb..49c6c73e2b 100644 --- a/src/dbnode/persist/fs/persist_manager_test.go +++ b/src/dbnode/persist/fs/persist_manager_test.go @@ -36,6 +36,7 @@ import ( m3ninxpersist "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" m3test "github.com/m3db/m3/src/x/test" xtest "github.com/m3db/m3/src/x/test" @@ -72,10 +73,9 @@ func TestPersistenceManagerPrepareDataFileExistsNoDelete(t *testing.T) { Shard: shard, BlockStart: blockStart, } - prepared, err := flush.PrepareData(prepareOpts) - require.Equal(t, errPersistManagerFileSetAlreadyExists, err) - require.Nil(t, prepared.Persist) - require.Nil(t, prepared.Close) + + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { _, _ = flush.PrepareData(prepareOpts) }) } func TestPersistenceManagerPrepareDataFileExistsWithDelete(t *testing.T) { diff --git a/src/dbnode/runtime/runtime_mock.go b/src/dbnode/runtime/runtime_mock.go index bb995f0c60..86b032e418 100644 --- a/src/dbnode/runtime/runtime_mock.go +++ b/src/dbnode/runtime/runtime_mock.go @@ -408,6 +408,34 @@ func (mr *MockOptionsMockRecorder) ClientWriteConsistencyLevel() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ClientWriteConsistencyLevel", reflect.TypeOf((*MockOptions)(nil).ClientWriteConsistencyLevel)) } +// SetTickCancellationCheckInterval mocks base method +func (m *MockOptions) SetTickCancellationCheckInterval(value time.Duration) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetTickCancellationCheckInterval", value) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetTickCancellationCheckInterval indicates an expected call of SetTickCancellationCheckInterval +func (mr *MockOptionsMockRecorder) SetTickCancellationCheckInterval(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTickCancellationCheckInterval", reflect.TypeOf((*MockOptions)(nil).SetTickCancellationCheckInterval), value) +} + +// TickCancellationCheckInterval mocks base method +func (m *MockOptions) TickCancellationCheckInterval() time.Duration { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TickCancellationCheckInterval") + ret0, _ := ret[0].(time.Duration) + return ret0 +} + +// TickCancellationCheckInterval indicates an expected call of TickCancellationCheckInterval +func (mr *MockOptionsMockRecorder) TickCancellationCheckInterval() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TickCancellationCheckInterval", reflect.TypeOf((*MockOptions)(nil).TickCancellationCheckInterval)) +} + // MockOptionsManager is a mock of OptionsManager interface type MockOptionsManager struct { ctrl *gomock.Controller diff --git a/src/dbnode/runtime/runtime_options.go b/src/dbnode/runtime/runtime_options.go index 656b2c4a40..3901968b25 100644 --- a/src/dbnode/runtime/runtime_options.go +++ b/src/dbnode/runtime/runtime_options.go @@ -44,6 +44,7 @@ const ( defaultTickSeriesBatchSize = 512 defaultTickPerSeriesSleepDuration = 100 * time.Microsecond defaultTickMinimumInterval = 10 * time.Second + defaultTickCancellationCheckInterval = time.Second defaultMaxWiredBlocks = uint(1 << 16) // 65,536 ) @@ -71,6 +72,7 @@ type options struct { clientBootstrapConsistencyLevel topology.ReadConsistencyLevel clientReadConsistencyLevel topology.ReadConsistencyLevel clientWriteConsistencyLevel topology.ConsistencyLevel + tickCancellationCheckInterval time.Duration } // NewOptions creates a new set of runtime options with defaults @@ -87,6 +89,7 @@ func NewOptions() Options { clientBootstrapConsistencyLevel: DefaultBootstrapConsistencyLevel, clientReadConsistencyLevel: DefaultReadConsistencyLevel, clientWriteConsistencyLevel: DefaultWriteConsistencyLevel, + tickCancellationCheckInterval: defaultTickCancellationCheckInterval, } } @@ -234,3 +237,13 @@ func (o *options) SetClientWriteConsistencyLevel(value topology.ConsistencyLevel func (o *options) ClientWriteConsistencyLevel() topology.ConsistencyLevel { return o.clientWriteConsistencyLevel } + +func (o *options) SetTickCancellationCheckInterval(value time.Duration) Options { + opts := *o + opts.tickCancellationCheckInterval = value + return &opts +} + +func (o *options) TickCancellationCheckInterval() time.Duration { + return o.tickCancellationCheckInterval +} diff --git a/src/dbnode/runtime/types.go b/src/dbnode/runtime/types.go index f65c72e72b..6c6ff1dae6 100644 --- a/src/dbnode/runtime/types.go +++ b/src/dbnode/runtime/types.go @@ -174,6 +174,14 @@ type Options interface { // ClientWriteConsistencyLevel returns the client write consistency level // used when fetching data from peers for coordinated writes ClientWriteConsistencyLevel() topology.ConsistencyLevel + + // SetTickCancellationCheckInterval sets the interval to check whether the tick + // has been canceled. This duration also affects the minimum tick duration. + SetTickCancellationCheckInterval(value time.Duration) Options + + // TickCancellationCheckInterval is the interval to check whether the tick + // has been canceled. This duration also affects the minimum tick duration. + TickCancellationCheckInterval() time.Duration } // OptionsManager updates and supplies runtime options. diff --git a/src/dbnode/server/server.go b/src/dbnode/server/server.go index 48f38856d1..1d29e7f21b 100644 --- a/src/dbnode/server/server.go +++ b/src/dbnode/server/server.go @@ -473,13 +473,18 @@ func Run(runOpts RunOptions) { defer seriesReadPermits.Stop() permitOptions := opts.PermitsOptions().SetSeriesReadPermitsManager(seriesReadPermits) - if cfg.Index.MaxQueryIDsConcurrency != 0 { - permitOptions = permitOptions.SetIndexQueryPermitsManager( - permits.NewFixedPermitsManager(cfg.Index.MaxQueryIDsConcurrency)) + maxIdxConcurrency := int(math.Ceil(float64(runtime.NumCPU()) / 2)) + if cfg.Index.MaxQueryIDsConcurrency > 0 { + maxIdxConcurrency = cfg.Index.MaxQueryIDsConcurrency } else { logger.Warn("max index query IDs concurrency was not set, falling back to default value") } - opts = opts.SetPermitsOptions(permitOptions) + maxWorkerTime := time.Second + if cfg.Index.MaxWorkerTime > 0 { + maxWorkerTime = cfg.Index.MaxWorkerTime + } + opts = opts.SetPermitsOptions(permitOptions.SetIndexQueryPermitsManager( + permits.NewFixedPermitsManager(maxIdxConcurrency, int64(maxWorkerTime), iOpts))) // Setup postings list cache. var ( @@ -522,10 +527,6 @@ func Run(runOpts RunOptions) { SetMmapReporter(mmapReporter). SetQueryLimits(queryLimits) - if cfg.Index.MaxWorkerTime > 0 { - indexOpts = indexOpts.SetMaxWorkerTime(cfg.Index.MaxWorkerTime) - } - opts = opts.SetIndexOptions(indexOpts) if tick := cfg.Tick; tick != nil { @@ -747,7 +748,6 @@ func Run(runOpts RunOptions) { SetMaxOutstandingWriteRequests(cfg.Limits.MaxOutstandingWriteRequests). SetMaxOutstandingReadRequests(cfg.Limits.MaxOutstandingReadRequests). SetQueryLimits(queryLimits). - SetFetchTaggedSeriesBlocksPerBatch(cfg.FetchTagged.SeriesBlocksPerBatchOrDefault()). SetPermitsOptions(opts.PermitsOptions()) // Start servers before constructing the DB so orchestration tools can check health endpoints @@ -1533,7 +1533,7 @@ func withEncodingAndPoolingOptions( logger.Info("bytes pool configured", zap.Int("capacity", bucket.CapacityOrDefault()), - zap.Int("size", bucket.SizeOrDefault()), + zap.Int("size", int(bucket.SizeOrDefault())), zap.Float64("refillLowWaterMark", bucket.RefillLowWaterMarkOrDefault()), zap.Float64("refillHighWaterMark", bucket.RefillHighWaterMarkOrDefault())) } @@ -1920,7 +1920,7 @@ func poolOptions( ) if size > 0 { - opts = opts.SetSize(size) + opts = opts.SetSize(int(size)) if refillLowWaterMark > 0 && refillHighWaterMark > 0 && refillHighWaterMark > refillLowWaterMark { @@ -1929,6 +1929,8 @@ func poolOptions( SetRefillHighWatermark(refillHighWaterMark) } } + opts = opts.SetDynamic(size.IsDynamic()) + if scope != nil { opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). SetMetricsScope(scope)) @@ -1948,7 +1950,7 @@ func capacityPoolOptions( ) if size > 0 { - opts = opts.SetSize(size) + opts = opts.SetSize(int(size)) if refillLowWaterMark > 0 && refillHighWaterMark > 0 && refillHighWaterMark > refillLowWaterMark { @@ -1956,6 +1958,8 @@ func capacityPoolOptions( opts = opts.SetRefillHighWatermark(refillHighWaterMark) } } + opts = opts.SetDynamic(size.IsDynamic()) + if scope != nil { opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). SetMetricsScope(scope)) @@ -1975,7 +1979,7 @@ func maxCapacityPoolOptions( ) if size > 0 { - opts = opts.SetSize(size) + opts = opts.SetSize(int(size)) if refillLowWaterMark > 0 && refillHighWaterMark > 0 && refillHighWaterMark > refillLowWaterMark { @@ -1983,6 +1987,8 @@ func maxCapacityPoolOptions( opts = opts.SetRefillHighWatermark(refillHighWaterMark) } } + opts = opts.SetDynamic(size.IsDynamic()) + if scope != nil { opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). SetMetricsScope(scope)) diff --git a/src/dbnode/storage/bootstrap.go b/src/dbnode/storage/bootstrap.go index f0e5765ce8..c616e8cc01 100644 --- a/src/dbnode/storage/bootstrap.go +++ b/src/dbnode/storage/bootstrap.go @@ -164,7 +164,7 @@ func (m *bootstrapManager) Bootstrap() (BootstrapResult, error) { // 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.instrumentation.bootstrapFailed(i + 1) + m.instrumentation.bootstrapFailed(i+1, bootstrapErr) m.sleepFn(bootstrapRetryInterval) continue } diff --git a/src/dbnode/storage/bootstrap/process.go b/src/dbnode/storage/bootstrap/process.go index c7b1c62392..7d4e3de61c 100644 --- a/src/dbnode/storage/bootstrap/process.go +++ b/src/dbnode/storage/bootstrap/process.go @@ -21,6 +21,7 @@ package bootstrap import ( + "errors" "fmt" "sync" "time" @@ -58,6 +59,11 @@ const ( bootstrapIndexRunType = bootstrapRunType("bootstrap-index") ) +// ErrFileSetSnapshotTypeRangeAdvanced is an error of bootstrap time ranges for snapshot-type +// blocks advancing during the bootstrap +var ErrFileSetSnapshotTypeRangeAdvanced = errors.New("the time ranges of snapshot-type blocks " + + "have advanced since they were calculated at the beginning of bootstrap") + // NewProcessProvider creates a new bootstrap process provider. func NewProcessProvider( bootstrapperProvider BootstrapperProvider, @@ -236,6 +242,32 @@ func (b bootstrapProcess) Run( namespacesRunFirst, namespacesRunSecond, } { + for _, entry := range namespaces.Namespaces.Iter() { + ns := entry.Value() + // Check if snapshot-type ranges have advanced while bootstrapping previous ranges. + // If yes, return an error to force a retry + if persistConf := ns.DataRunOptions.RunOptions.PersistConfig(); persistConf.Enabled && + persistConf.FileSetType == persist.FileSetSnapshotType { + var ( + now = b.nowFn() + nsOptions = ns.Metadata.Options() + upToDateDataRanges = b.targetRangesForData(now, nsOptions.RetentionOptions()) + ) + // Only checking data ranges. Since index blocks can only be a multiple of + // data block size, the ranges for index could advance only if data ranges + // have advanced, too (while opposite is not necessarily true) + if !upToDateDataRanges.secondRangeWithPersistFalse.Range.Equal(ns.DataTargetRange.Range) { + upToDateIndexRanges := b.targetRangesForIndex(now, nsOptions.RetentionOptions(), + nsOptions.IndexOptions()) + fields := b.logFields(ns.Metadata, ns.Shards, + upToDateDataRanges.secondRangeWithPersistFalse.Range, + upToDateIndexRanges.secondRangeWithPersistFalse.Range) + b.log.Error("time ranges of snapshot-type blocks advanced", fields...) + return NamespaceResults{}, ErrFileSetSnapshotTypeRangeAdvanced + } + } + } + res, err := b.runPass(ctx, namespaces, cache) if err != nil { return NamespaceResults{}, err diff --git a/src/dbnode/storage/bootstrap/process_test.go b/src/dbnode/storage/bootstrap/process_test.go new file mode 100644 index 0000000000..069ca42b5e --- /dev/null +++ b/src/dbnode/storage/bootstrap/process_test.go @@ -0,0 +1,92 @@ +// Copyright (c) 2021 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 ( + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/persist/fs" + "github.com/m3db/m3/src/dbnode/retention" + "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" + "github.com/m3db/m3/src/dbnode/topology" + xcontext "github.com/m3db/m3/src/x/context" + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" +) + +func TestBootstrapProcessRunActiveBlockAdvanced(t *testing.T) { + var ( + ctrl = gomock.NewController(t) + ctx = xcontext.NewBackground() + + blockSize = time.Hour + startTime = time.Now().Truncate(blockSize) + bufferPast = 30 * time.Minute + bufferFuture = 30 * time.Minute + // shift 'now' just enough so that after adding 'bufferFuture' it would reach the next block + now = startTime.Add(blockSize - bufferFuture) + + retentionOpts = retention.NewOptions(). + SetBlockSize(blockSize). + SetRetentionPeriod(12 * blockSize). + SetBufferPast(bufferPast). + SetBufferFuture(bufferFuture) + nsOptions = namespace.NewOptions().SetRetentionOptions(retentionOpts) + nsID = ident.StringID("ns") + ns, err = namespace.NewMetadata(nsID, nsOptions) + ) + require.NoError(t, err) + + processNs := []ProcessNamespace{ + { + Metadata: ns, + Shards: []uint32{0}, + DataAccumulator: NewMockNamespaceDataAccumulator(ctrl), + }, + } + + bootstrapper := NewMockBootstrapper(ctrl) + bootstrapper.EXPECT().String().Return("mock_bootstrapper").AnyTimes() + bootstrapper.EXPECT(). + Bootstrap(gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn(func(_, _, _ interface{}) (NamespaceResults, error) { + return NewNamespaceResults(NewNamespaces(processNs)), nil + }). + AnyTimes() + + process := bootstrapProcess{ + processOpts: NewProcessOptions(), + resultOpts: result.NewOptions(), + fsOpts: fs.NewOptions(), + nowFn: func() time.Time { return now }, + log: instrument.NewOptions().Logger(), + bootstrapper: bootstrapper, + initialTopologyState: &topology.StateSnapshot{}, + } + + _, err = process.Run(ctx, startTime, processNs) + require.Equal(t, ErrFileSetSnapshotTypeRangeAdvanced, err) +} diff --git a/src/dbnode/storage/bootstrap_instrumentation.go b/src/dbnode/storage/bootstrap_instrumentation.go index 37e593fd92..4a5f2421a7 100644 --- a/src/dbnode/storage/bootstrap_instrumentation.go +++ b/src/dbnode/storage/bootstrap_instrumentation.go @@ -27,7 +27,9 @@ import ( "go.uber.org/zap" "go.uber.org/zap/zapcore" + "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/x/clock" + xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" ) @@ -111,6 +113,25 @@ func (i *instrumentationContext) emitAndLogInvariantViolation(err error, msg str }) } +type bootstrapRetriesMetrics struct { + obsoleteRanges tally.Counter + other tally.Counter +} + +func newBootstrapRetriesMetrics(scope tally.Scope) bootstrapRetriesMetrics { + const metricName = "bootstrap-retries" + + reason := func(reason string) map[string]string { + return map[string]string{ + "reason": reason, + } + } + return bootstrapRetriesMetrics{ + obsoleteRanges: scope.Tagged(reason("obsolete-ranges")).Counter(metricName), + other: scope.Tagged(reason("other")).Counter(metricName), + } +} + type bootstrapInstrumentation struct { opts Options scope tally.Scope @@ -118,7 +139,7 @@ type bootstrapInstrumentation struct { nowFn clock.NowFn status tally.Gauge durableStatus tally.Gauge - numRetries tally.Counter + numRetries bootstrapRetriesMetrics } func newBootstrapInstrumentation(opts Options) *bootstrapInstrumentation { @@ -130,7 +151,7 @@ func newBootstrapInstrumentation(opts Options) *bootstrapInstrumentation { nowFn: opts.ClockOptions().NowFn(), status: scope.Gauge("bootstrapped"), durableStatus: scope.Gauge("bootstrapped-durable"), - numRetries: scope.Counter("bootstrap-retries"), + numRetries: newBootstrapRetriesMetrics(scope), } } @@ -139,8 +160,13 @@ func (i *bootstrapInstrumentation) bootstrapPreparing() *instrumentationContext return newInstrumentationContext(i.nowFn, i.log, i.scope, i.opts) } -func (i *bootstrapInstrumentation) bootstrapFailed(retry int) { - i.numRetries.Inc(1) +func (i *bootstrapInstrumentation) bootstrapFailed(retry int, err error) { + numRetries := i.numRetries.other + if xerrors.Is(err, bootstrap.ErrFileSetSnapshotTypeRangeAdvanced) { + numRetries = i.numRetries.obsoleteRanges + } + numRetries.Inc(1) + i.log.Warn("retrying bootstrap after backoff", zap.Duration("backoff", bootstrapRetryInterval), zap.Int("numRetries", retry)) diff --git a/src/dbnode/storage/bootstrap_instrumentation_test.go b/src/dbnode/storage/bootstrap_instrumentation_test.go new file mode 100644 index 0000000000..84ac56a302 --- /dev/null +++ b/src/dbnode/storage/bootstrap_instrumentation_test.go @@ -0,0 +1,90 @@ +// Copyright (c) 2021 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" + "strings" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/uber-go/tally" + + "github.com/m3db/m3/src/dbnode/storage/bootstrap" + xerrors "github.com/m3db/m3/src/x/errors" +) + +func TestBootstrapRetryMetricReason(t *testing.T) { + tests := []struct { + name string + err error + expectedReason string + }{ + { + name: "any error", + err: xerrors.NewInvalidParamsError(errors.New("some error")), + expectedReason: "other", + }, + { + name: "obsolete ranges error", + err: xerrors.NewInvalidParamsError(bootstrap.ErrFileSetSnapshotTypeRangeAdvanced), + expectedReason: "obsolete-ranges", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + testScope := tally.NewTestScope("testScope", map[string]string{}) + options := NewOptions() + options = options.SetInstrumentOptions(options.InstrumentOptions().SetMetricsScope(testScope)) + instr := newBootstrapInstrumentation(options) + + instr.bootstrapFailed(1, tt.err) + + retryMetrics := getBootstrapRetryMetrics(testScope) + assert.NotEmpty(t, retryMetrics) + assert.Equal(t, 1, retryMetrics[tt.expectedReason], "metric with reason:%v should be set to 1") + for reason, value := range retryMetrics { + if reason != tt.expectedReason { + assert.Equal(t, 0, value, "metric with reason:%v should stay 0") + } + } + }) + } +} + +func getBootstrapRetryMetrics(testScope tally.TestScope) map[string]int { + const ( + metricName = "bootstrap-retries" + reasonTag = "reason" + ) + valuesByReason := make(map[string]int) + for _, counter := range testScope.Snapshot().Counters() { + if strings.Contains(counter.Name(), metricName) { + reason := "" + if r, ok := counter.Tags()[reasonTag]; ok { + reason = r + } + valuesByReason[reason] = int(counter.Value()) + } + } + return valuesByReason +} diff --git a/src/dbnode/storage/fs_test.go b/src/dbnode/storage/fs_test.go index 17bff01cc6..9ae349bf1b 100644 --- a/src/dbnode/storage/fs_test.go +++ b/src/dbnode/storage/fs_test.go @@ -27,6 +27,8 @@ import ( "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" + + "github.com/m3db/m3/src/x/instrument" ) func TestFileSystemManagerShouldRunDuringBootstrap(t *testing.T) { @@ -86,9 +88,8 @@ func TestFileSystemManagerRun(t *testing.T) { ts := time.Now() gomock.InOrder( cm.EXPECT().WarmFlushCleanup(ts, true).Return(errors.New("foo")), - fm.EXPECT().Flush(ts).Return(errors.New("bar")), ) - mgr.Run(ts, syncRun, noForce) - require.Equal(t, fileOpNotStarted, mgr.status) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { mgr.Run(ts, syncRun, noForce) }) } diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 18766491e9..92a8bfd0ae 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -123,7 +123,6 @@ type nsIndex struct { aggregateResultsPool index.AggregateResultsPool permitsManager permits.Manager - maxWorkerTime time.Duration // queriesWg tracks outstanding queries to ensure // we wait for all queries to complete before actually closing @@ -209,6 +208,7 @@ type newNamespaceIndexOpts struct { type execBlockQueryFn func( ctx context.Context, block index.Block, + permit permits.Permit, iter index.ResultIterator, opts index.QueryOptions, state *asyncQueryExecState, @@ -369,7 +369,6 @@ func newNamespaceIndexWithOptions( doNotIndexWithFields: doNotIndexWithFields, shardSet: shardSet, - maxWorkerTime: indexOpts.MaxWorkerTime(), } // Assign shard set upfront. @@ -1594,7 +1593,7 @@ func (i *nsIndex) queryWithSpan( state = &asyncQueryExecState{} wg sync.WaitGroup ) - permits, err := i.permitsManager.NewPermits(ctx) + perms, err := i.permitsManager.NewPermits(ctx) if err != nil { return false, err } @@ -1623,25 +1622,25 @@ func (i *nsIndex) queryWithSpan( queryCanceled := func() bool { return opts.LimitsExceeded(results.Size(), results.TotalDocsCount()) || state.hasErr() } - // waitForPermit waits for a permit. returns true if the permit was acquired and the wait time. - waitForPermit := func() (bool, time.Duration) { + // waitForPermit waits for a permit. returns non-nil if the permit was acquired and the wait time. + waitForPermit := func() (permits.Permit, time.Duration) { // make sure the query hasn't been canceled before waiting for a permit. if queryCanceled() { - return false, 0 + return nil, 0 } startWait := time.Now() - err := permits.Acquire(ctx) + permit, err := perms.Acquire(ctx) waitTime := time.Since(startWait) if err != nil { state.addErr(err) - return false, waitTime + return nil, waitTime } // make sure the query hasn't been canceled while waiting for a permit. if queryCanceled() { - permits.Release(0) - return false, waitTime + perms.Release(permit) + return nil, waitTime } - return true, waitTime + return permit, waitTime } // We're looping through all the blocks that we need to query and kicking @@ -1655,9 +1654,9 @@ func (i *nsIndex) queryWithSpan( // acquire a permit before kicking off the goroutine to process the iterator. this limits the number of // concurrent goroutines to # of permits + large queries that needed multiple iterations to finish. - acq, waitTime := waitForPermit() + permit, waitTime := waitForPermit() blockIter.waitTime += waitTime - if !acq { + if permit == nil { break } @@ -1669,23 +1668,24 @@ func (i *nsIndex) queryWithSpan( for !blockIter.iter.Done() { // if this is not the first iteration of the iterator, need to acquire another permit. if !first { - acq, waitTime := waitForPermit() + permit, waitTime = waitForPermit() blockIter.waitTime += waitTime - if !acq { + if permit == nil { break } } first = false startProcessing := time.Now() - execBlockFn(ctx, blockIter.block, blockIter.iter, opts, state, results, logFields) + execBlockFn(ctx, blockIter.block, permit, blockIter.iter, opts, state, results, logFields) processingTime := time.Since(startProcessing) queryMetrics.blockProcessingTime.RecordDuration(processingTime) blockIter.processingTime += processingTime - permits.Release(int64(processingTime)) + permit.Use(int64(processingTime)) + perms.Release(permit) } if first { // this should never happen since a new iter cannot be Done, but just to be safe. - permits.Release(0) + perms.Release(permit) } blockIter.searchTime += blockIter.iter.SearchDuration() @@ -1749,6 +1749,7 @@ func (i *nsIndex) newBlockQueryIterFn( func (i *nsIndex) execBlockQueryFn( ctx context.Context, block index.Block, + permit permits.Permit, iter index.ResultIterator, opts index.QueryOptions, state *asyncQueryExecState, @@ -1775,7 +1776,8 @@ func (i *nsIndex) execBlockQueryFn( return } - err := block.QueryWithIter(ctx, opts, queryIter, docResults, time.Now().Add(i.maxWorkerTime), logFields) + deadline := time.Now().Add(time.Duration(permit.AllowedQuota())) + err := block.QueryWithIter(ctx, opts, queryIter, docResults, deadline, logFields) if err == index.ErrUnableToQueryBlockClosed { // NB(r): Because we query this block outside of the results lock, it's // possible this block may get closed if it slides out of retention, in @@ -1793,6 +1795,7 @@ func (i *nsIndex) execBlockQueryFn( func (i *nsIndex) execBlockWideQueryFn( ctx context.Context, block index.Block, + permit permits.Permit, iter index.ResultIterator, opts index.QueryOptions, state *asyncQueryExecState, @@ -1819,7 +1822,8 @@ func (i *nsIndex) execBlockWideQueryFn( return } - err := block.QueryWithIter(ctx, opts, queryIter, docResults, time.Now().Add(i.maxWorkerTime), logFields) + deadline := time.Now().Add(time.Duration(permit.AllowedQuota())) + err := block.QueryWithIter(ctx, opts, queryIter, docResults, deadline, logFields) if err == index.ErrUnableToQueryBlockClosed { // NB(r): Because we query this block outside of the results lock, it's // possible this block may get closed if it slides out of retention, in @@ -1854,6 +1858,7 @@ func (i *nsIndex) newBlockAggregatorIterFn( func (i *nsIndex) execBlockAggregateQueryFn( ctx context.Context, block index.Block, + permit permits.Permit, iter index.ResultIterator, opts index.QueryOptions, state *asyncQueryExecState, @@ -1880,7 +1885,8 @@ func (i *nsIndex) execBlockAggregateQueryFn( return } - err := block.AggregateWithIter(ctx, aggIter, opts, aggResults, time.Now().Add(i.maxWorkerTime), logFields) + deadline := time.Now().Add(time.Duration(permit.AllowedQuota())) + err := block.AggregateWithIter(ctx, aggIter, opts, aggResults, deadline, logFields) if err == index.ErrUnableToQueryBlockClosed { // NB(r): Because we query this block outside of the results lock, it's // possible this block may get closed if it slides out of retention, in diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index fa10affc80..a4d290e113 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -2387,31 +2387,3 @@ func (mr *MockOptionsMockRecorder) QueryLimits() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "QueryLimits", reflect.TypeOf((*MockOptions)(nil).QueryLimits)) } - -// MaxWorkerTime mocks base method -func (m *MockOptions) MaxWorkerTime() time.Duration { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "MaxWorkerTime") - ret0, _ := ret[0].(time.Duration) - return ret0 -} - -// MaxWorkerTime indicates an expected call of MaxWorkerTime -func (mr *MockOptionsMockRecorder) MaxWorkerTime() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MaxWorkerTime", reflect.TypeOf((*MockOptions)(nil).MaxWorkerTime)) -} - -// SetMaxWorkerTime mocks base method -func (m *MockOptions) SetMaxWorkerTime(value time.Duration) Options { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SetMaxWorkerTime", value) - ret0, _ := ret[0].(Options) - return ret0 -} - -// SetMaxWorkerTime indicates an expected call of SetMaxWorkerTime -func (mr *MockOptionsMockRecorder) SetMaxWorkerTime(value interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetMaxWorkerTime", reflect.TypeOf((*MockOptions)(nil).SetMaxWorkerTime), value) -} diff --git a/src/dbnode/storage/index/options.go b/src/dbnode/storage/index/options.go index 3e4328a473..055ab95cac 100644 --- a/src/dbnode/storage/index/options.go +++ b/src/dbnode/storage/index/options.go @@ -22,7 +22,6 @@ package index import ( "errors" - "time" "github.com/m3db/m3/src/dbnode/storage/index/compaction" "github.com/m3db/m3/src/dbnode/storage/limits" @@ -82,8 +81,6 @@ var ( defaultForegroundCompactionOpts compaction.PlannerOptions defaultBackgroundCompactionOpts compaction.PlannerOptions - // defaultMaxWorkerTime sets the default time a query can hold an index worker. - defaultMaxWorkerTime = time.Second ) func init() { @@ -138,7 +135,6 @@ type opts struct { readThroughSegmentOptions ReadThroughSegmentOptions mmapReporter mmap.Reporter queryLimits limits.QueryLimits - maxWorkerTime time.Duration } var undefinedUUIDFn = func() ([]byte, error) { return nil, errIDGenerationDisabled } @@ -199,7 +195,6 @@ func NewOptions() Options { foregroundCompactionPlannerOpts: defaultForegroundCompactionOpts, backgroundCompactionPlannerOpts: defaultBackgroundCompactionOpts, queryLimits: limits.NoOpQueryLimits(), - maxWorkerTime: defaultMaxWorkerTime, } resultsPool.Init(func() QueryResults { return NewQueryResults(nil, QueryResultsOptions{}, opts) @@ -465,13 +460,3 @@ func (o *opts) SetQueryLimits(value limits.QueryLimits) Options { func (o *opts) QueryLimits() limits.QueryLimits { return o.queryLimits } - -func (o *opts) MaxWorkerTime() time.Duration { - return o.maxWorkerTime -} - -func (o *opts) SetMaxWorkerTime(value time.Duration) Options { - opts := *o - opts.maxWorkerTime = value - return &opts -} diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index 5e30c4f72b..27b549c175 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -1097,10 +1097,4 @@ type Options interface { // QueryLimits returns the current query limits. QueryLimits() limits.QueryLimits - - // MaxWorkerTime returns the max time a query can hold an index worker. - MaxWorkerTime() time.Duration - - // SetMaxWorkerTime sets MaxWorkerTime. - SetMaxWorkerTime(value time.Duration) Options } diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index 71f5fb8284..1298558338 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -40,6 +40,7 @@ import ( "github.com/m3db/m3/src/x/context" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xtest "github.com/m3db/m3/src/x/test" xtime "github.com/m3db/m3/src/x/time" @@ -196,11 +197,13 @@ func TestNamespaceIndexNewBlockFnRandomErr(t *testing.T) { ) (index.Block, error) { return nil, fmt.Errorf("randomerr") } + defer instrument.SetShouldPanicEnvironmentVariable(true)() md := testNamespaceMetadata(blockSize, 4*time.Hour) - _, err := newNamespaceIndexWithNewBlockFn(md, - namespace.NewRuntimeOptionsManager(md.ID().String()), - testShardSet, newBlockFn, opts) - require.Error(t, err) + require.Panics(t, func() { + _, _ = newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) + }) } func TestNamespaceIndexWrite(t *testing.T) { diff --git a/src/dbnode/storage/index_query_concurrent_test.go b/src/dbnode/storage/index_query_concurrent_test.go index fb5f24127e..2114129717 100644 --- a/src/dbnode/storage/index_query_concurrent_test.go +++ b/src/dbnode/storage/index_query_concurrent_test.go @@ -40,6 +40,7 @@ import ( "github.com/m3db/m3/src/m3ninx/idx" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/instrument" xtest "github.com/m3db/m3/src/x/test" "github.com/fortytw2/leaktest" @@ -123,7 +124,7 @@ func testNamespaceIndexHighConcurrentQueries( nsIdx := test.index.(*nsIndex) nsIdx.state.Lock() // Make the query pool really high to improve concurrency likelihood - nsIdx.permitsManager = permits.NewFixedPermitsManager(1000) + nsIdx.permitsManager = permits.NewFixedPermitsManager(1000, int64(time.Millisecond), instrument.NewOptions()) currNow := min nowLock := &sync.Mutex{} diff --git a/src/dbnode/storage/limits/permits/fixed_permits.go b/src/dbnode/storage/limits/permits/fixed_permits.go index 13db162038..0ce934ad3e 100644 --- a/src/dbnode/storage/limits/permits/fixed_permits.go +++ b/src/dbnode/storage/limits/permits/fixed_permits.go @@ -21,11 +21,15 @@ package permits import ( + "go.uber.org/zap" + "github.com/m3db/m3/src/x/context" + "github.com/m3db/m3/src/x/instrument" ) type fixedPermits struct { - permits chan struct{} + permits chan Permit + iOpts instrument.Options } type fixedPermitsManager struct { @@ -38,54 +42,60 @@ var ( ) // NewFixedPermitsManager returns a permits manager that uses a fixed size of permits. -func NewFixedPermitsManager(size int) Manager { - fp := fixedPermits{permits: make(chan struct{}, size)} +func NewFixedPermitsManager(size int, quotaPerPermit int64, iOpts instrument.Options) Manager { + fp := fixedPermits{permits: make(chan Permit, size), iOpts: iOpts} for i := 0; i < size; i++ { - fp.permits <- struct{}{} + fp.permits <- NewPermit(quotaPerPermit, iOpts) } - return &fixedPermitsManager{fp} + return &fixedPermitsManager{fp: fp} } func (f *fixedPermitsManager) NewPermits(_ context.Context) (Permits, error) { return &f.fp, nil } -func (f *fixedPermits) Acquire(ctx context.Context) error { +func (f *fixedPermits) Acquire(ctx context.Context) (Permit, error) { // don't acquire a permit if ctx is already done. select { case <-ctx.GoContext().Done(): - return ctx.GoContext().Err() + return nil, ctx.GoContext().Err() default: } select { case <-ctx.GoContext().Done(): - return ctx.GoContext().Err() - case <-f.permits: - return nil + return nil, ctx.GoContext().Err() + case p := <-f.permits: + p.PreAcquire() + return p, nil } } -func (f *fixedPermits) TryAcquire(ctx context.Context) (bool, error) { +func (f *fixedPermits) TryAcquire(ctx context.Context) (Permit, error) { // don't acquire a permit if ctx is already done. select { case <-ctx.GoContext().Done(): - return false, ctx.GoContext().Err() + return nil, ctx.GoContext().Err() default: } select { - case <-f.permits: - return true, nil + case p := <-f.permits: + p.PreAcquire() + return p, nil default: - return false, nil + return nil, nil } } -func (f *fixedPermits) Release(_ int64) { +func (f *fixedPermits) Release(permit Permit) { + permit.PostRelease() + select { - case f.permits <- struct{}{}: + case f.permits <- permit: default: - panic("more permits released than acquired") + instrument.EmitAndLogInvariantViolation(f.iOpts, func(l *zap.Logger) { + l.Error("more permits released than acquired") + }) } } diff --git a/src/dbnode/storage/limits/permits/fixed_permits_test.go b/src/dbnode/storage/limits/permits/fixed_permits_test.go index 7b6f0dd80e..10153d8710 100644 --- a/src/dbnode/storage/limits/permits/fixed_permits_test.go +++ b/src/dbnode/storage/limits/permits/fixed_permits_test.go @@ -27,41 +27,71 @@ import ( "github.com/stretchr/testify/require" "github.com/m3db/m3/src/x/context" + "github.com/m3db/m3/src/x/instrument" ) func TestFixedPermits(t *testing.T) { ctx := context.NewBackground() - fp, err := NewFixedPermitsManager(3).NewPermits(ctx) + iOpts := instrument.NewOptions() + fp, err := NewFixedPermitsManager(3, 1, iOpts).NewPermits(ctx) require.NoError(t, err) - require.NoError(t, fp.Acquire(ctx)) - require.NoError(t, fp.Acquire(ctx)) - require.NoError(t, fp.Acquire(ctx)) + expectedP := NewPermit(1, iOpts) + expectedP.(*permit).refCount.Inc() + p, err := fp.Acquire(ctx) + require.NoError(t, err) + require.Equal(t, expectedP, p) + p, err = fp.Acquire(ctx) + require.NoError(t, err) + require.Equal(t, expectedP, p) + p, err = fp.Acquire(ctx) + require.NoError(t, err) + require.Equal(t, expectedP, p) - acq, err := fp.TryAcquire(ctx) + tryP, err := fp.TryAcquire(ctx) require.NoError(t, err) - require.False(t, acq) + require.Nil(t, tryP) - fp.Release(0) - require.NoError(t, fp.Acquire(ctx)) + fp.Release(p) + p, err = fp.Acquire(ctx) + require.NoError(t, err) + require.Equal(t, expectedP, p) +} + +func TestPanics(t *testing.T) { + ctx := context.NewBackground() + iOpts := instrument.NewOptions() + fp, err := NewFixedPermitsManager(3, 1, iOpts).NewPermits(ctx) + require.NoError(t, err) + p, err := fp.Acquire(ctx) + require.NoError(t, err) + fp.Release(p) + + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { fp.Release(p) }) } func TestFixedPermitsTimeouts(t *testing.T) { ctx := context.NewBackground() - fp, err := NewFixedPermitsManager(1).NewPermits(ctx) + iOpts := instrument.NewOptions() + fp, err := NewFixedPermitsManager(1, 1, iOpts).NewPermits(ctx) + expectedP := NewPermit(1, iOpts) + expectedP.(*permit).refCount.Inc() + require.NoError(t, err) + p, err := fp.Acquire(ctx) require.NoError(t, err) - require.NoError(t, fp.Acquire(ctx)) + require.Equal(t, expectedP, p) - acq, err := fp.TryAcquire(ctx) + tryP, err := fp.TryAcquire(ctx) require.NoError(t, err) - require.False(t, acq) + require.Nil(t, tryP) stdCtx, cancel := stdctx.WithCancel(stdctx.Background()) cancel() ctx = context.NewWithGoContext(stdCtx) - fp.Release(0) + fp.Release(p) - err = fp.Acquire(ctx) + _, err = fp.Acquire(ctx) require.Error(t, err) _, err = fp.TryAcquire(ctx) diff --git a/src/dbnode/storage/limits/permits/lookback_limit_permit.go b/src/dbnode/storage/limits/permits/lookback_limit_permit.go index b495e37596..4b2b35111a 100644 --- a/src/dbnode/storage/limits/permits/lookback_limit_permit.go +++ b/src/dbnode/storage/limits/permits/lookback_limit_permit.go @@ -44,7 +44,12 @@ type LookbackLimitPermit struct { var _ Manager = (*LookbackLimitPermitManager)(nil) -var _ Permits = (*LookbackLimitPermit)(nil) +var ( + _ Permits = (*LookbackLimitPermit)(nil) + // use a single permit for everybody to avoid allocations. since limits don't track quotas it's fine + // to share the same instance. + singlePermit = &limitPermit{} +) // NewLookbackLimitPermitsManager builds a new lookback limit permits manager. func NewLookbackLimitPermitsManager( @@ -87,19 +92,18 @@ func (p *LookbackLimitPermitManager) Stop() { } // Acquire increments the underlying querying limit. -func (p *LookbackLimitPermit) Acquire(context.Context) error { - return p.limit.Inc(1, p.source) +func (p *LookbackLimitPermit) Acquire(context.Context) (Permit, error) { + return singlePermit, p.limit.Inc(1, p.source) } // TryAcquire increments the underlying querying limit. Functionally equivalent // to Acquire. -func (p *LookbackLimitPermit) TryAcquire(context.Context) (bool, error) { - err := p.limit.Inc(1, p.source) - return err != nil, err +func (p *LookbackLimitPermit) TryAcquire(context.Context) (Permit, error) { + return singlePermit, p.limit.Inc(1, p.source) } // Release is a no-op in this implementation. -func (p *LookbackLimitPermit) Release(_ int64) { +func (p *LookbackLimitPermit) Release(_ Permit) { } func sourceFromContext(ctx context.Context) []byte { @@ -110,3 +114,22 @@ func sourceFromContext(ctx context.Context) []byte { } return parsed } + +type limitPermit struct{} + +func (l limitPermit) PostRelease() { +} + +func (l limitPermit) PreAcquire() { +} + +func (l limitPermit) AllowedQuota() int64 { + return 1 +} + +func (l limitPermit) QuotaRemaining() int64 { + return 0 +} + +func (l limitPermit) Use(_ int64) { +} diff --git a/src/dbnode/storage/limits/permits/lookback_limit_permit_test.go b/src/dbnode/storage/limits/permits/lookback_limit_permit_test.go index da9f0d5bbd..6492b0ab14 100644 --- a/src/dbnode/storage/limits/permits/lookback_limit_permit_test.go +++ b/src/dbnode/storage/limits/permits/lookback_limit_permit_test.go @@ -40,7 +40,8 @@ func TestLookbackLimitPermit(t *testing.T) { require.Equal(t, 0, lookbackLimit.count) - require.NoError(t, permits.Acquire(ctx)) + _, err = permits.Acquire(ctx) + require.NoError(t, err) require.Equal(t, 1, lookbackLimit.count) _, err = permits.TryAcquire(ctx) diff --git a/src/dbnode/storage/limits/permits/noop_permit.go b/src/dbnode/storage/limits/permits/noop_permit.go deleted file mode 100644 index 6afb174b4a..0000000000 --- a/src/dbnode/storage/limits/permits/noop_permit.go +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright (c) 2021 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 permits - -import "github.com/m3db/m3/src/x/context" - -type noOpPermits struct { -} - -var _ Manager = (*noOpPermits)(nil) - -var _ Permits = (*noOpPermits)(nil) - -// NewNoOpPermitsManager builds a new no-op permits manager. -func NewNoOpPermitsManager() Manager { - return &noOpPermits{} -} - -// NewNoOpPermits builds a new no-op permits. -func NewNoOpPermits() Permits { - return &noOpPermits{} -} - -func (p noOpPermits) NewPermits(context.Context) (Permits, error) { - return p, nil -} - -func (p noOpPermits) Acquire(context.Context) error { - return nil -} - -func (p noOpPermits) TryAcquire(context.Context) (bool, error) { - return true, nil -} - -func (p noOpPermits) Release(_ int64) { -} diff --git a/src/dbnode/storage/limits/permits/options.go b/src/dbnode/storage/limits/permits/options.go index 7e74cb1899..68dac8c6fa 100644 --- a/src/dbnode/storage/limits/permits/options.go +++ b/src/dbnode/storage/limits/permits/options.go @@ -21,8 +21,9 @@ package permits import ( - "math" - "runtime" + "time" + + "github.com/m3db/m3/src/x/instrument" ) type options struct { @@ -32,10 +33,16 @@ type options struct { // NewOptions return a new set of default permit managers. func NewOptions() Options { + // provide some defaults to exercise parallel processing in tests. return &options{ - seriesReadManager: NewNoOpPermitsManager(), - // Default to using half of the available cores for querying IDs - indexQueryManager: NewFixedPermitsManager(int(math.Ceil(float64(runtime.NumCPU()) / 2))), + seriesReadManager: NewFixedPermitsManager( + 100000, + 100, + instrument.NewOptions()), + indexQueryManager: NewFixedPermitsManager( + 8, + int64(time.Millisecond*10), + instrument.NewOptions()), } } diff --git a/src/dbnode/storage/limits/permits/permit.go b/src/dbnode/storage/limits/permits/permit.go new file mode 100644 index 0000000000..daa9e2b991 --- /dev/null +++ b/src/dbnode/storage/limits/permits/permit.go @@ -0,0 +1,83 @@ +// Copyright (c) 2021 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 permits + +import ( + "go.uber.org/atomic" + "go.uber.org/zap" + + "github.com/m3db/m3/src/x/instrument" +) + +// permit tracks the quota used by a caller and provides basic sanity checks that a caller +// correctly uses the permit. +type permit struct { + // immutable state + quota int64 + iOpts instrument.Options + + // mutable state + quotaUsed int64 + // refCount is used to check if a caller incorrectly double releases/acquires a permit. the value should + // always be 0 (nobody holds the permit) or 1 (somebody holds the permit). + refCount atomic.Int32 +} + +// NewPermit constructs a new Permit with the provided quota. +func NewPermit(quota int64, iOpts instrument.Options) Permit { + return &permit{ + quota: quota, + iOpts: iOpts, + } +} + +func (p *permit) PostRelease() { + if p.iOpts != nil && p.refCount.Dec() != 0 { + instrument.EmitAndLogInvariantViolation(p.iOpts, func(l *zap.Logger) { + l.Error("permit released more than once") + }) + } +} + +func (p *permit) PreAcquire() { + if p.iOpts != nil && p.refCount.Inc() != 1 { + instrument.EmitAndLogInvariantViolation(p.iOpts, func(l *zap.Logger) { + l.Error("permit acquired more than once") + }) + } + p.quotaUsed = 0 +} + +// AllowedQuota is the amount of quota the caller can use with this Permit. +func (p *permit) AllowedQuota() int64 { + return p.quota +} + +// QuotaRemaining is the amount of remaining quota for this Permit. Can be negative if the caller used more quota +// than they were allowed. +func (p *permit) QuotaRemaining() int64 { + return p.quota - p.quotaUsed +} + +// Use adds the quota to the total used quota. +func (p *permit) Use(quota int64) { + p.quotaUsed += quota +} diff --git a/src/dbnode/storage/limits/permits/types.go b/src/dbnode/storage/limits/permits/types.go index 61c69248b8..043ceff92f 100644 --- a/src/dbnode/storage/limits/permits/types.go +++ b/src/dbnode/storage/limits/permits/types.go @@ -21,7 +21,9 @@ // Package permits contains logic for granting permits to resources. package permits -import "github.com/m3db/m3/src/x/context" +import ( + "github.com/m3db/m3/src/x/context" +) // Options is the permit options. type Options interface { @@ -43,15 +45,37 @@ type Manager interface { // Permits are the set of permits that individual codepaths will utilize. type Permits interface { - // Acquire blocks until an available resource is made available for the request permit - Acquire(ctx context.Context) error + // Acquire blocks until a Permit is available. The returned Permit is guaranteed to be non-nil if error is + // non-nil. + Acquire(ctx context.Context) (Permit, error) // TryAcquire attempts to acquire an available resource without blocking, returning - // true if an resource was acquired. - TryAcquire(ctx context.Context) (bool, error) + // a non-nil a Permit if one is available. Returns nil if no Permit is currently available. + TryAcquire(ctx context.Context) (Permit, error) - // Release gives back one acquired permit from the specific permits instance. The user can pass an optional quota - // indicating how much of quota was used while holding the permit. + // Release gives back one acquired permit from the specific permits instance. // Cannot release more permits than have been acquired. - Release(quota int64) + Release(permit Permit) +} + +// Permit is granted to a caller which is allowed to consume some amount of quota. +type Permit interface { + + // AllowedQuota is the amount of quota the caller can use with this Permit. + AllowedQuota() int64 + + // QuotaRemaining is the amount of remaining quota for this Permit. Can be negative if the caller used more quota + // than they were allowed. + QuotaRemaining() int64 + + // Use adds the quota to the total used quota. + Use(quota int64) + + // PostRelease is called by the Manager after a caller releases the permit back. + // Provides a hook for the Manager. Clients should not call this method. + PostRelease() + + // PreAcquire is called by the Manager before giving the permit to the caller. + // Provides a hook for the Manager. Clients should not call this method. + PreAcquire() } diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index da99357e0d..a2ba1e95b5 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -490,6 +490,8 @@ func TestNamespaceBootstrapOnlyNonBootstrappedShards(t *testing.T) { ctx := context.NewBackground() defer ctx.Close() + // do not panic for invariant violation to test some shards are still bootstrapped. + defer instrument.SetShouldPanicEnvironmentVariable(false)() require.Error(t, ns.Bootstrap(ctx, nsResult)) require.Equal(t, BootstrapNotStarted, ns.bootstrapState) } diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 574c77e4ad..eb21bf2498 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -592,8 +592,9 @@ func (s *dbSeries) OnEvictedFromWiredList(id ident.ID, blockStart time.Time) { s.Lock() defer s.Unlock() - // Should never happen - if !id.Equal(s.id) { + // id can be nil at this point if this dbSeries gets closed just before it + // gets evicted from the wiredlist. + if id == nil || s.id == nil || !id.Equal(s.id) { return } diff --git a/src/dbnode/storage/tick.go b/src/dbnode/storage/tick.go index 8a95d2f54b..557ce04001 100644 --- a/src/dbnode/storage/tick.go +++ b/src/dbnode/storage/tick.go @@ -34,8 +34,7 @@ import ( ) const ( - tokenCheckInterval = time.Second - cancellationCheckInterval = time.Second + tokenCheckInterval = time.Second ) var ( @@ -95,7 +94,8 @@ func (o *tickManagerRuntimeOptions) values() tickManagerRuntimeOptionsValues { } type tickManagerRuntimeOptionsValues struct { - tickMinInterval time.Duration + tickMinInterval time.Duration + tickCancellationCheckInterval time.Duration } func newTickManager(database database, opts Options) databaseTickManager { @@ -120,7 +120,8 @@ func newTickManager(database database, opts Options) databaseTickManager { func (mgr *tickManager) SetRuntimeOptions(opts runtime.Options) { mgr.runtimeOpts.set(tickManagerRuntimeOptionsValues{ - tickMinInterval: opts.TickMinimumInterval(), + tickMinInterval: opts.TickMinimumInterval(), + tickCancellationCheckInterval: opts.TickCancellationCheckInterval(), }) } @@ -182,11 +183,12 @@ func (mgr *tickManager) Tick(forceType forceType, startTime time.Time) error { took := mgr.nowFn().Sub(start) mgr.metrics.tickWorkDuration.Record(took) - min := mgr.runtimeOpts.values().tickMinInterval + vals := mgr.runtimeOpts.values() + min := vals.tickMinInterval // Sleep in a loop so that cancellations propagate if need to // wait to fulfill the tick min interval - interval := cancellationCheckInterval + interval := vals.tickCancellationCheckInterval for d := time.Duration(0); d < min-took; d += interval { if mgr.c.IsCancelled() { break @@ -194,7 +196,7 @@ func (mgr *tickManager) Tick(forceType forceType, startTime time.Time) error { mgr.sleepFn(interval) // Check again at the end of each sleep to see if it // has changed. Particularly useful for integration tests. - min = mgr.runtimeOpts.values().tickMinInterval + min = vals.tickMinInterval } end := mgr.nowFn() diff --git a/src/metrics/aggregation/id.go b/src/metrics/aggregation/id.go index ceef5d6deb..f6e28d5e8e 100644 --- a/src/metrics/aggregation/id.go +++ b/src/metrics/aggregation/id.go @@ -141,21 +141,13 @@ func (id *ID) UnmarshalYAML(unmarshal func(interface{}) error) error { } // ToProto converts the aggregation id to a protobuf message in place. -func (id ID) ToProto(pb *aggregationpb.AggregationID) error { - if IDLen != 1 { - return fmt.Errorf("id length %d cannot be represented by a single integer", IDLen) - } +func (id ID) ToProto(pb *aggregationpb.AggregationID) { pb.Id = id[0] - return nil } // FromProto converts the protobuf message to an aggregation id in place. -func (id *ID) FromProto(pb aggregationpb.AggregationID) error { - if IDLen != 1 { - return fmt.Errorf("id length %d cannot be represented by a single integer", IDLen) - } +func (id *ID) FromProto(pb aggregationpb.AggregationID) { (*id)[0] = pb.Id - return nil } // CompressTypes compresses a list of aggregation types to an ID. @@ -172,3 +164,10 @@ func MustCompressTypes(aggTypes ...Type) ID { } return res } + +func init() { + if IDLen != 1 { + // changing this const requires extensive surgery + panic(fmt.Sprintf("id length %d cannot be represented by a single integer", IDLen)) + } +} diff --git a/src/metrics/aggregation/id_test.go b/src/metrics/aggregation/id_test.go index 20eef73b46..9e2615c3f9 100644 --- a/src/metrics/aggregation/id_test.go +++ b/src/metrics/aggregation/id_test.go @@ -39,13 +39,13 @@ var ( func TestIDToProto(t *testing.T) { var pb aggregationpb.AggregationID - require.NoError(t, testID.ToProto(&pb)) + testID.ToProto(&pb) require.Equal(t, testIDProto, pb) } func TestIDFromProto(t *testing.T) { var res ID - require.NoError(t, res.FromProto(testIDProto)) + res.FromProto(testIDProto) require.Equal(t, testID, res) } @@ -54,8 +54,8 @@ func TestIDRoundTrip(t *testing.T) { pb aggregationpb.AggregationID res ID ) - require.NoError(t, testID.ToProto(&pb)) - require.NoError(t, res.FromProto(pb)) + testID.ToProto(&pb) + res.FromProto(pb) require.Equal(t, testID, res) } diff --git a/src/metrics/encoding/protobuf/unaggregated_encoder_test.go b/src/metrics/encoding/protobuf/unaggregated_encoder_test.go index 7c3ad0026d..2b9063d31b 100644 --- a/src/metrics/encoding/protobuf/unaggregated_encoder_test.go +++ b/src/metrics/encoding/protobuf/unaggregated_encoder_test.go @@ -253,7 +253,7 @@ var ( } testPassthroughMetadata1 = policy.NewStoragePolicy(time.Minute, xtime.Minute, 12*time.Hour) testPassthroughMetadata2 = policy.NewStoragePolicy(10*time.Second, xtime.Second, 6*time.Hour) - testCounter1Proto = metricpb.Counter{ + testCounter1Proto = metricpb.Counter{ Id: []byte("testCounter1"), Value: 123, } @@ -673,11 +673,11 @@ func TestUnaggregatedEncoderEncodeBatchTimerWithMetadatas(t *testing.T) { enc.(*unaggregatedEncoder).encodeMessageFn = func(pb metricpb.MetricWithMetadatas) error { pbRes = pb; return nil } for i, input := range inputs { require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, + Type: encoding.BatchTimerWithMetadatasType, BatchTimerWithMetadatas: input, })) expectedProto := metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_BATCH_TIMER_WITH_METADATAS, + Type: metricpb.MetricWithMetadatas_BATCH_TIMER_WITH_METADATAS, BatchTimerWithMetadatas: &expected[i], } expectedMsgSize := expectedProto.Size() @@ -793,11 +793,11 @@ func TestUnaggregatedEncoderEncodeForwardedMetricWithMetadata(t *testing.T) { enc.(*unaggregatedEncoder).encodeMessageFn = func(pb metricpb.MetricWithMetadatas) error { pbRes = pb; return nil } for i, input := range inputs { require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.ForwardedMetricWithMetadataType, + Type: encoding.ForwardedMetricWithMetadataType, ForwardedMetricWithMetadata: input, })) expectedProto := metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_FORWARDED_METRIC_WITH_METADATA, + Type: metricpb.MetricWithMetadatas_FORWARDED_METRIC_WITH_METADATA, ForwardedMetricWithMetadata: &expected[i], } expectedMsgSize := expectedProto.Size() @@ -853,11 +853,11 @@ func TestUnaggregatedEncoderEncodeTimedMetricWithMetadata(t *testing.T) { enc.(*unaggregatedEncoder).encodeMessageFn = func(pb metricpb.MetricWithMetadatas) error { pbRes = pb; return nil } for i, input := range inputs { require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.TimedMetricWithMetadataType, + Type: encoding.TimedMetricWithMetadataType, TimedMetricWithMetadata: input, })) expectedProto := metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATA, + Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATA, TimedMetricWithMetadata: &expected[i], } expectedMsgSize := expectedProto.Size() @@ -1121,12 +1121,12 @@ func TestUnaggregatedEncoderStress(t *testing.T) { } case unaggregated.BatchTimerWithMetadatas: msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, + Type: encoding.BatchTimerWithMetadatasType, BatchTimerWithMetadatas: input, } res := expected[i].(metricpb.BatchTimerWithMetadatas) expectedProto = metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_BATCH_TIMER_WITH_METADATAS, + Type: metricpb.MetricWithMetadatas_BATCH_TIMER_WITH_METADATAS, BatchTimerWithMetadatas: &res, } case unaggregated.GaugeWithMetadatas: @@ -1141,32 +1141,32 @@ func TestUnaggregatedEncoderStress(t *testing.T) { } case aggregated.ForwardedMetricWithMetadata: msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.ForwardedMetricWithMetadataType, + Type: encoding.ForwardedMetricWithMetadataType, ForwardedMetricWithMetadata: input, } res := expected[i].(metricpb.ForwardedMetricWithMetadata) expectedProto = metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_FORWARDED_METRIC_WITH_METADATA, + Type: metricpb.MetricWithMetadatas_FORWARDED_METRIC_WITH_METADATA, ForwardedMetricWithMetadata: &res, } case aggregated.TimedMetricWithMetadata: msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.TimedMetricWithMetadataType, + Type: encoding.TimedMetricWithMetadataType, TimedMetricWithMetadata: input, } res := expected[i].(metricpb.TimedMetricWithMetadata) expectedProto = metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATA, + Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATA, TimedMetricWithMetadata: &res, } case aggregated.PassthroughMetricWithMetadata: msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.PassthroughMetricWithMetadataType, + Type: encoding.PassthroughMetricWithMetadataType, PassthroughMetricWithMetadata: input, } res := expected[i].(metricpb.TimedMetricWithStoragePolicy) expectedProto = metricpb.MetricWithMetadatas{ - Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_STORAGE_POLICY, + Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_STORAGE_POLICY, TimedMetricWithStoragePolicy: &res, } default: diff --git a/src/metrics/generated/proto/pipelinepb/pipeline.pb.go b/src/metrics/generated/proto/pipelinepb/pipeline.pb.go index 58fc1a9054..4a18f57d97 100644 --- a/src/metrics/generated/proto/pipelinepb/pipeline.pb.go +++ b/src/metrics/generated/proto/pipelinepb/pipeline.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: github.com/m3db/m3/src/metrics/generated/proto/pipelinepb/pipeline.proto -// Copyright (c) 2020 Uber Technologies, Inc. +// Copyright (c) 2021 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 @@ -59,6 +59,27 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +type RollupOp_Type int32 + +const ( + RollupOp_GROUP_BY RollupOp_Type = 0 + RollupOp_EXCLUDE_BY RollupOp_Type = 1 +) + +var RollupOp_Type_name = map[int32]string{ + 0: "GROUP_BY", + 1: "EXCLUDE_BY", +} +var RollupOp_Type_value = map[string]int32{ + "GROUP_BY": 0, + "EXCLUDE_BY": 1, +} + +func (x RollupOp_Type) String() string { + return proto.EnumName(RollupOp_Type_name, int32(x)) +} +func (RollupOp_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorPipeline, []int{2, 0} } + type PipelineOp_Type int32 const ( @@ -148,6 +169,7 @@ type RollupOp struct { NewName string `protobuf:"bytes,1,opt,name=new_name,json=newName,proto3" json:"new_name,omitempty"` Tags []string `protobuf:"bytes,2,rep,name=tags" json:"tags,omitempty"` AggregationTypes []aggregationpb.AggregationType `protobuf:"varint,3,rep,packed,name=aggregation_types,json=aggregationTypes,enum=aggregationpb.AggregationType" json:"aggregation_types,omitempty"` + Type RollupOp_Type `protobuf:"varint,4,opt,name=type,proto3,enum=pipelinepb.RollupOp_Type" json:"type,omitempty"` } func (m *RollupOp) Reset() { *m = RollupOp{} } @@ -176,6 +198,13 @@ func (m *RollupOp) GetAggregationTypes() []aggregationpb.AggregationType { return nil } +func (m *RollupOp) GetType() RollupOp_Type { + if m != nil { + return m.Type + } + return RollupOp_GROUP_BY +} + type PipelineOp struct { Type PipelineOp_Type `protobuf:"varint,1,opt,name=type,proto3,enum=pipelinepb.PipelineOp_Type" json:"type,omitempty"` Aggregation *AggregationOp `protobuf:"bytes,2,opt,name=aggregation" json:"aggregation,omitempty"` @@ -319,6 +348,7 @@ func init() { proto.RegisterType((*AppliedRollupOp)(nil), "pipelinepb.AppliedRollupOp") proto.RegisterType((*AppliedPipelineOp)(nil), "pipelinepb.AppliedPipelineOp") proto.RegisterType((*AppliedPipeline)(nil), "pipelinepb.AppliedPipeline") + proto.RegisterEnum("pipelinepb.RollupOp_Type", RollupOp_Type_name, RollupOp_Type_value) proto.RegisterEnum("pipelinepb.PipelineOp_Type", PipelineOp_Type_name, PipelineOp_Type_value) proto.RegisterEnum("pipelinepb.AppliedPipelineOp_Type", AppliedPipelineOp_Type_name, AppliedPipelineOp_Type_value) } @@ -421,6 +451,11 @@ func (m *RollupOp) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintPipeline(dAtA, i, uint64(j1)) i += copy(dAtA[i:], dAtA2[:j1]) } + if m.Type != 0 { + dAtA[i] = 0x20 + i++ + i = encodeVarintPipeline(dAtA, i, uint64(m.Type)) + } return i, nil } @@ -659,6 +694,9 @@ func (m *RollupOp) Size() (n int) { } n += 1 + sovPipeline(uint64(l)) + l } + if m.Type != 0 { + n += 1 + sovPipeline(uint64(m.Type)) + } return n } @@ -1036,6 +1074,25 @@ func (m *RollupOp) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field AggregationTypes", wireType) } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (RollupOp_Type(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPipeline(dAtA[iNdEx:]) @@ -1743,42 +1800,45 @@ func init() { } var fileDescriptorPipeline = []byte{ - // 587 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x4d, 0x6e, 0xd3, 0x40, - 0x1c, 0xc5, 0x3b, 0x76, 0x94, 0xa6, 0xff, 0xd0, 0x34, 0x1d, 0x21, 0xe4, 0x7e, 0x10, 0x22, 0x8b, - 0x45, 0x16, 0x60, 0x4b, 0x89, 0x40, 0x7c, 0xac, 0x52, 0x02, 0x69, 0x94, 0x60, 0x57, 0x43, 0x2a, - 0x24, 0x36, 0x95, 0x1d, 0x4f, 0x8d, 0xa5, 0xd8, 0x1e, 0xd9, 0x8e, 0xaa, 0x5e, 0x80, 0x35, 0x87, - 0xe1, 0x10, 0x5d, 0x72, 0x02, 0x84, 0xc2, 0x31, 0xd8, 0xa0, 0xd8, 0x4e, 0x32, 0x93, 0x04, 0x44, - 0xbb, 0x1b, 0x8f, 0xdf, 0x7b, 0xf3, 0x9f, 0xf7, 0xb3, 0x0c, 0xa7, 0xae, 0x97, 0x7c, 0x9e, 0xd8, - 0xda, 0x28, 0xf4, 0x75, 0xbf, 0xe5, 0xd8, 0xba, 0xdf, 0xd2, 0xe3, 0x68, 0xa4, 0xfb, 0x34, 0x89, - 0xbc, 0x51, 0xac, 0xbb, 0x34, 0xa0, 0x91, 0x95, 0x50, 0x47, 0x67, 0x51, 0x98, 0x84, 0x3a, 0xf3, - 0x18, 0x1d, 0x7b, 0x01, 0x65, 0xf6, 0x62, 0xa9, 0xa5, 0x6f, 0x30, 0x2c, 0x5f, 0x1d, 0x3e, 0xe5, - 0x52, 0xdd, 0xd0, 0x0d, 0x33, 0xb3, 0x3d, 0xb9, 0x4c, 0x9f, 0xb2, 0xa4, 0xd9, 0x2a, 0xb3, 0x1e, - 0x1a, 0xb7, 0x1c, 0xc2, 0x72, 0xdd, 0x88, 0xba, 0x56, 0xe2, 0x85, 0x01, 0xb3, 0xf9, 0xa7, 0x3c, - 0x6f, 0x78, 0xcb, 0xbc, 0x24, 0xb2, 0x82, 0xf8, 0x32, 0x8c, 0xfc, 0x79, 0xa4, 0xb8, 0x91, 0xa5, - 0xaa, 0x6f, 0x60, 0xb7, 0xbd, 0x3c, 0xca, 0x64, 0xb8, 0x09, 0x85, 0xe4, 0x9a, 0x51, 0x05, 0xd5, - 0x51, 0xa3, 0xd2, 0xac, 0x69, 0xc2, 0x58, 0x1a, 0xa7, 0x1d, 0x5e, 0x33, 0x4a, 0x52, 0xad, 0x3a, - 0x80, 0xea, 0x50, 0x08, 0x37, 0x19, 0x7e, 0x21, 0xe4, 0x3c, 0xd6, 0x56, 0xc7, 0xd1, 0x44, 0x07, - 0x97, 0xf6, 0x05, 0x41, 0x89, 0x84, 0xe3, 0xf1, 0x84, 0x99, 0x0c, 0x1f, 0x40, 0x29, 0xa0, 0x57, - 0x17, 0x81, 0xe5, 0x67, 0x51, 0x3b, 0x64, 0x3b, 0xa0, 0x57, 0x86, 0xe5, 0x53, 0x8c, 0xa1, 0x90, - 0x58, 0x6e, 0xac, 0x48, 0x75, 0xb9, 0xb1, 0x43, 0xd2, 0x35, 0xee, 0xc3, 0x3e, 0x37, 0xf0, 0xc5, - 0x2c, 0x2f, 0x56, 0xe4, 0xba, 0xfc, 0x1f, 0x57, 0xa9, 0x5a, 0xe2, 0x46, 0xac, 0x7e, 0x93, 0x00, - 0xce, 0x72, 0xfe, 0x26, 0xc3, 0xba, 0x70, 0xa3, 0x23, 0x6d, 0xf9, 0x69, 0x68, 0x4b, 0x95, 0xb6, - 0xbc, 0x08, 0x7e, 0x0d, 0x65, 0x2e, 0x53, 0x91, 0xea, 0xa8, 0x51, 0x6e, 0x1e, 0xf0, 0x3e, 0xa1, - 0x7a, 0xc2, 0xab, 0x71, 0x07, 0x2a, 0x62, 0x65, 0x8a, 0x9c, 0xfa, 0x8f, 0x79, 0xff, 0x6a, 0xeb, - 0x64, 0xc5, 0x83, 0x9f, 0x40, 0x31, 0x4a, 0xab, 0x54, 0x0a, 0xa9, 0xfb, 0x3e, 0xef, 0x9e, 0x97, - 0x4c, 0x72, 0x8d, 0xda, 0x81, 0xc2, 0x6c, 0x7c, 0x5c, 0x86, 0xed, 0x73, 0xa3, 0x6f, 0x98, 0x1f, - 0x8d, 0xea, 0x16, 0xde, 0x83, 0x72, 0xbb, 0xdb, 0x25, 0x6f, 0xbb, 0xed, 0x61, 0xcf, 0x34, 0xaa, - 0x08, 0x63, 0xa8, 0x0c, 0x49, 0xdb, 0xf8, 0xf0, 0xce, 0x24, 0xef, 0xb3, 0x3d, 0x09, 0x03, 0x14, - 0x89, 0x39, 0x18, 0x9c, 0x9f, 0x55, 0x65, 0xf5, 0x15, 0x94, 0xe6, 0x7d, 0x60, 0x0d, 0xe4, 0x90, - 0xc5, 0x0a, 0xaa, 0xcb, 0x8d, 0x72, 0xf3, 0xc1, 0xe6, 0xca, 0x4e, 0x0a, 0x37, 0x3f, 0x1e, 0x6d, - 0x91, 0x99, 0x50, 0x1d, 0xc3, 0x5e, 0x9b, 0xb1, 0xb1, 0x47, 0x9d, 0xc5, 0x17, 0x50, 0x01, 0xc9, - 0x73, 0xd2, 0xd2, 0xef, 0x11, 0xc9, 0x73, 0x70, 0x0f, 0x2a, 0x3c, 0x62, 0xcf, 0xc9, 0x8b, 0x3d, - 0xfe, 0x3b, 0xdf, 0x5e, 0x27, 0x3f, 0x63, 0x97, 0x93, 0xf4, 0x1c, 0xf5, 0x37, 0x82, 0xfd, 0xfc, - 0x38, 0x8e, 0xf3, 0x73, 0x81, 0xb3, 0x2a, 0xf0, 0x5a, 0x15, 0xf3, 0xb8, 0xd7, 0x89, 0x49, 0x77, - 0x20, 0xd6, 0x5a, 0x10, 0xcb, 0x78, 0x1f, 0x6d, 0x38, 0x7f, 0x0d, 0x5c, 0x6b, 0x13, 0xb8, 0x75, - 0x4e, 0x88, 0xe3, 0x24, 0xa9, 0xa7, 0x8b, 0xae, 0x17, 0xb8, 0x9e, 0xf1, 0xb8, 0x1e, 0xfe, 0xf3, - 0xe6, 0x1c, 0xb5, 0x93, 0xfe, 0xcd, 0xb4, 0x86, 0xbe, 0x4f, 0x6b, 0xe8, 0xe7, 0xb4, 0x86, 0xbe, - 0xfe, 0xaa, 0x6d, 0x7d, 0x7a, 0x79, 0xe7, 0x3f, 0xb0, 0x5d, 0x4c, 0x77, 0x5a, 0x7f, 0x02, 0x00, - 0x00, 0xff, 0xff, 0x26, 0x29, 0x00, 0xcc, 0xc5, 0x05, 0x00, 0x00, + // 626 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x4d, 0x4f, 0xdb, 0x4c, + 0x10, 0xc7, 0xb3, 0x76, 0x04, 0x61, 0x02, 0xc1, 0xac, 0x1e, 0x3d, 0x32, 0x2f, 0x4d, 0x23, 0x8b, + 0x43, 0x0e, 0xc5, 0x96, 0x12, 0xb5, 0xea, 0xcb, 0x29, 0x10, 0x1a, 0x22, 0x52, 0x1b, 0x6d, 0x13, + 0xf5, 0xe5, 0x82, 0x6c, 0xbc, 0xb8, 0x96, 0x62, 0x7b, 0x65, 0x1b, 0x21, 0xbe, 0x45, 0x3f, 0x4c, + 0x3f, 0x04, 0xc7, 0xde, 0x2b, 0x55, 0x15, 0xfd, 0x18, 0xbd, 0x54, 0xb1, 0x4d, 0xb2, 0x1b, 0xd2, + 0xaa, 0x70, 0xdb, 0x5d, 0xcf, 0xfc, 0x67, 0xe6, 0xff, 0x1b, 0x19, 0x8e, 0x3c, 0x3f, 0xfd, 0x74, + 0xe1, 0xe8, 0x67, 0x51, 0x60, 0x04, 0x6d, 0xd7, 0x31, 0x82, 0xb6, 0x91, 0xc4, 0x67, 0x46, 0x40, + 0xd3, 0xd8, 0x3f, 0x4b, 0x0c, 0x8f, 0x86, 0x34, 0xb6, 0x53, 0xea, 0x1a, 0x2c, 0x8e, 0xd2, 0xc8, + 0x60, 0x3e, 0xa3, 0x63, 0x3f, 0xa4, 0xcc, 0x99, 0x1e, 0xf5, 0xec, 0x0b, 0x86, 0xd9, 0xa7, 0xad, + 0x3d, 0x4e, 0xd5, 0x8b, 0xbc, 0x28, 0x4f, 0x76, 0x2e, 0xce, 0xb3, 0x5b, 0xae, 0x34, 0x39, 0xe5, + 0xa9, 0x5b, 0xe6, 0x3d, 0x9b, 0xb0, 0x3d, 0x2f, 0xa6, 0x9e, 0x9d, 0xfa, 0x51, 0xc8, 0x1c, 0xfe, + 0x56, 0xe8, 0x0d, 0xef, 0xa9, 0x97, 0xc6, 0x76, 0x98, 0x9c, 0x47, 0x71, 0x70, 0x2b, 0x29, 0x3e, + 0xe4, 0xaa, 0xda, 0x01, 0xac, 0x75, 0x66, 0xa5, 0x2c, 0x86, 0x5b, 0x50, 0x4e, 0xaf, 0x18, 0x55, + 0x51, 0x03, 0x35, 0x6b, 0xad, 0xba, 0x2e, 0xb4, 0xa5, 0x73, 0xb1, 0xc3, 0x2b, 0x46, 0x49, 0x16, + 0xab, 0x0d, 0x40, 0x19, 0x0a, 0xe2, 0x16, 0xc3, 0xcf, 0x05, 0x9d, 0x5d, 0x7d, 0xbe, 0x1d, 0x5d, + 0xcc, 0xe0, 0xd4, 0xbe, 0x21, 0xa8, 0x90, 0x68, 0x3c, 0xbe, 0x60, 0x16, 0xc3, 0x9b, 0x50, 0x09, + 0xe9, 0xe5, 0x69, 0x68, 0x07, 0xb9, 0xd4, 0x0a, 0x59, 0x0e, 0xe9, 0xa5, 0x69, 0x07, 0x14, 0x63, + 0x28, 0xa7, 0xb6, 0x97, 0xa8, 0x52, 0x43, 0x6e, 0xae, 0x90, 0xec, 0x8c, 0x8f, 0x61, 0x83, 0x6b, + 0xf8, 0x74, 0xa2, 0x97, 0xa8, 0x72, 0x43, 0xfe, 0x87, 0x51, 0x14, 0x5b, 0x7c, 0x48, 0xf0, 0x5e, + 0x31, 0x42, 0x39, 0x1b, 0x61, 0x53, 0x9f, 0xed, 0x82, 0x7e, 0xdb, 0x9f, 0xce, 0xf5, 0xbd, 0x0b, + 0xe5, 0xc9, 0x0d, 0xaf, 0x42, 0xa5, 0x47, 0xac, 0xd1, 0xc9, 0xe9, 0xfe, 0x07, 0xa5, 0x84, 0x6b, + 0x00, 0x87, 0xef, 0x0f, 0x06, 0xa3, 0xee, 0xe1, 0xe4, 0x8e, 0xb4, 0x2f, 0x12, 0xc0, 0x49, 0x21, + 0x64, 0x31, 0x6c, 0x08, 0x36, 0x6d, 0xf3, 0x35, 0x66, 0x51, 0x5c, 0x15, 0xfc, 0x0a, 0xaa, 0x5c, + 0xa3, 0xaa, 0xd4, 0x40, 0xcd, 0xaa, 0xd8, 0x9b, 0xc0, 0x93, 0xf0, 0xd1, 0xb8, 0x0b, 0x35, 0x91, + 0x83, 0x2a, 0x67, 0xf9, 0x3b, 0x7c, 0xfe, 0x3c, 0x4a, 0x32, 0x97, 0x83, 0x9f, 0xc0, 0x52, 0x9c, + 0xcd, 0x9f, 0x39, 0x53, 0x6d, 0xfd, 0xb7, 0xc8, 0x19, 0x52, 0xc4, 0x68, 0xdd, 0xc2, 0x96, 0x2a, + 0x2c, 0x8f, 0xcc, 0x63, 0xd3, 0x7a, 0x67, 0x2a, 0x25, 0xbc, 0x0e, 0xd5, 0x4e, 0xaf, 0x47, 0x0e, + 0x7b, 0x9d, 0x61, 0xdf, 0x32, 0x15, 0x84, 0x31, 0xd4, 0x86, 0xa4, 0x63, 0xbe, 0x7d, 0x6d, 0x91, + 0x37, 0xf9, 0x9b, 0x84, 0x01, 0x96, 0x88, 0x35, 0x18, 0x8c, 0x4e, 0x14, 0x59, 0x7b, 0x09, 0x95, + 0x5b, 0x3f, 0xb0, 0x0e, 0x72, 0xc4, 0x12, 0x15, 0x35, 0xe4, 0x66, 0xb5, 0xf5, 0xff, 0x62, 0xcb, + 0xf6, 0xcb, 0xd7, 0xdf, 0x1f, 0x97, 0xc8, 0x24, 0x50, 0x1b, 0xc3, 0x7a, 0x87, 0xb1, 0xb1, 0x4f, + 0xdd, 0xe9, 0x5a, 0xd5, 0x40, 0xf2, 0xdd, 0xcc, 0xf4, 0x55, 0x22, 0xf9, 0x2e, 0xee, 0x43, 0x8d, + 0xdf, 0x1b, 0xdf, 0x2d, 0x8c, 0xdd, 0xf9, 0xf3, 0xd2, 0xf4, 0xbb, 0x45, 0x8d, 0x35, 0x2e, 0xa4, + 0xef, 0x6a, 0xbf, 0x10, 0x6c, 0x14, 0xe5, 0x38, 0xce, 0xcf, 0x04, 0xce, 0x9a, 0xc0, 0x6b, 0x3e, + 0x98, 0xc7, 0x7d, 0x97, 0x98, 0xf4, 0x00, 0x62, 0xed, 0x29, 0xb1, 0x9c, 0xf7, 0xf6, 0x82, 0xfa, + 0x77, 0xc0, 0xb5, 0x17, 0x81, 0xbb, 0xcb, 0x09, 0x71, 0x9c, 0x24, 0xed, 0x68, 0xea, 0xf5, 0x14, + 0xd7, 0x53, 0x1e, 0xd7, 0xa3, 0xbf, 0x4e, 0xce, 0x51, 0xdb, 0x3f, 0xbe, 0xbe, 0xa9, 0xa3, 0xaf, + 0x37, 0x75, 0xf4, 0xe3, 0xa6, 0x8e, 0x3e, 0xff, 0xac, 0x97, 0x3e, 0xbe, 0x78, 0xf0, 0x6f, 0xdd, + 0x59, 0xca, 0x5e, 0xda, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x12, 0x00, 0x0a, 0x5b, 0x1a, 0x06, + 0x00, 0x00, } diff --git a/src/metrics/generated/proto/pipelinepb/pipeline.proto b/src/metrics/generated/proto/pipelinepb/pipeline.proto index dcdcc45f29..7383342665 100644 --- a/src/metrics/generated/proto/pipelinepb/pipeline.proto +++ b/src/metrics/generated/proto/pipelinepb/pipeline.proto @@ -37,9 +37,14 @@ message TransformationOp { } message RollupOp { + enum Type { + GROUP_BY = 0; + EXCLUDE_BY = 1; + } string new_name = 1; repeated string tags = 2; repeated aggregationpb.AggregationType aggregation_types = 3; + Type type = 4; } message PipelineOp { diff --git a/src/metrics/metadata/metadata.go b/src/metrics/metadata/metadata.go index b69da3daa4..94aa0275be 100644 --- a/src/metrics/metadata/metadata.go +++ b/src/metrics/metadata/metadata.go @@ -121,7 +121,12 @@ func (m PipelineMetadata) IsDropPolicyApplied() bool { return m.AggregationID.IsDefault() && m.StoragePolicies.IsDefault() && m.Pipeline.IsEmpty() && - !m.DropPolicy.IsDefault() + m.IsDropPolicySet() +} + +// IsDropPolicySet returns whether a drop policy is set. +func (m PipelineMetadata) IsDropPolicySet() bool { + return !m.DropPolicy.IsDefault() } // Clone clones the pipeline metadata. @@ -135,9 +140,7 @@ func (m PipelineMetadata) Clone() PipelineMetadata { // ToProto converts the pipeline metadata to a protobuf message in place. func (m PipelineMetadata) ToProto(pb *metricpb.PipelineMetadata) error { - if err := m.AggregationID.ToProto(&pb.AggregationId); err != nil { - return err - } + m.AggregationID.ToProto(&pb.AggregationId) if err := m.Pipeline.ToProto(&pb.Pipeline); err != nil { return err } @@ -158,9 +161,7 @@ func (m PipelineMetadata) ToProto(pb *metricpb.PipelineMetadata) error { // FromProto converts the protobuf message to a pipeline metadata in place. func (m *PipelineMetadata) FromProto(pb metricpb.PipelineMetadata) error { - if err := m.AggregationID.FromProto(pb.AggregationId); err != nil { - return err - } + m.AggregationID.FromProto(pb.AggregationId) if err := m.Pipeline.FromProto(pb.Pipeline); err != nil { return err } @@ -204,6 +205,17 @@ func (metadatas PipelineMetadatas) Clone() PipelineMetadatas { return cloned } +// IsDropPolicySet returns whether any drop policies are set (but +// does not discriminate if they have been applied or not). +func (metadatas PipelineMetadatas) IsDropPolicySet() bool { + for i := range metadatas { + if metadatas[i].IsDropPolicySet() { + return true + } + } + return false +} + // ApplyOrRemoveDropPoliciesResult is the result of applying or removing // the drop policies for pipelines. type ApplyOrRemoveDropPoliciesResult uint @@ -301,6 +313,17 @@ func (m Metadata) IsDropPolicyApplied() bool { return len(m.Pipelines) == 1 && m.Pipelines[0].IsDropPolicyApplied() } +// IsDropPolicySet returns whether any drop policies are set (but +// does not discriminate if they have been applied or not). +func (m Metadata) IsDropPolicySet() bool { + for i := range m.Pipelines { + if m.Pipelines[i].IsDropPolicySet() { + return true + } + } + return false +} + // Equal returns true if two metadatas are considered equal. func (m Metadata) Equal(other Metadata) bool { return m.Pipelines.Equal(other.Pipelines) @@ -358,9 +381,7 @@ type ForwardMetadata struct { // ToProto converts the forward metadata to a protobuf message in place. func (m ForwardMetadata) ToProto(pb *metricpb.ForwardMetadata) error { - if err := m.AggregationID.ToProto(&pb.AggregationId); err != nil { - return err - } + m.AggregationID.ToProto(&pb.AggregationId) if err := m.StoragePolicy.ToProto(&pb.StoragePolicy); err != nil { return err } @@ -374,9 +395,7 @@ func (m ForwardMetadata) ToProto(pb *metricpb.ForwardMetadata) error { // FromProto converts the protobuf message to a forward metadata in place. func (m *ForwardMetadata) FromProto(pb metricpb.ForwardMetadata) error { - if err := m.AggregationID.FromProto(pb.AggregationId); err != nil { - return err - } + m.AggregationID.FromProto(pb.AggregationId) if err := m.StoragePolicy.FromProto(pb.StoragePolicy); err != nil { return err } @@ -417,6 +436,11 @@ func (sm StagedMetadata) IsDropPolicyApplied() bool { return !sm.Tombstoned && sm.Metadata.IsDropPolicyApplied() } +// IsDropPolicySet returns whether a drop policy is set. +func (sm StagedMetadata) IsDropPolicySet() bool { + return sm.Pipelines.IsDropPolicySet() +} + // ToProto converts the staged metadata to a protobuf message in place. func (sm StagedMetadata) ToProto(pb *metricpb.StagedMetadata) error { if err := sm.Metadata.ToProto(&pb.Metadata); err != nil { @@ -470,6 +494,11 @@ func (sms StagedMetadatas) IsDropPolicyApplied() bool { return len(sms) == 1 && sms[0].IsDropPolicyApplied() } +// IsDropPolicySet returns if the active staged metadata has a drop policy set. +func (sms StagedMetadatas) IsDropPolicySet() bool { + return len(sms) > 0 && sms[len(sms)-1].IsDropPolicySet() +} + // ToProto converts the staged metadatas to a protobuf message in place. func (sms StagedMetadatas) ToProto(pb *metricpb.StagedMetadatas) error { numMetadatas := len(sms) @@ -487,6 +516,7 @@ func (sms StagedMetadatas) ToProto(pb *metricpb.StagedMetadatas) error { } // FromProto converts the protobuf message to a staged metadatas in place. +// This is an optimized method that merges some nested steps. func (sms *StagedMetadatas) FromProto(pb metricpb.StagedMetadatas) error { numMetadatas := len(pb.Metadatas) if cap(*sms) >= numMetadatas { @@ -494,6 +524,78 @@ func (sms *StagedMetadatas) FromProto(pb metricpb.StagedMetadatas) error { } else { *sms = make([]StagedMetadata, numMetadatas) } + + for i := 0; i < numMetadatas; i++ { + metadata := &(*sms)[i] + metadataPb := &pb.Metadatas[i] + numPipelines := len(metadataPb.Metadata.Pipelines) + + metadata.CutoverNanos = metadataPb.CutoverNanos + metadata.Tombstoned = metadataPb.Tombstoned + + if cap(metadata.Pipelines) >= numPipelines { + metadata.Pipelines = metadata.Pipelines[:numPipelines] + } else { + metadata.Pipelines = make(PipelineMetadatas, numPipelines) + } + + for j := 0; j < numPipelines; j++ { + var ( + pipelinePb = &metadataPb.Metadata.Pipelines[j] + pipeline = &metadata.Pipelines[j] + numStoragePolicies = len(pipelinePb.StoragePolicies) + numOps = len(pipelinePb.Pipeline.Ops) + err error + ) + + pipeline.AggregationID[0] = pipelinePb.AggregationId.Id + pipeline.DropPolicy = policy.DropPolicy(pipelinePb.DropPolicy) + + if len(pipeline.Tags) > 0 { + pipeline.Tags = pipeline.Tags[:0] + } + if len(pipeline.GraphitePrefix) > 0 { + pipeline.GraphitePrefix = pipeline.GraphitePrefix[:0] + } + + if cap(pipeline.StoragePolicies) >= numStoragePolicies { + pipeline.StoragePolicies = pipeline.StoragePolicies[:numStoragePolicies] + } else { + pipeline.StoragePolicies = make([]policy.StoragePolicy, numStoragePolicies) + } + + if cap(pipeline.Pipeline.Operations) >= numOps { + pipeline.Pipeline.Operations = pipeline.Pipeline.Operations[:numOps] + } else { + pipeline.Pipeline.Operations = make([]applied.OpUnion, numOps) + } + + if len(pipelinePb.Pipeline.Ops) > 0 { + err = applied.OperationsFromProto(pipelinePb.Pipeline.Ops, pipeline.Pipeline.Operations) + if err != nil { + return err + } + } + if len(pipelinePb.StoragePolicies) > 0 { + err = policy.StoragePoliciesFromProto(pipelinePb.StoragePolicies, pipeline.StoragePolicies) + if err != nil { + return err + } + } + } + } + + return nil +} + +// fromProto is a non-optimized in place protobuf conversion method, used as a reference for tests. +func (sms *StagedMetadatas) fromProto(pb metricpb.StagedMetadatas) error { + numMetadatas := len(pb.Metadatas) + if cap(*sms) >= numMetadatas { + *sms = (*sms)[:numMetadatas] + } else { + *sms = make([]StagedMetadata, numMetadatas) + } for i := 0; i < numMetadatas; i++ { if err := (*sms)[i].FromProto(pb.Metadatas[i]); err != nil { return err @@ -519,9 +621,7 @@ type TimedMetadata struct { // ToProto converts the timed metadata to a protobuf message in place. func (m TimedMetadata) ToProto(pb *metricpb.TimedMetadata) error { - if err := m.AggregationID.ToProto(&pb.AggregationId); err != nil { - return err - } + m.AggregationID.ToProto(&pb.AggregationId) if err := m.StoragePolicy.ToProto(&pb.StoragePolicy); err != nil { return err } @@ -530,9 +630,7 @@ func (m TimedMetadata) ToProto(pb *metricpb.TimedMetadata) error { // FromProto converts the protobuf message to a timed metadata in place. func (m *TimedMetadata) FromProto(pb metricpb.TimedMetadata) error { - if err := m.AggregationID.FromProto(pb.AggregationId); err != nil { - return err - } + m.AggregationID.FromProto(pb.AggregationId) if err := m.StoragePolicy.FromProto(pb.StoragePolicy); err != nil { return err } diff --git a/src/metrics/metadata/metadata_benchmark_test.go b/src/metrics/metadata/metadata_benchmark_test.go index ff505e28ae..49099e55b7 100644 --- a/src/metrics/metadata/metadata_benchmark_test.go +++ b/src/metrics/metadata/metadata_benchmark_test.go @@ -25,6 +25,7 @@ import ( "testing" "github.com/m3db/m3/src/metrics/aggregation" + "github.com/m3db/m3/src/metrics/generated/proto/metricpb" "github.com/m3db/m3/src/metrics/policy" ) @@ -54,3 +55,85 @@ func BenchmarkMetadata_IsDefault(b *testing.B) { } runtime.KeepAlive(m) } + +func BenchmarkMetadata_FromProto(b *testing.B) { + var ( + testAllPayload metricpb.StagedMetadatas + m StagedMetadatas + ) + + testAllPayload.Metadatas = append(testAllPayload.Metadatas, + testSmallStagedMetadatasProto.Metadatas...) + testAllPayload.Metadatas = append(testAllPayload.Metadatas, + testLargeStagedMetadatasProto.Metadatas...) + testAllPayload.Metadatas = append(testAllPayload.Metadatas, + testSmallStagedMetadatasWithLargeStoragePoliciesProto.Metadatas...) + + b.Run("large metadatas", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.FromProto(testLargeStagedMetadatasProto); err != nil { + b.Fail() + } + } + }) + + b.Run("small metadatas", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.FromProto(testSmallStagedMetadatasProto); err != nil { + b.Fail() + } + } + }) + + b.Run("storage policies", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.FromProto( + testSmallStagedMetadatasWithLargeStoragePoliciesProto, + ); err != nil { + b.Fail() + } + } + }) + + b.Run("all", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.FromProto(testAllPayload); err != nil { + b.Fail() + } + } + }) + + b.Run("reference, large metadatas", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.fromProto(testLargeStagedMetadatasProto); err != nil { + b.Fail() + } + } + }) + + b.Run("reference, small metadatas", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.fromProto(testSmallStagedMetadatasProto); err != nil { + b.Fail() + } + } + }) + + b.Run("reference, storage policies", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.fromProto(testSmallStagedMetadatasWithLargeStoragePoliciesProto); err != nil { + b.Fail() + } + } + }) + + b.Run("reference, all", func(b *testing.B) { + for i := 0; i < b.N; i++ { + if err := m.fromProto(testAllPayload); err != nil { + b.Fail() + } + } + }) + + runtime.KeepAlive(m) +} diff --git a/src/metrics/metadata/metadata_test.go b/src/metrics/metadata/metadata_test.go index 7376747a07..475c952f07 100644 --- a/src/metrics/metadata/metadata_test.go +++ b/src/metrics/metadata/metadata_test.go @@ -413,6 +413,68 @@ var ( }, }, } + testSmallStagedMetadatasWithLargeStoragePoliciesProto = metricpb.StagedMetadatas{ + Metadatas: []metricpb.StagedMetadata{ + { + CutoverNanos: 4567, + Tombstoned: true, + Metadata: metricpb.Metadata{ + Pipelines: []metricpb.PipelineMetadata{ + { + AggregationId: aggregationpb.AggregationID{Id: aggregation.MustCompressTypes(aggregation.Sum)[0]}, + StoragePolicies: []policypb.StoragePolicy{ + { + Resolution: policypb.Resolution{ + WindowSize: time.Second.Nanoseconds(), + Precision: time.Second.Nanoseconds(), + }, + Retention: policypb.Retention{ + Period: 10 * time.Second.Nanoseconds(), + }, + }, + { + Resolution: policypb.Resolution{ + WindowSize: 10 * time.Second.Nanoseconds(), + Precision: time.Second.Nanoseconds(), + }, + Retention: policypb.Retention{ + Period: time.Hour.Nanoseconds(), + }, + }, + { + Resolution: policypb.Resolution{ + WindowSize: 10 * time.Minute.Nanoseconds(), + Precision: time.Second.Nanoseconds(), + }, + Retention: policypb.Retention{ + Period: time.Minute.Nanoseconds(), + }, + }, + { + Resolution: policypb.Resolution{ + WindowSize: 10 * time.Minute.Nanoseconds(), + Precision: time.Second.Nanoseconds(), + }, + Retention: policypb.Retention{ + Period: time.Second.Nanoseconds(), + }, + }, + { + Resolution: policypb.Resolution{ + WindowSize: 10 * time.Hour.Nanoseconds(), + Precision: time.Second.Nanoseconds(), + }, + Retention: policypb.Retention{ + Period: time.Second.Nanoseconds(), + }, + }, + }, + }, + }, + }, + }, + }, + } testLargeStagedMetadatasProto = metricpb.StagedMetadatas{ Metadatas: []metricpb.StagedMetadata{ { @@ -1013,10 +1075,13 @@ func TestStagedMetadatasFromProto(t *testing.T) { } for _, input := range inputs { - var res StagedMetadatas + var resOpt, resReference StagedMetadatas for i, pb := range input.sequence { - require.NoError(t, res.FromProto(pb)) - require.Equal(t, input.expected[i], res) + require.NoError(t, resReference.fromProto(pb)) + require.NoError(t, resOpt.FromProto(pb)) + require.Equal(t, input.expected[i], resOpt) + require.Equal(t, input.expected[i], resReference) + require.Equal(t, resOpt, resReference) } } } diff --git a/src/metrics/metric/aggregated/types_test.go b/src/metrics/metric/aggregated/types_test.go index 9d27d8de02..a0760ce752 100644 --- a/src/metrics/metric/aggregated/types_test.go +++ b/src/metrics/metric/aggregated/types_test.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/metrics/transformation" xtime "github.com/m3db/m3/src/x/time" + "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/require" ) @@ -365,13 +366,18 @@ func TestForwardedMetricWithMetadataFromProto(t *testing.T) { } var res ForwardedMetricWithMetadata + comparer := cmp.Comparer(spComparer) for _, input := range inputs { require.NoError(t, res.FromProto(&input.data)) expected := ForwardedMetricWithMetadata{ ForwardedMetric: input.expectedMetric, ForwardMetadata: input.expectedMetadata, } - require.Equal(t, expected, res) + + if !cmp.Equal(expected, res, comparer) { + t.Log(cmp.Diff(expected, res, comparer)) + t.Fail() + } } } @@ -416,6 +422,7 @@ func TestForwardedMetricWithMetadataRoundtrip(t *testing.T) { res ForwardedMetricWithMetadata pb metricpb.ForwardedMetricWithMetadata ) + comparer := cmp.Comparer(spComparer) for _, input := range inputs { data := ForwardedMetricWithMetadata{ ForwardedMetric: input.metric, @@ -423,6 +430,13 @@ func TestForwardedMetricWithMetadataRoundtrip(t *testing.T) { } require.NoError(t, data.ToProto(&pb)) require.NoError(t, res.FromProto(&pb)) - require.Equal(t, data, res) + if !cmp.Equal(data, res, comparer) { + t.Log(cmp.Diff(data, res, comparer)) + t.Fail() + } } } + +func spComparer(x, y policy.StoragePolicy) bool { + return x.Equivalent(y) +} diff --git a/src/metrics/metric/types.go b/src/metrics/metric/types.go index c361961bf9..4377505882 100644 --- a/src/metrics/metric/types.go +++ b/src/metrics/metric/types.go @@ -50,6 +50,15 @@ var ( M3GaugeValue = []byte("gauge") M3TimerValue = []byte("timer") + PromUnknownValue = []byte("unknown") + PromCounterValue = []byte("counter") + PromGaugeValue = []byte("gauge") + PromHistogramValue = []byte("histogram") + PromGaugeHistogramValue = []byte("gauge_histogram") + PromSummaryValue = []byte("summary") + PromInfoValue = []byte("info") + PromStateSetValue = []byte("state_set") + M3MetricsPrefix = []byte("__m3") M3MetricsPrefixString = string(M3MetricsPrefix) @@ -57,6 +66,7 @@ var ( M3MetricsGraphiteAggregation = []byte(M3MetricsPrefixString + "_graphite_aggregation__") M3MetricsGraphitePrefix = []byte(M3MetricsPrefixString + "_graphite_prefix__") M3MetricsDropTimestamp = []byte(M3MetricsPrefixString + "_drop_timestamp__") + M3PromTypeTag = []byte(M3MetricsPrefixString + "_prom_type__") ) func (t Type) String() string { diff --git a/src/metrics/pipeline/applied/type.go b/src/metrics/pipeline/applied/type.go index fd320f4d82..fd5ae22f65 100644 --- a/src/metrics/pipeline/applied/type.go +++ b/src/metrics/pipeline/applied/type.go @@ -28,13 +28,17 @@ import ( "github.com/m3db/m3/src/metrics/aggregation" "github.com/m3db/m3/src/metrics/generated/proto/pipelinepb" "github.com/m3db/m3/src/metrics/pipeline" + "github.com/m3db/m3/src/metrics/transformation" ) var ( // DefaultPipeline is a default pipeline. DefaultPipeline Pipeline - errNilAppliedRollupOpProto = errors.New("nil applied rollup op proto message") + errNilAppliedRollupOpProto = errors.New("nil applied rollup op proto message") + errUnknownOpType = errors.New("unknown op type") + errOperationsLengthMismatch = errors.New("operations list length does not match proto") + errNilTransformationOpProto = errors.New("nil transformation op proto message") ) // RollupOp captures the rollup metadata after the operation is applied against a metric ID. @@ -45,7 +49,7 @@ type RollupOp struct { AggregationID aggregation.ID } -// Equal determines whether two rollup operations are equal. +// Equal determines whether two rollup Operations are equal. func (op RollupOp) Equal(other RollupOp) bool { return op.AggregationID == other.AggregationID && bytes.Equal(op.ID, other.ID) } @@ -63,9 +67,7 @@ func (op RollupOp) String() string { // ToProto converts the applied rollup op to a protobuf message in place. func (op RollupOp) ToProto(pb *pipelinepb.AppliedRollupOp) error { - if err := op.AggregationID.ToProto(&pb.AggregationId); err != nil { - return err - } + op.AggregationID.ToProto(&pb.AggregationId) pb.Id = op.ID return nil } @@ -75,9 +77,7 @@ func (op *RollupOp) FromProto(pb *pipelinepb.AppliedRollupOp) error { if pb == nil { return errNilAppliedRollupOpProto } - if err := op.AggregationID.FromProto(pb.AggregationId); err != nil { - return err - } + op.AggregationID.FromProto(pb.AggregationId) op.ID = pb.Id return nil } @@ -143,7 +143,7 @@ func (u OpUnion) ToProto(pb *pipelinepb.AppliedPipelineOp) error { pb.Rollup = &pipelinepb.AppliedRollupOp{} return u.Rollup.ToProto(pb.Rollup) default: - return fmt.Errorf("unknown op type: %v", u.Type) + return errUnknownOpType } } @@ -152,58 +152,62 @@ func (u *OpUnion) Reset() { *u = OpUnion{} } // FromProto converts the protobuf message to an applied pipeline op in place. func (u *OpUnion) FromProto(pb pipelinepb.AppliedPipelineOp) error { - u.Reset() switch pb.Type { case pipelinepb.AppliedPipelineOp_TRANSFORMATION: u.Type = pipeline.TransformationOpType + if u.Rollup.ID != nil { + u.Rollup.ID = u.Rollup.ID[:0] + } + u.Rollup.AggregationID[0] = aggregation.DefaultID[0] return u.Transformation.FromProto(pb.Transformation) case pipelinepb.AppliedPipelineOp_ROLLUP: u.Type = pipeline.RollupOpType + u.Transformation.Type = transformation.UnknownType return u.Rollup.FromProto(pb.Rollup) default: - return fmt.Errorf("unknown op type in proto: %v", pb.Type) + return errUnknownOpType } } // Pipeline is a pipeline of operations. type Pipeline struct { - // a list of pipeline operations. - operations []OpUnion + // a list of pipeline Operations. + Operations []OpUnion } // NewPipeline creates a new pipeline. func NewPipeline(ops []OpUnion) Pipeline { - return Pipeline{operations: ops} + return Pipeline{Operations: ops} } // Len returns the number of steps in a pipeline. -func (p Pipeline) Len() int { return len(p.operations) } +func (p Pipeline) Len() int { return len(p.Operations) } // IsEmpty determines whether a pipeline is empty. -func (p Pipeline) IsEmpty() bool { return len(p.operations) == 0 } +func (p Pipeline) IsEmpty() bool { return len(p.Operations) == 0 } // At returns the operation at a given step. -func (p Pipeline) At(i int) OpUnion { return p.operations[i] } +func (p Pipeline) At(i int) OpUnion { return p.Operations[i] } // Equal determines whether two pipelines are equal. func (p Pipeline) Equal(other Pipeline) bool { // keep in sync with OpUnion.Equal as go is terrible at inlining anything with a loop - if len(p.operations) != len(other.operations) { + if len(p.Operations) != len(other.Operations) { return false } - for i := 0; i < len(p.operations); i++ { - if p.operations[i].Type != other.operations[i].Type { + for i := 0; i < len(p.Operations); i++ { + if p.Operations[i].Type != other.Operations[i].Type { return false } //nolint:exhaustive - switch p.operations[i].Type { + switch p.Operations[i].Type { case pipeline.RollupOpType: - if !p.operations[i].Rollup.Equal(other.operations[i].Rollup) { + if !p.Operations[i].Rollup.Equal(other.Operations[i].Rollup) { return false } case pipeline.TransformationOpType: - if p.operations[i].Transformation.Type != other.operations[i].Transformation.Type { + if p.Operations[i].Transformation.Type != other.Operations[i].Transformation.Type { return false } } @@ -214,25 +218,25 @@ func (p Pipeline) Equal(other Pipeline) bool { // Clone clones the pipeline. func (p Pipeline) Clone() Pipeline { - clone := make([]OpUnion, len(p.operations)) - for i := range p.operations { - clone[i] = p.operations[i].Clone() + clone := make([]OpUnion, len(p.Operations)) + for i := range p.Operations { + clone[i] = p.Operations[i].Clone() } - return Pipeline{operations: clone} + return Pipeline{Operations: clone} } -// SubPipeline returns a sub-pipeline containing operations between step `startInclusive` +// SubPipeline returns a sub-pipeline containing Operations between step `startInclusive` // and step `endExclusive` of the current pipeline. func (p Pipeline) SubPipeline(startInclusive int, endExclusive int) Pipeline { - return Pipeline{operations: p.operations[startInclusive:endExclusive]} + return Pipeline{Operations: p.Operations[startInclusive:endExclusive]} } func (p Pipeline) String() string { var b bytes.Buffer b.WriteString("{operations: [") - for i, op := range p.operations { + for i, op := range p.Operations { b.WriteString(op.String()) - if i < len(p.operations)-1 { + if i < len(p.Operations)-1 { b.WriteString(", ") } } @@ -242,14 +246,14 @@ func (p Pipeline) String() string { // ToProto converts the applied pipeline to a protobuf message in place. func (p Pipeline) ToProto(pb *pipelinepb.AppliedPipeline) error { - numOps := len(p.operations) + numOps := len(p.Operations) if cap(pb.Ops) >= numOps { pb.Ops = pb.Ops[:numOps] } else { pb.Ops = make([]pipelinepb.AppliedPipelineOp, numOps) } for i := 0; i < numOps; i++ { - if err := p.operations[i].ToProto(&pb.Ops[i]); err != nil { + if err := p.Operations[i].ToProto(&pb.Ops[i]); err != nil { return err } } @@ -259,13 +263,13 @@ func (p Pipeline) ToProto(pb *pipelinepb.AppliedPipeline) error { // FromProto converts the protobuf message to an applied pipeline in place. func (p *Pipeline) FromProto(pb pipelinepb.AppliedPipeline) error { numOps := len(pb.Ops) - if cap(p.operations) >= numOps { - p.operations = p.operations[:numOps] + if cap(p.Operations) >= numOps { + p.Operations = p.Operations[:numOps] } else { - p.operations = make([]OpUnion, numOps) + p.Operations = make([]OpUnion, numOps) } for i := 0; i < numOps; i++ { - if err := p.operations[i].FromProto(pb.Ops[i]); err != nil { + if err := p.Operations[i].FromProto(pb.Ops[i]); err != nil { return err } } @@ -275,10 +279,45 @@ func (p *Pipeline) FromProto(pb pipelinepb.AppliedPipeline) error { // IsMappingRule returns whether this is a mapping rule, determined by // if any rollup pipelines are included. func (p Pipeline) IsMappingRule() bool { - for _, op := range p.operations { + for _, op := range p.Operations { if op.Rollup.ID != nil { return false } } return true } + +// OperationsFromProto converts a list of protobuf AppliedPipelineOps, used in optimized staged metadata methods. +func OperationsFromProto(pb []pipelinepb.AppliedPipelineOp, ops []OpUnion) error { + numOps := len(pb) + if numOps != len(ops) { + return errOperationsLengthMismatch + } + for i := 0; i < numOps; i++ { + u := &ops[i] + u.Type = pipeline.OpType(pb[i].Type + 1) + switch u.Type { + case pipeline.TransformationOpType: + if u.Rollup.ID != nil { + u.Rollup.ID = u.Rollup.ID[:0] + } + u.Rollup.AggregationID[0] = aggregation.DefaultID[0] + if pb[i].Transformation == nil { + return errNilTransformationOpProto + } + if err := u.Transformation.Type.FromProto(pb[i].Transformation.Type); err != nil { + return err + } + case pipeline.RollupOpType: + u.Transformation.Type = transformation.UnknownType + if pb == nil { + return errNilAppliedRollupOpProto + } + u.Rollup.AggregationID[0] = pb[i].Rollup.AggregationId.Id + u.Rollup.ID = pb[i].Rollup.Id + default: + return errUnknownOpType + } + } + return nil +} diff --git a/src/metrics/pipeline/applied/type_test.go b/src/metrics/pipeline/applied/type_test.go index edd8e26b39..75366fef16 100644 --- a/src/metrics/pipeline/applied/type_test.go +++ b/src/metrics/pipeline/applied/type_test.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3/src/metrics/pipeline" "github.com/m3db/m3/src/metrics/transformation" + "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/require" ) @@ -367,18 +368,18 @@ func TestPipelineEqual(t *testing.T) { require.Equal(t, input.expected, input.p2.Equal(input.p1)) // assert implementation is equal to OpUnion if input.expected { - for i, op := range input.p1.operations { - require.True(t, op.Equal(input.p2.operations[i])) + for i, op := range input.p1.Operations { + require.True(t, op.Equal(input.p2.Operations[i])) } - for i, op := range input.p2.operations { - require.True(t, op.Equal(input.p1.operations[i])) + for i, op := range input.p2.Operations { + require.True(t, op.Equal(input.p1.Operations[i])) } - } else if len(input.p1.operations) == len(input.p2.operations) { - for i, op := range input.p1.operations { - require.False(t, op.Equal(input.p2.operations[i])) + } else if len(input.p1.Operations) == len(input.p2.Operations) { + for i, op := range input.p1.Operations { + require.False(t, op.Equal(input.p2.Operations[i])) } - for i, op := range input.p2.operations { - require.False(t, op.Equal(input.p1.operations[i])) + for i, op := range input.p2.Operations { + require.False(t, op.Equal(input.p1.Operations[i])) } } }) @@ -392,7 +393,7 @@ func TestPipelineCloneEmptyPipeline(t *testing.T) { p2 := p1.Clone() require.True(t, p1.Equal(p2)) - p2.operations = append(p2.operations, OpUnion{ + p2.Operations = append(p2.Operations, OpUnion{ Type: pipeline.RollupOpType, Rollup: RollupOp{ ID: []byte("foo"), @@ -438,9 +439,9 @@ func TestPipelineCloneMultiLevelPipeline(t *testing.T) { require.True(t, p1.Equal(p3)) // Mutate the operations of a cloned pipeline. - p2.operations[0].Transformation.Type = transformation.PerSecond - p2.operations[1].Rollup.ID[0] = 'z' - p2.operations[3].Rollup.AggregationID = aggregation.MustCompressTypes(aggregation.Count) + p2.Operations[0].Transformation.Type = transformation.PerSecond + p2.Operations[1].Rollup.ID[0] = 'z' + p2.Operations[3].Rollup.AggregationID = aggregation.MustCompressTypes(aggregation.Count) // Verify the mutations do not affect the source pipeline or other clones. require.False(t, p1.Equal(p2)) @@ -616,7 +617,10 @@ func TestPipelineFromProto(t *testing.T) { var res Pipeline for i, pb := range input.sequence { require.NoError(t, res.FromProto(pb)) - require.Equal(t, input.expected[i], res) + if !cmp.Equal(input.expected[i], res) { + t.Log(cmp.Diff(input.expected[i], res)) + t.Fail() + } } } } @@ -646,7 +650,10 @@ func TestPipelineRoundTrip(t *testing.T) { for _, pipeline := range input { require.NoError(t, pipeline.ToProto(&pb)) require.NoError(t, res.FromProto(pb)) - require.Equal(t, pipeline, res) + if !cmp.Equal(pipeline, res) { + t.Log(cmp.Diff(pipeline, res)) + t.Fail() + } } } } diff --git a/src/metrics/pipeline/type.go b/src/metrics/pipeline/type.go index 9e25da72e8..602ea1c904 100644 --- a/src/metrics/pipeline/type.go +++ b/src/metrics/pipeline/type.go @@ -26,6 +26,7 @@ import ( "errors" "fmt" "sort" + "strings" "github.com/m3db/m3/src/metrics/aggregation" "github.com/m3db/m3/src/metrics/generated/proto/pipelinepb" @@ -41,6 +42,22 @@ var ( errNoOpInUnionMarshaler = errors.New("no operation in union JSON value") ) +const ( + templateMetricNameVar = ".MetricName" + templateOpen = "{{" + templateClose = "}}" + templateMetricNameExactMatch = templateOpen + " " + templateMetricNameVar + " " + templateClose +) + +var ( + templateMetricNameExactMatchBytes = []byte(templateMetricNameExactMatch) + templateAllowed = []string{templateMetricNameExactMatch} +) + +func maybeContainsTemplate(str string) bool { + return strings.Contains(str, templateOpen) || strings.Contains(str, templateClose) +} + // OpType defines the type of an operation. type OpType int @@ -166,14 +183,29 @@ func (op TransformationOp) MarshalText() (text []byte, err error) { return op.Type.MarshalText() } +// RollupType is the rollup type. +// Note: Must match the protobuf enum definition since this is a direct cast. +type RollupType int + +const ( + // GroupByRollupType defines the group by rollup op type (default). + GroupByRollupType RollupType = iota + // ExcludeByRollupType defines the exclude by rollup op type. + ExcludeByRollupType +) + // RollupOp is a rollup operation. type RollupOp struct { - // New metric name generated as a result of the rollup. - NewName []byte + // Type is the rollup type. + Type RollupType // Dimensions along which the rollup is performed. Tags [][]byte // Types of aggregation performed within each unique dimension combination. AggregationID aggregation.ID + + // New metric name generated as a result of the rollup. + newName []byte + newNameTemplated bool } // NewRollupOpFromProto creates a new rollup op from proto. @@ -183,27 +215,91 @@ func NewRollupOpFromProto(pb *pipelinepb.RollupOp) (RollupOp, error) { if pb == nil { return rollup, errNilRollupOpProto } + aggregationID, err := aggregation.NewIDFromProto(pb.AggregationTypes) if err != nil { return rollup, err } - tags := make([]string, len(pb.Tags)) - copy(tags, pb.Tags) + + return NewRollupOp(RollupType(pb.Type), pb.NewName, pb.Tags, aggregationID) +} + +// NewRollupOp creates a new rollup op. +func NewRollupOp( + rollupType RollupType, + rollupNewName string, + rollupTags []string, + rollupAggregationID aggregation.ID, +) (RollupOp, error) { + var rollup RollupOp + + tags := make([]string, len(rollupTags)) + copy(tags, rollupTags) sort.Strings(tags) + + var newNameTemplated bool + if maybeContainsTemplate(rollupNewName) { + // This metric might have a templated metric name. + newNameTemplated = true + + // Right now only support "{{ .MetricName }}" to be able to generate + // the resulting metric name without using a Go template and only + // a single instance of it. + if n := strings.Count(rollupNewName, templateMetricNameExactMatch); n > 1 { + return rollup, fmt.Errorf( + "rollup contained template variable metric name more than once: "+ + "input=%s, count_var_metric_name=%v", rollupNewName, n) + } + + // Replace and see if all template tags resolved. + replacedNewName := strings.Replace(rollupNewName, templateMetricNameExactMatch, "", 1) + + // Make sure fully replaced all instances of template usage, otherwise + // there are some other variables not supported or invalid use of + // template variable tags. + if maybeContainsTemplate(replacedNewName) { + return rollup, fmt.Errorf( + "rollup contained template tags but variables not resolved: "+ + "input=%s, allowed=%v", rollupNewName, templateAllowed) + } + } + return RollupOp{ - NewName: []byte(pb.NewName), - Tags: xbytes.ArraysFromStringArray(tags), - AggregationID: aggregationID, + Type: rollupType, + Tags: xbytes.ArraysFromStringArray(tags), + AggregationID: rollupAggregationID, + newName: []byte(rollupNewName), + newNameTemplated: newNameTemplated, }, nil } +// NewName returns the new rollup name based on an existing name if +// the new name uses a template, or otherwise the literal new name. +func (op RollupOp) NewName(currName []byte) []byte { + if !op.newNameTemplated { + // No templated name, just return the "literal" new name. + return op.newName + } + + out := make([]byte, 0, len(op.newName)+len(currName)) + idx := bytes.Index(op.newName, templateMetricNameExactMatchBytes) + if idx == -1 { + return op.newName + } + + out = append(out, op.newName[0:idx]...) + out = append(out, currName...) + out = append(out, op.newName[idx+len(templateMetricNameExactMatchBytes):]...) + return out +} + // SameTransform returns true if the two rollup operations have the same rollup transformation // (i.e., same new rollup metric name and same set of rollup tags). func (op RollupOp) SameTransform(other RollupOp) bool { if len(op.Tags) != len(other.Tags) { return false } - if !bytes.Equal(op.NewName, other.NewName) { + if !bytes.Equal(op.newName, other.newName) { return false } // Sort the tags and compare. @@ -224,17 +320,22 @@ func (op RollupOp) Equal(other RollupOp) bool { if !op.AggregationID.Equal(other.AggregationID) { return false } + if op.Type != other.Type { + return false + } return op.SameTransform(other) } // Clone clones the rollup operation. func (op RollupOp) Clone() RollupOp { - newName := make([]byte, len(op.NewName)) - copy(newName, op.NewName) + newName := make([]byte, len(op.newName)) + copy(newName, op.newName) return RollupOp{ - NewName: newName, - Tags: xbytes.ArrayCopy(op.Tags), - AggregationID: op.AggregationID, + Type: op.Type, + Tags: xbytes.ArrayCopy(op.Tags), + AggregationID: op.AggregationID, + newName: newName, + newNameTemplated: op.newNameTemplated, } } @@ -249,7 +350,8 @@ func (op RollupOp) Proto() (*pipelinepb.RollupOp, error) { return nil, err } return &pipelinepb.RollupOp{ - NewName: string(op.NewName), + Type: pipelinepb.RollupOp_Type(op.Type), + NewName: string(op.newName), Tags: xbytes.ArraysToStringArray(op.Tags), AggregationTypes: pbAggTypes, }, nil @@ -258,7 +360,7 @@ func (op RollupOp) Proto() (*pipelinepb.RollupOp, error) { func (op RollupOp) String() string { var b bytes.Buffer b.WriteString("{") - fmt.Fprintf(&b, "name: %s, ", op.NewName) + fmt.Fprintf(&b, "name: %s, ", op.newName) b.WriteString("tags: [") for i, t := range op.Tags { fmt.Fprintf(&b, "%s", t) @@ -280,21 +382,23 @@ func (op RollupOp) MarshalJSON() ([]byte, error) { // UnmarshalJSON unmarshals JSON-encoded data into a rollup operation. func (op *RollupOp) UnmarshalJSON(data []byte) error { var converted rollupMarshaler - if err := json.Unmarshal(data, &converted); err != nil { + err := json.Unmarshal(data, &converted) + if err != nil { return err } - *op = converted.RollupOp() - return nil + *op, err = converted.RollupOp() + return err } // UnmarshalYAML unmarshals YAML-encoded data into a rollup operation. func (op *RollupOp) UnmarshalYAML(unmarshal func(interface{}) error) error { var converted rollupMarshaler - if err := unmarshal(&converted); err != nil { + err := unmarshal(&converted) + if err != nil { return err } - *op = converted.RollupOp() - return nil + *op, err = converted.RollupOp() + return err } // MarshalYAML returns the YAML representation of this type. @@ -303,6 +407,7 @@ func (op RollupOp) MarshalYAML() (interface{}, error) { } type rollupMarshaler struct { + Type RollupType `json:"type" yaml:"type"` NewName string `json:"newName" yaml:"newName"` Tags []string `json:"tags" yaml:"tags"` AggregationID aggregation.ID `json:"aggregation,omitempty" yaml:"aggregation"` @@ -310,18 +415,14 @@ type rollupMarshaler struct { func newRollupMarshaler(op RollupOp) rollupMarshaler { return rollupMarshaler{ - NewName: string(op.NewName), + NewName: string(op.newName), Tags: xbytes.ArraysToStringArray(op.Tags), AggregationID: op.AggregationID, } } -func (m rollupMarshaler) RollupOp() RollupOp { - return RollupOp{ - NewName: []byte(m.NewName), - Tags: xbytes.ArraysFromStringArray(m.Tags), - AggregationID: m.AggregationID, - } +func (m rollupMarshaler) RollupOp() (RollupOp, error) { + return NewRollupOp(m.Type, m.NewName, m.Tags, m.AggregationID) } // OpUnion is a union of different types of operation. diff --git a/src/metrics/pipeline/type_test.go b/src/metrics/pipeline/type_test.go index a244304d4a..b5bc2d5fe7 100644 --- a/src/metrics/pipeline/type_test.go +++ b/src/metrics/pipeline/type_test.go @@ -165,7 +165,7 @@ func TestPipelineString(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("foo"), + newName: b("foo"), Tags: [][]byte{b("tag1"), b("tag2")}, AggregationID: aggregation.MustCompressTypes(aggregation.Sum), }, @@ -223,9 +223,38 @@ func TestTransformationOpRoundTrip(t *testing.T) { require.Equal(t, testTransformationOp, res) } +func TestRollupOpEqual(t *testing.T) { + inputs := []struct { + a1 RollupOp + a2 RollupOp + expected bool + }{ + { + a1: RollupOp{Type: GroupByRollupType}, + a2: RollupOp{Type: GroupByRollupType}, + expected: true, + }, + { + a1: RollupOp{Type: ExcludeByRollupType}, + a2: RollupOp{Type: ExcludeByRollupType}, + expected: true, + }, + { + a1: RollupOp{Type: GroupByRollupType}, + a2: RollupOp{Type: ExcludeByRollupType}, + expected: false, + }, + } + + for _, input := range inputs { + require.Equal(t, input.expected, input.a1.Equal(input.a2)) + require.Equal(t, input.expected, input.a2.Equal(input.a1)) + } +} + func TestRollupOpSameTransform(t *testing.T) { rollupOp := RollupOp{ - NewName: b("foo"), + newName: b("foo"), Tags: bs("bar1", "bar2"), } inputs := []struct { @@ -233,31 +262,31 @@ func TestRollupOpSameTransform(t *testing.T) { result bool }{ { - op: RollupOp{NewName: b("foo"), Tags: bs("bar1", "bar2")}, + op: RollupOp{newName: b("foo"), Tags: bs("bar1", "bar2")}, result: true, }, { - op: RollupOp{NewName: b("foo"), Tags: bs("bar2", "bar1")}, + op: RollupOp{newName: b("foo"), Tags: bs("bar2", "bar1")}, result: true, }, { - op: RollupOp{NewName: b("foo"), Tags: bs("bar1")}, + op: RollupOp{newName: b("foo"), Tags: bs("bar1")}, result: false, }, { - op: RollupOp{NewName: b("foo"), Tags: bs("bar1", "bar2", "bar3")}, + op: RollupOp{newName: b("foo"), Tags: bs("bar1", "bar2", "bar3")}, result: false, }, { - op: RollupOp{NewName: b("foo"), Tags: bs("bar1", "bar3")}, + op: RollupOp{newName: b("foo"), Tags: bs("bar1", "bar3")}, result: false, }, { - op: RollupOp{NewName: b("baz"), Tags: bs("bar1", "bar2")}, + op: RollupOp{newName: b("baz"), Tags: bs("bar1", "bar2")}, result: false, }, { - op: RollupOp{NewName: b("baz"), Tags: bs("bar2", "bar1")}, + op: RollupOp{newName: b("baz"), Tags: bs("bar2", "bar1")}, result: false, }, } @@ -289,23 +318,23 @@ func TestOpUnionMarshalJSON(t *testing.T) { op: OpUnion{ Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), }, }, - expected: `{"rollup":{"newName":"testRollup","tags":["tag1","tag2"],"aggregation":["Min","Max"]}}`, + expected: `{"rollup":{"type":0,"newName":"testRollup","tags":["tag1","tag2"],"aggregation":["Min","Max"]}}`, }, { op: OpUnion{ Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.DefaultID, }, }, - expected: `{"rollup":{"newName":"testRollup","tags":["tag1","tag2"],"aggregation":null}}`, + expected: `{"rollup":{"type":0,"newName":"testRollup","tags":["tag1","tag2"],"aggregation":null}}`, }, } @@ -335,7 +364,7 @@ func TestOpUnionMarshalRoundtrip(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), }, @@ -343,7 +372,7 @@ func TestOpUnionMarshalRoundtrip(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.DefaultID, }, @@ -366,7 +395,7 @@ func TestPipelineMarshalJSON(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), }, @@ -374,7 +403,7 @@ func TestPipelineMarshalJSON(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.DefaultID, }, @@ -385,8 +414,8 @@ func TestPipelineMarshalJSON(t *testing.T) { expected := `[{"aggregation":"Sum"},` + `{"transformation":"PerSecond"},` + - `{"rollup":{"newName":"testRollup","tags":["tag1","tag2"],"aggregation":["Min","Max"]}},` + - `{"rollup":{"newName":"testRollup","tags":["tag1","tag2"],"aggregation":null}}]` + `{"rollup":{"type":0,"newName":"testRollup","tags":["tag1","tag2"],"aggregation":["Min","Max"]}},` + + `{"rollup":{"type":0,"newName":"testRollup","tags":["tag1","tag2"],"aggregation":null}}]` require.Equal(t, expected, string(b)) } @@ -403,7 +432,7 @@ func TestPipelineMarshalRoundtrip(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), }, @@ -411,7 +440,7 @@ func TestPipelineMarshalRoundtrip(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.DefaultID, }, @@ -455,7 +484,7 @@ func TestPipelineUnmarshalYAML(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup"), + newName: b("testRollup"), Tags: bs("tag1", "tag2"), AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), }, @@ -463,7 +492,7 @@ func TestPipelineUnmarshalYAML(t *testing.T) { { Type: RollupOpType, Rollup: RollupOp{ - NewName: b("testRollup2"), + newName: b("testRollup2"), Tags: bs("tag3", "tag4"), AggregationID: aggregation.DefaultID, }, diff --git a/src/metrics/policy/storage_policy.go b/src/metrics/policy/storage_policy.go index 3d8e2905c0..0cdbf92390 100644 --- a/src/metrics/policy/storage_policy.go +++ b/src/metrics/policy/storage_policy.go @@ -39,8 +39,9 @@ var ( // EmptyStoragePolicy represents an empty storage policy. EmptyStoragePolicy StoragePolicy - errNilStoragePolicyProto = errors.New("nil storage policy proto") - errInvalidStoragePolicyString = errors.New("invalid storage policy string") + errNilStoragePolicyProto = errors.New("nil storage policy proto") + errInvalidStoragePolicyString = errors.New("invalid storage policy string") + errStoragePolicyLengthMismatch = errors.New("storage policy list length does not match proto") ) // StoragePolicy represents the resolution and retention period metric datapoints @@ -265,3 +266,22 @@ func (sp ByRetentionAscResolutionAsc) Less(i, j int) bool { } return sp[i].Resolution().Precision < sp[j].Resolution().Precision } + +// StoragePoliciesFromProto converts a list of protobuf storage policies to a storage policy in place. +func StoragePoliciesFromProto(src []policypb.StoragePolicy, dst []StoragePolicy) error { + if len(src) != len(dst) { + return errStoragePolicyLengthMismatch + } + for i := 0; i < len(src); i++ { + d := &dst[i] + if err := d.resolution.FromProto(src[i].Resolution); err != nil { + return err + } + + if err := d.retention.FromProto(src[i].Retention); err != nil { + return err + } + } + + return nil +} diff --git a/src/metrics/rules/active_ruleset.go b/src/metrics/rules/active_ruleset.go index 2f04841cae..6e624843f2 100644 --- a/src/metrics/rules/active_ruleset.go +++ b/src/metrics/rules/active_ruleset.go @@ -403,8 +403,7 @@ func (as *activeRuleSet) toRollupResults( var matched bool rollupID, matched = as.matchRollupTarget( sortedTagPairBytes, - firstOp.Rollup.NewName, - firstOp.Rollup.Tags, + firstOp.Rollup, tagPairs, matchRollupTargetOptions{generateRollupID: true}, ) @@ -457,43 +456,89 @@ func (as *activeRuleSet) toRollupResults( // tags, and nil otherwise. func (as *activeRuleSet) matchRollupTarget( sortedTagPairBytes []byte, - newName []byte, - rollupTags [][]byte, + rollupOp mpipeline.RollupOp, tagPairs []metricid.TagPair, // buffer for reuse to generate rollup ID across calls opts matchRollupTargetOptions, ) ([]byte, bool) { + if rollupOp.Type == mpipeline.ExcludeByRollupType && !opts.generateRollupID { + // Exclude by tag always matches, if not generating rollup ID + // then immediately return. + return nil, true + } + var ( + rollupTags = rollupOp.Tags sortedTagIter = as.tagsFilterOpts.SortedTagIteratorFn(sortedTagPairBytes) - hasMoreTags = sortedTagIter.Next() - currTagIdx = 0 + matchTagIdx = 0 + nameTagName = as.tagsFilterOpts.NameTagKey + nameTagValue []byte ) - for hasMoreTags && currTagIdx < len(rollupTags) { + + defer sortedTagIter.Close() + + // Iterate through each tag, looking to match it with corresponding filter tags on the rule + for hasMoreTags := sortedTagIter.Next(); hasMoreTags; hasMoreTags = sortedTagIter.Next() { tagName, tagVal := sortedTagIter.Current() - res := bytes.Compare(tagName, rollupTags[currTagIdx]) - if res == 0 { + // nolint:gosimple + isNameTag := bytes.Compare(tagName, nameTagName) == 0 + if isNameTag { + nameTagValue = tagVal + } + + switch rollupOp.Type { + case mpipeline.GroupByRollupType: + // If we've matched all tags, no need to process. + // We don't break out of the for loop, because we may still need to find the name tag. + if matchTagIdx >= len(rollupTags) { + continue + } + + res := bytes.Compare(tagName, rollupTags[matchTagIdx]) + if res == 0 { + // Include grouped by tag. + if opts.generateRollupID { + tagPairs = append(tagPairs, metricid.TagPair{Name: tagName, Value: tagVal}) + } + matchTagIdx++ + continue + } + + // If one of the target tags is not found in the ID, this is considered a non-match so return immediately. + if res > 0 { + return nil, false + } + case mpipeline.ExcludeByRollupType: + if isNameTag { + // Don't copy name tag since we'll add that using the new rollup ID fn. + continue + } + + if matchTagIdx >= len(rollupTags) { + // Have matched all the tags to exclude, just blindly copy. + if opts.generateRollupID { + tagPairs = append(tagPairs, metricid.TagPair{Name: tagName, Value: tagVal}) + } + continue + } + + res := bytes.Compare(tagName, rollupTags[matchTagIdx]) + if res == 0 { + // Skip excluded tag. + matchTagIdx++ + continue + } + if opts.generateRollupID { tagPairs = append(tagPairs, metricid.TagPair{Name: tagName, Value: tagVal}) } - currTagIdx++ - hasMoreTags = sortedTagIter.Next() - continue } - // If one of the target tags is not found in the ID, this is considered - // a non-match so bail immediately. - if res > 0 { - break - } - hasMoreTags = sortedTagIter.Next() } - sortedTagIter.Close() - // If not all the target tags are found, this is considered a no match. - if currTagIdx < len(rollupTags) { - return nil, false - } if !opts.generateRollupID { return nil, true } + + newName := rollupOp.NewName(nameTagValue) return as.newRollupIDFn(newName, tagPairs), true } @@ -517,8 +562,7 @@ func (as *activeRuleSet) applyIDToPipeline( var matched bool rollupID, matched := as.matchRollupTarget( sortedTagPairBytes, - rollupOp.NewName, - rollupOp.Tags, + rollupOp, tagPairs, matchRollupTargetOptions{generateRollupID: true}, ) @@ -620,13 +664,12 @@ func (as *activeRuleSet) reverseMappingsForRollupID( continue } rollupOp := pipelineOp.Rollup - if !bytes.Equal(rollupOp.NewName, name) { + if !bytes.Equal(rollupOp.NewName(name), name) { continue } if _, matched := as.matchRollupTarget( sortedTagPairBytes, - rollupOp.NewName, - rollupOp.Tags, + rollupOp, nil, matchRollupTargetOptions{generateRollupID: false}, ); !matched { diff --git a/src/metrics/rules/active_ruleset_test.go b/src/metrics/rules/active_ruleset_test.go index 809e36b893..0a8336ae04 100644 --- a/src/metrics/rules/active_ruleset_test.go +++ b/src/metrics/rules/active_ruleset_test.go @@ -3222,6 +3222,14 @@ func TestActiveRuleSetReverseMatchWithRollupRulesForRollupID(t *testing.T) { } func TestMatchedKeepOriginal(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rollup.r2", + []string{"foo"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + filter, err := filters.NewTagsFilter( filters.TagFilterValueMap{ "foo": filters.FilterValue{Pattern: "bar"}, @@ -3237,12 +3245,8 @@ func TestMatchedKeepOriginal(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rollup.r2"), - Tags: bs("foo"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3254,21 +3258,21 @@ func TestMatchedKeepOriginal(t *testing.T) { { uuid: "rollup", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollup.1.nokeep", cutoverNanos: 0, filter: filter, keepOriginal: false, targets: targets, }, - &rollupRuleSnapshot{ + { name: "rollup.2.keep", cutoverNanos: 10000, filter: filter, keepOriginal: true, targets: targets, }, - &rollupRuleSnapshot{ + { name: "rollup.2.nokeep", tombstoned: false, cutoverNanos: 20000, @@ -3368,7 +3372,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule1 := &mappingRule{ uuid: "mappingRule1", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule1.snapshot1", tombstoned: false, cutoverNanos: 10000, @@ -3379,7 +3383,7 @@ func testMappingRules(t *testing.T) []*mappingRule { }, tags: tags, }, - &mappingRuleSnapshot{ + { name: "mappingRule1.snapshot1", tombstoned: false, cutoverNanos: 15000, @@ -3390,7 +3394,7 @@ func testMappingRules(t *testing.T) []*mappingRule { }, tags: tags, }, - &mappingRuleSnapshot{ + { name: "mappingRule1.snapshot2", tombstoned: false, cutoverNanos: 20000, @@ -3403,7 +3407,7 @@ func testMappingRules(t *testing.T) []*mappingRule { }, tags: tags, }, - &mappingRuleSnapshot{ + { name: "mappingRule1.snapshot3", tombstoned: false, cutoverNanos: 30000, @@ -3420,7 +3424,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule2 := &mappingRule{ uuid: "mappingRule2", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule2.snapshot1", tombstoned: false, cutoverNanos: 15000, @@ -3430,7 +3434,7 @@ func testMappingRules(t *testing.T) []*mappingRule { policy.NewStoragePolicy(10*time.Second, xtime.Second, 12*time.Hour), }, }, - &mappingRuleSnapshot{ + { name: "mappingRule2.snapshot2", tombstoned: false, cutoverNanos: 22000, @@ -3441,7 +3445,7 @@ func testMappingRules(t *testing.T) []*mappingRule { policy.NewStoragePolicy(time.Minute, xtime.Minute, time.Hour), }, }, - &mappingRuleSnapshot{ + { name: "mappingRule2.snapshot3", tombstoned: true, cutoverNanos: 35000, @@ -3458,7 +3462,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule3 := &mappingRule{ uuid: "mappingRule3", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule3.snapshot1", tombstoned: false, cutoverNanos: 22000, @@ -3470,7 +3474,7 @@ func testMappingRules(t *testing.T) []*mappingRule { policy.NewStoragePolicy(5*time.Minute, xtime.Minute, 48*time.Hour), }, }, - &mappingRuleSnapshot{ + { name: "mappingRule3.snapshot2", tombstoned: false, cutoverNanos: 34000, @@ -3487,7 +3491,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule4 := &mappingRule{ uuid: "mappingRule4", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule4.snapshot1", tombstoned: false, cutoverNanos: 24000, @@ -3503,7 +3507,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule5 := &mappingRule{ uuid: "mappingRule5", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule5.snapshot1", tombstoned: false, cutoverNanos: 100000, @@ -3519,7 +3523,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule6 := &mappingRule{ uuid: "mappingRule6", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule6.snapshot1", tombstoned: false, cutoverNanos: 5000, @@ -3529,7 +3533,7 @@ func testMappingRules(t *testing.T) []*mappingRule { policy.NewStoragePolicy(10*time.Second, xtime.Second, 24*time.Hour), }, }, - &mappingRuleSnapshot{ + { name: "mappingRule6.snapshot2", tombstoned: false, cutoverNanos: 8000, @@ -3548,7 +3552,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule7 := &mappingRule{ uuid: "mappingRule7", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule7.snapshot1", tombstoned: false, cutoverNanos: 20000, @@ -3564,7 +3568,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule8 := &mappingRule{ uuid: "mappingRule8", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule8.snapshot1", tombstoned: false, cutoverNanos: 20000, @@ -3581,7 +3585,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule9 := &mappingRule{ uuid: "mappingRule9", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule9.snapshot1", tombstoned: false, cutoverNanos: 20000, @@ -3600,7 +3604,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule10 := &mappingRule{ uuid: "mappingRule10", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule10.snapshot1", tombstoned: false, cutoverNanos: 20000, @@ -3616,7 +3620,7 @@ func testMappingRules(t *testing.T) []*mappingRule { mappingRule11 := &mappingRule{ uuid: "mappingRule11", snapshots: []*mappingRuleSnapshot{ - &mappingRuleSnapshot{ + { name: "mappingRule11.snapshot1", tombstoned: false, cutoverNanos: 20000, @@ -3643,10 +3647,32 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { ) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName3", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rollupRule1 := &rollupRule{ uuid: "rollupRule1", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule1.snapshot", tombstoned: false, cutoverNanos: 0, @@ -3656,12 +3682,8 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName1"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3676,7 +3698,7 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { rollupRule2 := &rollupRule{ uuid: "rollupRule2", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule2.snapshot", tombstoned: false, cutoverNanos: 0, @@ -3686,12 +3708,8 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName2"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3706,7 +3724,7 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { rollupRule3 := &rollupRule{ uuid: "rollupRule3", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule3.snapshot", tombstoned: false, cutoverNanos: 0, @@ -3716,12 +3734,8 @@ func testKeepOriginalRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3755,10 +3769,95 @@ func testRollupRules(t *testing.T) []*rollupRule { ) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2_2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Sum), + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName3", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3_2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName3", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName4", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr5, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName5", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr6, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName6", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr7, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName7", + []string{"rtagName1"}, + aggregation.MustCompressTypes(aggregation.Sum, aggregation.P90), + ) + require.NoError(t, err) + rr8, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName81", + []string{"rtagName1", "rtagName2", "rtagName3"}, + aggregation.MustCompressTypes(aggregation.Sum, aggregation.P90), + ) + require.NoError(t, err) + rr8_2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName82", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Count), + ) + require.NoError(t, err) + rr8_3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName83", + []string{"rtagName1"}, + aggregation.MustCompressTypes(aggregation.Min), + ) + require.NoError(t, err) + rollupRule1 := &rollupRule{ uuid: "rollupRule1", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule1.snapshot1", tombstoned: false, cutoverNanos: 10000, @@ -3767,12 +3866,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName1"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3781,7 +3876,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule1.snapshot2", tombstoned: false, cutoverNanos: 20000, @@ -3797,12 +3892,8 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName1"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3813,7 +3904,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule1.snapshot3", tombstoned: false, cutoverNanos: 30000, @@ -3828,12 +3919,8 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName1"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3848,7 +3935,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule2 := &rollupRule{ uuid: "rollupRule2", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule2.snapshot1", tombstoned: false, cutoverNanos: 15000, @@ -3857,12 +3944,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName2"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3871,7 +3954,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule2.snapshot2", tombstoned: false, cutoverNanos: 22000, @@ -3892,12 +3975,8 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName2"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - }, + Type: pipeline.RollupOpType, + Rollup: rr2_2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3907,7 +3986,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule2.snapshot3", tombstoned: false, cutoverNanos: 35000, @@ -3916,12 +3995,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName2"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3936,7 +4011,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule3 := &rollupRule{ uuid: "rollupRule3", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule3.snapshot1", tombstoned: false, cutoverNanos: 22000, @@ -3945,12 +4020,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3962,12 +4033,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3_2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -3976,7 +4043,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule3.snapshot2", tombstoned: false, cutoverNanos: 34000, @@ -3985,12 +4052,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4000,7 +4063,7 @@ func testRollupRules(t *testing.T) []*rollupRule { }, }, }, - &rollupRuleSnapshot{ + { name: "rollupRule3.snapshot3", tombstoned: true, cutoverNanos: 38000, @@ -4009,12 +4072,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4030,7 +4089,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule4 := &rollupRule{ uuid: "rollupRule4", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule4.snapshot1", tombstoned: false, cutoverNanos: 24000, @@ -4039,12 +4098,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName4"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4059,7 +4114,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule5 := &rollupRule{ uuid: "rollupRule5", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule5.snapshot1", tombstoned: false, cutoverNanos: 24000, @@ -4070,12 +4125,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName5"), - Tags: bs("rtagName1"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr5, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4090,7 +4141,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule6 := &rollupRule{ uuid: "rollupRule6", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule6.snapshot1", tombstoned: false, cutoverNanos: 100000, @@ -4099,12 +4150,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName6"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr6, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4119,7 +4166,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule7 := &rollupRule{ uuid: "rollupRule7", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule7.snapshot1", tombstoned: false, cutoverNanos: 120000, @@ -4130,12 +4177,8 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName7"), - Tags: bs("rtagName1"), - AggregationID: aggregation.MustCompressTypes(aggregation.Sum, aggregation.P90), - }, + Type: pipeline.RollupOpType, + Rollup: rr7, }, }), StoragePolicies: policy.StoragePolicies{ @@ -4151,7 +4194,7 @@ func testRollupRules(t *testing.T) []*rollupRule { rollupRule8 := &rollupRule{ uuid: "rollupRule8", snapshots: []*rollupRuleSnapshot{ - &rollupRuleSnapshot{ + { name: "rollupRule8.snapshot1", tombstoned: false, cutoverNanos: 90000, @@ -4162,28 +4205,16 @@ func testRollupRules(t *testing.T) []*rollupRule { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName81"), - Tags: bs("rtagName1", "rtagName2", "rtagName3"), - AggregationID: aggregation.MustCompressTypes(aggregation.Sum, aggregation.P90), - }, + Type: pipeline.RollupOpType, + Rollup: rr8, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName82"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Count), - }, + Type: pipeline.RollupOpType, + Rollup: rr8_2, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName83"), - Tags: bs("rtagName1"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min), - }, + Type: pipeline.RollupOpType, + Rollup: rr8_3, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/rules/rollup_target.go b/src/metrics/rules/rollup_target.go index bc6863586b..e04b203287 100644 --- a/src/metrics/rules/rollup_target.go +++ b/src/metrics/rules/rollup_target.go @@ -22,13 +22,11 @@ package rules import ( "errors" - "sort" "github.com/m3db/m3/src/metrics/generated/proto/rulepb" "github.com/m3db/m3/src/metrics/pipeline" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/metrics/rules/view" - "github.com/m3db/m3/src/metrics/x/bytes" ) var ( @@ -56,16 +54,16 @@ func newRollupTargetFromV1Proto(pb *rulepb.RollupTarget) (rollupTarget, error) { if err != nil { return emptyRollupTarget, err } - tags := make([]string, len(pb.Tags)) - copy(tags, pb.Tags) - sort.Strings(tags) + + rollup, err := pipeline.NewRollupOp(pipeline.GroupByRollupType, pb.Name, + pb.Tags, aggregationID) + if err != nil { + return emptyRollupTarget, err + } + rollupOp := pipeline.OpUnion{ - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte(pb.Name), - Tags: bytes.ArraysFromStringArray(tags), - AggregationID: aggregationID, - }, + Type: pipeline.RollupOpType, + Rollup: rollup, } pipeline := pipeline.NewPipeline([]pipeline.OpUnion{rollupOp}) return rollupTarget{ diff --git a/src/metrics/rules/rollup_target_test.go b/src/metrics/rules/rollup_target_test.go index fab69e4dfc..f73aa7a3c1 100644 --- a/src/metrics/rules/rollup_target_test.go +++ b/src/metrics/rules/rollup_target_test.go @@ -69,7 +69,7 @@ func TestNewRollupTargetV1ProtoWithDefaultAggregationID(t *testing.T) { Name: "testV1Proto", Tags: []string{"testTag2", "testTag1"}, Policies: []*policypb.Policy{ - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: 10 * time.Second.Nanoseconds(), @@ -80,7 +80,7 @@ func TestNewRollupTargetV1ProtoWithDefaultAggregationID(t *testing.T) { }, }, }, - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), @@ -91,7 +91,7 @@ func TestNewRollupTargetV1ProtoWithDefaultAggregationID(t *testing.T) { }, }, }, - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: time.Hour.Nanoseconds(), @@ -107,15 +107,19 @@ func TestNewRollupTargetV1ProtoWithDefaultAggregationID(t *testing.T) { res, err := newRollupTargetFromV1Proto(proto) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testV1Proto", + []string{"testTag1", "testTag2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + expected := rollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testV1Proto"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -132,7 +136,7 @@ func TestNewRollupTargetV1ProtoWithCustomAggregationID(t *testing.T) { Name: "testV1Proto", Tags: []string{"testTag2", "testTag1"}, Policies: []*policypb.Policy{ - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: 10 * time.Second.Nanoseconds(), @@ -144,7 +148,7 @@ func TestNewRollupTargetV1ProtoWithCustomAggregationID(t *testing.T) { }, AggregationTypes: []aggregationpb.AggregationType{1, 2}, }, - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), @@ -156,7 +160,7 @@ func TestNewRollupTargetV1ProtoWithCustomAggregationID(t *testing.T) { }, AggregationTypes: []aggregationpb.AggregationType{1, 2}, }, - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: time.Hour.Nanoseconds(), @@ -173,15 +177,19 @@ func TestNewRollupTargetV1ProtoWithCustomAggregationID(t *testing.T) { res, err := newRollupTargetFromV1Proto(proto) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testV1Proto", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Last, aggregation.Min), + ) + require.NoError(t, err) + expected := rollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testV1Proto"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Last, aggregation.Min), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -268,7 +276,7 @@ func TestNewRollupTargetV2Proto(t *testing.T) { }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: 10 * time.Second.Nanoseconds(), Precision: time.Second.Nanoseconds(), @@ -277,7 +285,7 @@ func TestNewRollupTargetV2Proto(t *testing.T) { Period: 24 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -286,7 +294,7 @@ func TestNewRollupTargetV2Proto(t *testing.T) { Period: 720 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Hour.Nanoseconds(), Precision: time.Hour.Nanoseconds(), @@ -300,6 +308,14 @@ func TestNewRollupTargetV2Proto(t *testing.T) { res, err := newRollupTargetFromV2Proto(proto) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + require.NoError(t, err) + expected := rollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { @@ -315,12 +331,8 @@ func TestNewRollupTargetV2Proto(t *testing.T) { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -333,6 +345,14 @@ func TestNewRollupTargetV2Proto(t *testing.T) { } func TestRollupTargetClone(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + require.NoError(t, err) + source := rollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { @@ -348,12 +368,8 @@ func TestRollupTargetClone(t *testing.T) { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -410,6 +426,14 @@ func TestRollupTargetProtoInvalidStoragePolicies(t *testing.T) { } func TestRollupTargetProto(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + require.NoError(t, err) + target := rollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { @@ -425,12 +449,8 @@ func TestRollupTargetProto(t *testing.T) { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -471,7 +491,7 @@ func TestRollupTargetProto(t *testing.T) { }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: 10 * time.Second.Nanoseconds(), Precision: time.Second.Nanoseconds(), @@ -480,7 +500,7 @@ func TestRollupTargetProto(t *testing.T) { Period: 24 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -489,7 +509,7 @@ func TestRollupTargetProto(t *testing.T) { Period: 720 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Hour.Nanoseconds(), Precision: time.Hour.Nanoseconds(), diff --git a/src/metrics/rules/rollup_test.go b/src/metrics/rules/rollup_test.go index 0a2655290a..18aa1c4ab2 100644 --- a/src/metrics/rules/rollup_test.go +++ b/src/metrics/rules/rollup_test.go @@ -54,11 +54,11 @@ var ( Filter: "tag1:value1 tag2:value2", KeepOriginal: false, Targets: []*rulepb.RollupTarget{ - &rulepb.RollupTarget{ + { Name: "rName1", Tags: []string{"rtagName1", "rtagName2"}, Policies: []*policypb.Policy{ - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: int64(10 * time.Second), @@ -82,11 +82,11 @@ var ( Filter: "tag3:value3 tag4:value4", KeepOriginal: false, Targets: []*rulepb.RollupTarget{ - &rulepb.RollupTarget{ + { Name: "rName1", Tags: []string{"rtagName1", "rtagName2"}, Policies: []*policypb.Policy{ - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: int64(time.Minute), @@ -100,7 +100,7 @@ var ( aggregationpb.AggregationType_MEAN, }, }, - &policypb.Policy{ + { StoragePolicy: &policypb.StoragePolicy{ Resolution: policypb.Resolution{ WindowSize: int64(5 * time.Minute), @@ -127,7 +127,7 @@ var ( Filter: "tag1:value1 tag2:value2", KeepOriginal: false, TargetsV2: []*rulepb.RollupTargetV2{ - &rulepb.RollupTargetV2{ + { Pipeline: &pipelinepb.Pipeline{ Ops: []pipelinepb.PipelineOp{ { @@ -156,7 +156,7 @@ var ( }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: 10 * time.Second.Nanoseconds(), Precision: time.Second.Nanoseconds(), @@ -165,7 +165,7 @@ var ( Period: 24 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -174,7 +174,7 @@ var ( Period: 720 * time.Hour.Nanoseconds(), }, }, - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Hour.Nanoseconds(), Precision: time.Hour.Nanoseconds(), @@ -185,7 +185,7 @@ var ( }, }, }, - &rulepb.RollupTargetV2{ + { Pipeline: &pipelinepb.Pipeline{ Ops: []pipelinepb.PipelineOp{ { @@ -204,7 +204,7 @@ var ( }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -226,7 +226,7 @@ var ( Filter: "tag3:value3 tag4:value4", KeepOriginal: true, TargetsV2: []*rulepb.RollupTargetV2{ - &rulepb.RollupTargetV2{ + { Pipeline: &pipelinepb.Pipeline{ Ops: []pipelinepb.PipelineOp{ { @@ -242,7 +242,7 @@ var ( }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: 10 * time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -269,6 +269,36 @@ var ( testRollupRuleSnapshot4V2Proto, }, } + rr1, rr1err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + rr2, rr2err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Mean), + ) + rr3, rr3err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + rr4, rr4err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp2", + []string{"testTag3", "testTag4"}, + aggregation.DefaultID, + ) + rr5, rr5err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp2", + []string{"testTag3", "testTag4"}, + aggregation.MustCompressTypes(aggregation.Last), + ) testRollupRuleSnapshot1 = &rollupRuleSnapshot{ name: "foo", tombstoned: false, @@ -279,12 +309,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -305,12 +331,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: bs("rtagName1", "rtagName2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Mean), - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -344,12 +366,8 @@ var ( }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -367,12 +385,8 @@ var ( }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp2"), - Tags: bs("testTag3", "testTag4"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -393,12 +407,8 @@ var ( { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp2"), - Tags: bs("testTag3", "testTag4"), - AggregationID: aggregation.MustCompressTypes(aggregation.Last), - }, + Type: pipeline.RollupOpType, + Rollup: rr5, }, }), StoragePolicies: policy.StoragePolicies{ @@ -434,6 +444,14 @@ var ( } ) +func TestErrCheck(t *testing.T) { + require.NoError(t, rr1err) + require.NoError(t, rr2err) + require.NoError(t, rr3err) + require.NoError(t, rr4err) + require.NoError(t, rr5err) +} + func TestNewRollupRuleSnapshotFromProtoNilProto(t *testing.T) { _, err := newRollupRuleSnapshotFromProto(nil, testTagsFilterOptions()) require.Equal(t, errNilRollupRuleSnapshotProto, err) @@ -442,11 +460,11 @@ func TestNewRollupRuleSnapshotFromProtoNilProto(t *testing.T) { func TestNewRollupRuleSnapshotFromV1ProtoInvalidProto(t *testing.T) { proto := &rulepb.RollupRuleSnapshot{ Targets: []*rulepb.RollupTarget{ - &rulepb.RollupTarget{ + { Name: "rName1", Tags: []string{"rtagName1", "rtagName2"}, Policies: []*policypb.Policy{ - &policypb.Policy{}, + {}, }, }, }, @@ -477,7 +495,7 @@ func TestNewRollupRuleSnapshotFromV2ProtoInvalidProto(t *testing.T) { filterOpts := testTagsFilterOptions() proto := &rulepb.RollupRuleSnapshot{ TargetsV2: []*rulepb.RollupTargetV2{ - &rulepb.RollupTargetV2{ + { Pipeline: &pipelinepb.Pipeline{ Ops: []pipelinepb.PipelineOp{ { @@ -489,7 +507,7 @@ func TestNewRollupRuleSnapshotFromV2ProtoInvalidProto(t *testing.T) { }, }, StoragePolicies: []*policypb.StoragePolicy{ - &policypb.StoragePolicy{ + { Resolution: policypb.Resolution{ WindowSize: 10 * time.Minute.Nanoseconds(), Precision: time.Minute.Nanoseconds(), @@ -752,7 +770,13 @@ func TestRollupRuleMarkTombstonedAlreadyTombstoned(t *testing.T) { func TestRollupRuleRollupRuleView(t *testing.T) { res, err := testRollupRule2.rollupRuleView(1) require.NoError(t, err) - + rr1, err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp2", + []string{"testTag3", "testTag4"}, + aggregation.MustCompressTypes(aggregation.Last), + ) + require.NoError(t, err) expected := view.RollupRule{ ID: "12669817-13ae-40e6-ba2f-33087b262c68", Name: "bar", @@ -764,12 +788,8 @@ func TestRollupRuleRollupRuleView(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp2"), - Tags: bs("testTag3", "testTag4"), - AggregationID: aggregation.MustCompressTypes(aggregation.Last), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -795,6 +815,28 @@ func TestNewRollupRuleHistory(t *testing.T) { history, err := testRollupRule2.history() require.NoError(t, err) + rr1, err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp2", + []string{"testTag3", "testTag4"}, + aggregation.MustCompressTypes(aggregation.Last), + ) + require.NoError(t, err) + rr2, err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp", + []string{"testTag1", "testTag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + require.NoError(t, err) + rr3, err = pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollupOp2", + []string{"testTag3", "testTag4"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + expected := []view.RollupRule{ { ID: "12669817-13ae-40e6-ba2f-33087b262c68", @@ -807,12 +849,8 @@ func TestNewRollupRuleHistory(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp2"), - Tags: bs("testTag3", "testTag4"), - AggregationID: aggregation.MustCompressTypes(aggregation.Last), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -845,12 +883,8 @@ func TestNewRollupRuleHistory(t *testing.T) { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp"), - Tags: bs("testTag1", "testTag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -868,12 +902,8 @@ func TestNewRollupRuleHistory(t *testing.T) { }, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("testRollupOp2"), - Tags: bs("testTag3", "testTag4"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/rules/ruleset_test.go b/src/metrics/rules/ruleset_test.go index 322534e8c6..aca14b8e69 100644 --- a/src/metrics/rules/ruleset_test.go +++ b/src/metrics/rules/ruleset_test.go @@ -275,6 +275,42 @@ func TestRuleSetLatest(t *testing.T) { latest, err := rs.Latest() require.NoError(t, err) + r1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + r3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName3", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + r4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName4", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + r5, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName5", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + r6, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName6", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + expected := view.RuleSet{ Namespace: "testNamespace", Version: 123, @@ -337,11 +373,8 @@ func TestRuleSetLatest(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName1"), - Tags: bs("rtagName1", "rtagName2"), - }, + Type: pipeline.RollupOpType, + Rollup: r1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -359,11 +392,8 @@ func TestRuleSetLatest(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName3"), - Tags: bs("rtagName1", "rtagName2"), - }, + Type: pipeline.RollupOpType, + Rollup: r3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -382,11 +412,8 @@ func TestRuleSetLatest(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName4"), - Tags: bs("rtagName1", "rtagName2"), - }, + Type: pipeline.RollupOpType, + Rollup: r4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -404,11 +431,8 @@ func TestRuleSetLatest(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName5"), - Tags: bs("rtagName1"), - }, + Type: pipeline.RollupOpType, + Rollup: r5, }, }), StoragePolicies: policy.StoragePolicies{ @@ -426,11 +450,8 @@ func TestRuleSetLatest(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("rName6"), - Tags: bs("rtagName1", "rtagName2"), - }, + Type: pipeline.RollupOpType, + Rollup: r6, }, }), StoragePolicies: policy.StoragePolicies{ @@ -733,6 +754,13 @@ func TestRuleSetAddRollupRuleNewRule(t *testing.T) { _, err = rs.getRollupRuleByName("foo") require.Equal(t, errRuleNotFound, err) + r1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "blah", + []string{"a"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RollupRule{ Name: "foo", @@ -742,12 +770,8 @@ func TestRuleSetAddRollupRuleNewRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: r1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -778,12 +802,8 @@ func TestRuleSetAddRollupRuleNewRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: r1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -814,7 +834,13 @@ func TestRuleSetAddRollupRuleDuplicateRule(t *testing.T) { r, err := rs.getRollupRuleByID("rollupRule5") require.NoError(t, err) require.NotNil(t, r) - + r1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "blah", + []string{"a"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RollupRule{ Name: "rollupRule5.snapshot1", Filter: "test:bar", @@ -822,12 +848,8 @@ func TestRuleSetAddRollupRuleDuplicateRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: r1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -868,12 +890,8 @@ func TestRuleSetAddRollupRuleReviveRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -904,12 +922,8 @@ func TestRuleSetAddRollupRuleReviveRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -940,6 +954,14 @@ func TestRuleSetUpdateRollupRule(t *testing.T) { rr, err := rs.getRollupRuleByID("rollupRule5") require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "blah", + []string{"a"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RollupRule{ ID: "rollupRule5", Name: "rollupRule5.snapshot2", @@ -949,12 +971,8 @@ func TestRuleSetUpdateRollupRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -985,12 +1003,8 @@ func TestRuleSetUpdateRollupRule(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("blah"), - Tags: bs("a"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/rules/validator/validator.go b/src/metrics/rules/validator/validator.go index 087f75b2c0..485baf2238 100644 --- a/src/metrics/rules/validator/validator.go +++ b/src/metrics/rules/validator/validator.go @@ -371,14 +371,15 @@ func (v *validator) validateRollupOp( types []metric.Type, previousRollupTags map[string]struct{}, ) error { + newName := rollupOp.NewName([]byte("")) // Validate that the rollup metric name is valid. - if err := v.validateRollupMetricName(rollupOp.NewName); err != nil { - return fmt.Errorf("invalid rollup metric name '%s': %v", rollupOp.NewName, err) + if err := v.validateRollupMetricName(newName); err != nil { + return fmt.Errorf("invalid rollup metric name '%s': %w", newName, err) } // Validate that the rollup tags are valid. if err := v.validateRollupTags(rollupOp.Tags, previousRollupTags); err != nil { - return fmt.Errorf("invalid rollup tags %v: %v", rollupOp.Tags, err) + return fmt.Errorf("invalid rollup tags %v: %w", rollupOp.Tags, err) } // Validate that the aggregation ID is valid. @@ -387,7 +388,7 @@ func (v *validator) validateRollupOp( aggType = nonFirstLevelAggregationType } if err := v.validateAggregationID(rollupOp.AggregationID, aggType, types); err != nil { - return fmt.Errorf("invalid aggregation ID %v: %v", rollupOp.AggregationID, err) + return fmt.Errorf("invalid aggregation ID %v: %w", rollupOp.AggregationID, err) } return nil @@ -466,7 +467,11 @@ func validateNoDuplicateRollupIDIn(pipelines []mpipeline.Pipeline) error { rollupOp := pipelineOp.Rollup for _, existing := range rollupOps { if rollupOp.SameTransform(existing) { - return merrors.NewInvalidInputError(fmt.Sprintf("more than one rollup operations with name '%s' and tags '%s' exist", rollupOp.NewName, rollupOp.Tags)) + return merrors.NewInvalidInputError(fmt.Sprintf( + "more than one rollup operations with name '%s' and tags '%s' exist", + rollupOp.NewName([]byte("")), + rollupOp.Tags, + )) } } rollupOps = append(rollupOps, rollupOp) diff --git a/src/metrics/rules/validator/validator_test.go b/src/metrics/rules/validator/validator_test.go index 23533b0cf7..3b077bdb42 100644 --- a/src/metrics/rules/validator/validator_test.go +++ b/src/metrics/rules/validator/validator_test.go @@ -359,6 +359,21 @@ func TestValidatorValidateDuplicateRollupRules(t *testing.T) { } func TestValidatorValidateNoDuplicateRollupRulesWithTombstone(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -369,12 +384,8 @@ func TestValidatorValidateNoDuplicateRollupRulesWithTombstone(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -388,12 +399,8 @@ func TestValidatorValidateNoDuplicateRollupRulesWithTombstone(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: testStoragePolicies(), @@ -408,6 +415,14 @@ func TestValidatorValidateNoDuplicateRollupRulesWithTombstone(t *testing.T) { } func TestValidatorValidateRollupRuleInvalidFilterExpr(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -417,12 +432,8 @@ func TestValidatorValidateRollupRuleInvalidFilterExpr(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -437,6 +448,14 @@ func TestValidatorValidateRollupRuleInvalidFilterExpr(t *testing.T) { } func TestValidatorValidateRollupRuleInvalidFilterTagName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + invalidChars := []rune{'$'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -447,12 +466,8 @@ func TestValidatorValidateRollupRuleInvalidFilterTagName(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -466,6 +481,14 @@ func TestValidatorValidateRollupRuleInvalidFilterTagName(t *testing.T) { } func TestValidatorValidateRollupRuleInvalidMetricType(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -475,12 +498,8 @@ func TestValidatorValidateRollupRuleInvalidMetricType(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -659,6 +678,14 @@ func TestValidatorValidateRollupRulePipelineAggregationOpDisallowedAggregationTy } func TestValidatorValidateRollupRulePipelineTransformationDerivativeOrderNotSupported(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -668,12 +695,8 @@ func TestValidatorValidateRollupRulePipelineTransformationDerivativeOrderNotSupp { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, { Type: pipeline.TransformationOpType, @@ -691,7 +714,7 @@ func TestValidatorValidateRollupRulePipelineTransformationDerivativeOrderNotSupp }, } validator := NewValidator(testValidatorOptions()) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "transformation derivative order is 2 higher than supported 1")) } @@ -749,6 +772,21 @@ func TestValidatorValidateRollupRulePipelineNoRollupOp(t *testing.T) { } func TestValidatorValidateRollupRulePipelineRollupLevelHigherThanMax(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -758,20 +796,12 @@ func TestValidatorValidateRollupRulePipelineRollupLevelHigherThanMax(t *testing. { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName2"), - Tags: [][]byte{[]byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: testStoragePolicies(), @@ -781,12 +811,27 @@ func TestValidatorValidateRollupRulePipelineRollupLevelHigherThanMax(t *testing. }, } validator := NewValidator(testValidatorOptions()) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "number of rollup levels is 2 higher than supported 1")) } func TestValidatorValidateRollupRulePipelineRollupTagNotFoundInPrevRollupOp(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2", "rtagName3"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -796,20 +841,12 @@ func TestValidatorValidateRollupRulePipelineRollupTagNotFoundInPrevRollupOp(t *t { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName2"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2"), []byte("rtagName3")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: testStoragePolicies(), @@ -819,12 +856,26 @@ func TestValidatorValidateRollupRulePipelineRollupTagNotFoundInPrevRollupOp(t *t }, } validator := NewValidator(testValidatorOptions().SetMaxRollupLevels(100)) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "tag rtagName3 not found in previous rollup operations")) } func TestValidatorValidateRollupRulePipelineRollupTagUnchangedInConsecutiveRollupOps(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -834,20 +885,12 @@ func TestValidatorValidateRollupRulePipelineRollupTagUnchangedInConsecutiveRollu { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName2"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: testStoragePolicies(), @@ -857,12 +900,34 @@ func TestValidatorValidateRollupRulePipelineRollupTagUnchangedInConsecutiveRollu }, } validator := NewValidator(testValidatorOptions().SetMaxRollupLevels(100)) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "same set of 2 rollup tags in consecutive rollup operations")) } func TestValidatorValidateRollupRulePipelineMultiLevelRollup(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2", "rtagName3"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName2", + []string{"rtagName1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -872,28 +937,16 @@ func TestValidatorValidateRollupRulePipelineMultiLevelRollup(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2"), []byte("rtagName3")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName2"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName2"), - Tags: [][]byte{[]byte("rtagName1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: testStoragePolicies(), @@ -907,6 +960,14 @@ func TestValidatorValidateRollupRulePipelineMultiLevelRollup(t *testing.T) { } func TestValidatorValidateRollupRuleRollupOpDuplicateRollupTag(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -916,12 +977,8 @@ func TestValidatorValidateRollupRuleRollupOpDuplicateRollupTag(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -931,12 +988,20 @@ func TestValidatorValidateRollupRuleRollupOpDuplicateRollupTag(t *testing.T) { }, } validator := NewValidator(testValidatorOptions()) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "duplicate rollup tag: 'rtagName2'")) } func TestValidatorValidateRollupRuleRollupOpMissingRequiredTag(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -946,12 +1011,8 @@ func TestValidatorValidateRollupRuleRollupOpMissingRequiredTag(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -961,12 +1022,19 @@ func TestValidatorValidateRollupRuleRollupOpMissingRequiredTag(t *testing.T) { }, } validator := NewValidator(testValidatorOptions().SetRequiredRollupTags([]string{"requiredTag"})) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "missing required rollup tag: 'requiredTag'")) } func TestValidatorValidateRollupRuleRollupOpWithInvalidMetricName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName$1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) invalidChars := []rune{'$'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -977,12 +1045,8 @@ func TestValidatorValidateRollupRuleRollupOpWithInvalidMetricName(t *testing.T) { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName$1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -997,6 +1061,13 @@ func TestValidatorValidateRollupRuleRollupOpWithInvalidMetricName(t *testing.T) } func TestValidatorValidateRollupRuleRollupOpWithEmptyMetricName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) invalidChars := []rune{'$'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1007,12 +1078,8 @@ func TestValidatorValidateRollupRuleRollupOpWithEmptyMetricName(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte(""), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1027,6 +1094,13 @@ func TestValidatorValidateRollupRuleRollupOpWithEmptyMetricName(t *testing.T) { } func TestValidatorValidateRollupRuleRollupOpWithValidMetricName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) invalidChars := []rune{' ', '%'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1037,12 +1111,8 @@ func TestValidatorValidateRollupRuleRollupOpWithValidMetricName(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte(""), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1057,6 +1127,13 @@ func TestValidatorValidateRollupRuleRollupOpWithValidMetricName(t *testing.T) { } func TestValidatorValidateRollupRuleRollupOpWithInvalidTagName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "foo", + []string{"rtagName1", "rtagName2", "$"}, + aggregation.DefaultID, + ) + require.NoError(t, err) invalidChars := []rune{'$'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1067,12 +1144,8 @@ func TestValidatorValidateRollupRuleRollupOpWithInvalidTagName(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("foo"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2$"), []byte("$")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1087,6 +1160,13 @@ func TestValidatorValidateRollupRuleRollupOpWithInvalidTagName(t *testing.T) { } func TestValidatorValidateRollupRuleRollupOpWithValidTagName(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "foo", + []string{"rtagName1", "rtagName2", "$"}, + aggregation.DefaultID, + ) + require.NoError(t, err) invalidChars := []rune{' ', '%'} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1097,12 +1177,8 @@ func TestValidatorValidateRollupRuleRollupOpWithValidTagName(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("foo"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2$"), []byte("$")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1155,6 +1231,14 @@ func TestValidatorValidateNoTimertypeFilter(t *testing.T) { } func TestValidatorValidateRollupRuleRollupOpMultipleAggregationTypes(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), + ) + require.NoError(t, err) + testAggregationTypes := []aggregation.Type{aggregation.Count, aggregation.Max} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1165,12 +1249,8 @@ func TestValidatorValidateRollupRuleRollupOpMultipleAggregationTypes(t *testing. { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -1208,6 +1288,13 @@ func TestValidatorValidateRollupRuleRollupOpMultipleAggregationTypes(t *testing. } func TestValidatorValidateRollupRuleRollupOpFirstLevelAggregationTypes(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), + ) + require.NoError(t, err) testAggregationTypes := []aggregation.Type{aggregation.Count, aggregation.Max} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1218,12 +1305,8 @@ func TestValidatorValidateRollupRuleRollupOpFirstLevelAggregationTypes(t *testin { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -1266,6 +1349,13 @@ func TestValidatorValidateRollupRuleRollupOpFirstLevelAggregationTypes(t *testin } func TestValidatorValidateRollupRuleRollupOpNonFirstLevelAggregationTypes(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), + ) + require.NoError(t, err) testAggregationTypes := []aggregation.Type{aggregation.Count, aggregation.Max} view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1280,12 +1370,8 @@ func TestValidatorValidateRollupRuleRollupOpNonFirstLevelAggregationTypes(t *tes Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Count, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -1328,6 +1414,13 @@ func TestValidatorValidateRollupRuleRollupOpNonFirstLevelAggregationTypes(t *tes } func TestValidatorValidateRollupRuleRollupTargetWithStoragePolicies(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) storagePolicies := testStoragePolicies() view := view.RuleSet{ RollupRules: []view.RollupRule{ @@ -1338,12 +1431,8 @@ func TestValidatorValidateRollupRuleRollupTargetWithStoragePolicies(t *testing.T { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: storagePolicies, @@ -1385,6 +1474,13 @@ func TestValidatorValidateRollupRuleRollupTargetWithStoragePolicies(t *testing.T } func TestValidatorValidateRollupRuleRollupTargetWithNoStoragePolicies(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -1394,12 +1490,8 @@ func TestValidatorValidateRollupRuleRollupTargetWithNoStoragePolicies(t *testing { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), }, @@ -1408,12 +1500,19 @@ func TestValidatorValidateRollupRuleRollupTargetWithNoStoragePolicies(t *testing }, } validator := NewValidator(testValidatorOptions()) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "no storage policies")) } func TestValidatorValidateRollupRuleRollupOpWithDuplicateStoragePolicies(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -1423,12 +1522,8 @@ func TestValidatorValidateRollupRuleRollupOpWithDuplicateStoragePolicies(t *test { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -1442,12 +1537,19 @@ func TestValidatorValidateRollupRuleRollupOpWithDuplicateStoragePolicies(t *test } validator := NewValidator(testValidatorOptions()) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "duplicate storage policy '10s:6h'")) } func TestValidatorValidateRollupRuleDisallowedStoragePolicies(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -1457,12 +1559,8 @@ func TestValidatorValidateRollupRuleDisallowedStoragePolicies(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -1479,6 +1577,13 @@ func TestValidatorValidateRollupRuleDisallowedStoragePolicies(t *testing.T) { } func TestValidatorRollupRule(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Sum), + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -1496,12 +1601,8 @@ func TestValidatorRollupRule(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1521,6 +1622,20 @@ func TestValidatorRollupRule(t *testing.T) { } func TestValidatorValidateRollupRuleDuplicateRollupIDs(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.MustCompressTypes(aggregation.Sum), + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "rName1", + []string{"rtagName1", "rtagName2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) view := view.RuleSet{ RollupRules: []view.RollupRule{ { @@ -1538,12 +1653,8 @@ func TestValidatorValidateRollupRuleDuplicateRollupIDs(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: testStoragePolicies(), @@ -1557,12 +1668,8 @@ func TestValidatorValidateRollupRuleDuplicateRollupIDs(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("rName1"), - Tags: [][]byte{[]byte("rtagName1"), []byte("rtagName2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: testStoragePolicies(), @@ -1577,7 +1684,7 @@ func TestValidatorValidateRollupRuleDuplicateRollupIDs(t *testing.T) { SetAllowedFirstLevelAggregationTypesFor(metric.GaugeType, firstLevelAggregationTypes). SetAllowedNonFirstLevelAggregationTypesFor(metric.GaugeType, nonFirstLevelAggregationTypes) validator := NewValidator(opts) - err := validator.ValidateSnapshot(view) + err = validator.ValidateSnapshot(view) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "more than one rollup operations with name 'rName1' and tags '[rtagName1 rtagName2]' exist")) _, ok := err.(errors.InvalidInputError) diff --git a/src/metrics/rules/view/changes/rollup_test.go b/src/metrics/rules/view/changes/rollup_test.go index bcf8b9b717..2cfd4cda52 100644 --- a/src/metrics/rules/view/changes/rollup_test.go +++ b/src/metrics/rules/view/changes/rollup_test.go @@ -210,6 +210,21 @@ func TestRollupRuleJSONDeserialize(t *testing.T) { err := json.Unmarshal(jsonInput, &ruleChange) require.NoError(t, err) + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollup", + []string{"tag1", "tag2"}, + aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "testRollup", + []string{"tag1", "tag2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + expected := RollupRuleChange{ Op: ChangeOp, RuleID: ptr("validID"), @@ -229,12 +244,8 @@ func TestRollupRuleJSONDeserialize(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testRollup"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.MustCompressTypes(aggregation.Min, aggregation.Max), - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -245,12 +256,8 @@ func TestRollupRuleJSONDeserialize(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: b("testRollup"), - Tags: bs("tag1", "tag2"), - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/rules/view/rollup_test.go b/src/metrics/rules/view/rollup_test.go index 63158ebabd..df55503ccf 100644 --- a/src/metrics/rules/view/rollup_test.go +++ b/src/metrics/rules/view/rollup_test.go @@ -34,6 +34,21 @@ import ( ) func TestRollupTargetEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag1", "tag2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + target1 := RollupTarget{ Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { @@ -45,12 +60,8 @@ func TestRollupTargetEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag1"), []byte("tag2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -70,12 +81,8 @@ func TestRollupTargetEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -88,6 +95,35 @@ func TestRollupTargetEqual(t *testing.T) { } func TestRollupTargetNotEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.MustCompressTypes(aggregation.Sum), + ) + require.NoError(t, err) + rr4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.MustCompressTypes(aggregation.Sum), + ) + require.NoError(t, err) + targets := []RollupTarget{ { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ @@ -100,12 +136,8 @@ func TestRollupTargetNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -121,12 +153,8 @@ func TestRollupTargetNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -142,12 +170,8 @@ func TestRollupTargetNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -163,12 +187,8 @@ func TestRollupTargetNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - }, + Type: pipeline.RollupOpType, + Rollup: rr4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -194,6 +214,21 @@ func TestRollupTargetEqualNilCases(t *testing.T) { } func TestRollupRuleEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rule1 := RollupRule{ ID: "rr_id", Name: "rr_name", @@ -211,12 +246,8 @@ func TestRollupRuleEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -246,12 +277,8 @@ func TestRollupRuleEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -268,6 +295,21 @@ func TestRollupRuleEqual(t *testing.T) { } func TestRollupRuleNotEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rules := []RollupRule{ { ID: "rr_id", @@ -286,12 +328,8 @@ func TestRollupRuleNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -321,12 +359,8 @@ func TestRollupRuleNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -435,6 +469,35 @@ func TestRollupRuleEqualNilCases(t *testing.T) { } func TestRollupTargetsEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name2", + []string{"tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name2", + []string{"tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + targets1 := rollupTargets{ { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ @@ -447,12 +510,8 @@ func TestRollupTargetsEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -464,12 +523,8 @@ func TestRollupTargetsEqual(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name2"), - Tags: [][]byte{[]byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -489,12 +544,8 @@ func TestRollupTargetsEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -506,12 +557,8 @@ func TestRollupTargetsEqual(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name2"), - Tags: [][]byte{[]byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -523,6 +570,42 @@ func TestRollupTargetsEqual(t *testing.T) { } func TestRollupTargetsNotEqual(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name2", + []string{"tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr3, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr4, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name2", + []string{"tag2"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr5, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + targetsList := []rollupTargets{ { { @@ -536,12 +619,8 @@ func TestRollupTargetsNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -553,12 +632,8 @@ func TestRollupTargetsNotEqual(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name2"), - Tags: [][]byte{[]byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -578,12 +653,8 @@ func TestRollupTargetsNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr3, }, }), StoragePolicies: policy.StoragePolicies{ @@ -595,12 +666,8 @@ func TestRollupTargetsNotEqual(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name2"), - Tags: [][]byte{[]byte("tag2")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr4, }, }), StoragePolicies: policy.StoragePolicies{ @@ -620,12 +687,8 @@ func TestRollupTargetsNotEqual(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr5, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/rules/view/ruleset_test.go b/src/metrics/rules/view/ruleset_test.go index 78777ee5a6..c4a1c4c24f 100644 --- a/src/metrics/rules/view/ruleset_test.go +++ b/src/metrics/rules/view/ruleset_test.go @@ -34,6 +34,21 @@ import ( ) func TestRuleSetSort(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + ruleset := RuleSet{ Namespace: "testNamespace", Version: 1, @@ -68,12 +83,8 @@ func TestRuleSetSort(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -101,12 +112,8 @@ func TestRuleSetSort(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -165,12 +172,8 @@ func TestRuleSetSort(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -192,12 +195,8 @@ func TestRuleSetSort(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -214,6 +213,21 @@ func TestRuleSetSort(t *testing.T) { } func TestRuleSetsSort(t *testing.T) { + rr1, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rr2, err := pipeline.NewRollupOp( + pipeline.GroupByRollupType, + "name", + []string{"tag2", "tag1"}, + aggregation.DefaultID, + ) + require.NoError(t, err) + rulesets := RuleSets{ "ns1": &RuleSet{ Namespace: "ns1", @@ -262,12 +276,8 @@ func TestRuleSetsSort(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -289,12 +299,8 @@ func TestRuleSetsSort(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ @@ -350,12 +356,8 @@ func TestRuleSetsSort(t *testing.T) { { Pipeline: pipeline.NewPipeline([]pipeline.OpUnion{ { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr1, }, }), StoragePolicies: policy.StoragePolicies{ @@ -383,12 +385,8 @@ func TestRuleSetsSort(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, }, { - Type: pipeline.RollupOpType, - Rollup: pipeline.RollupOp{ - NewName: []byte("name"), - Tags: [][]byte{[]byte("tag2"), []byte("tag1")}, - AggregationID: aggregation.DefaultID, - }, + Type: pipeline.RollupOpType, + Rollup: rr2, }, }), StoragePolicies: policy.StoragePolicies{ diff --git a/src/metrics/transformation/type.go b/src/metrics/transformation/type.go index e66ebbe4c3..46e428682e 100644 --- a/src/metrics/transformation/type.go +++ b/src/metrics/transformation/type.go @@ -22,13 +22,16 @@ package transformation import ( + "errors" "fmt" "github.com/m3db/m3/src/metrics/generated/proto/transformationpb" ) // Type defines a transformation function. -type Type int +type Type int32 + +var errUnknownTransformationType = errors.New("unknown transformation type") // Supported transformation types. const ( @@ -40,6 +43,11 @@ const ( Reset ) +const ( + _minValidTransformationType = Absolute + _maxValidTransformationType = Reset +) + // IsValid checks if the transformation type is valid. func (t Type) IsValid() bool { return t.IsUnaryTransform() || t.IsBinaryTransform() || t.IsUnaryMultiOutputTransform() @@ -79,7 +87,7 @@ func (t Type) NewOp() (Op, error) { case t.IsUnaryMultiOutputTransform(): unaryMulti, err = t.UnaryMultiOutputTransform() default: - err = fmt.Errorf("unknown transformation type: %v", t) + err = errUnknownTransformationType } if err != nil { return Op{}, err @@ -154,38 +162,18 @@ func (t Type) MustUnaryMultiOutputTransform() UnaryMultiOutputTransform { // ToProto converts the transformation type to a protobuf message in place. func (t Type) ToProto(pb *transformationpb.TransformationType) error { - switch t { - case Absolute: - *pb = transformationpb.TransformationType_ABSOLUTE - case PerSecond: - *pb = transformationpb.TransformationType_PERSECOND - case Increase: - *pb = transformationpb.TransformationType_INCREASE - case Add: - *pb = transformationpb.TransformationType_ADD - case Reset: - *pb = transformationpb.TransformationType_RESET - default: - return fmt.Errorf("unknown transformation type: %v", t) + if t < _minValidTransformationType || t > _maxValidTransformationType { + return errUnknownTransformationType } + *pb = transformationpb.TransformationType(t) return nil } // FromProto converts the protobuf message to a transformation type in place. func (t *Type) FromProto(pb transformationpb.TransformationType) error { - switch pb { - case transformationpb.TransformationType_ABSOLUTE: - *t = Absolute - case transformationpb.TransformationType_PERSECOND: - *t = PerSecond - case transformationpb.TransformationType_INCREASE: - *t = Increase - case transformationpb.TransformationType_ADD: - *t = Add - case transformationpb.TransformationType_RESET: - *t = Reset - default: - return fmt.Errorf("unknown transformation type in proto: %v", pb) + *t = Type(pb) + if *t < _minValidTransformationType || *t > _maxValidTransformationType { + return errUnknownTransformationType } return nil } diff --git a/src/msg/consumer/config.go b/src/msg/consumer/config.go index 9ef1c99620..20babac82f 100644 --- a/src/msg/consumer/config.go +++ b/src/msg/consumer/config.go @@ -44,7 +44,7 @@ type Configuration struct { // options, which extends the default object pool configuration. type MessagePoolConfiguration struct { // Size is the size of the pool. - Size int `yaml:"size"` + Size pool.Size `yaml:"size"` // Watermark is the object pool watermark configuration. Watermark pool.WatermarkConfiguration `yaml:"watermark"` diff --git a/src/msg/producer/writer/message_writer.go b/src/msg/producer/writer/message_writer.go index 8e09b9a8b2..eaa96aba86 100644 --- a/src/msg/producer/writer/message_writer.go +++ b/src/msg/producer/writer/message_writer.go @@ -23,7 +23,7 @@ package writer import ( "container/list" "errors" - "math/rand" + "math" "sync" "time" @@ -32,6 +32,7 @@ import ( "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/retry" + "github.com/m3db/m3/src/x/unsafe" "github.com/uber-go/tally" ) @@ -41,6 +42,8 @@ var ( errNoWriters = errors.New("no writers") ) +const _recordMessageDelayEvery = 4 // keep it a power of two value to keep modulo fast + type messageWriter interface { // Write writes a message, messages not acknowledged in time will be retried. // New messages will be written in order, but retries could be out of order. @@ -96,8 +99,8 @@ type messageWriter interface { } type messageWriterMetrics struct { - scope tally.Scope - opts instrument.TimerOptions + scope tally.Scope + opts instrument.TimerOptions writeSuccess tally.Counter oneConsumerWriteError tally.Counter allConsumersWriteError tally.Counter @@ -139,10 +142,10 @@ func newMessageWriterMetricsWithConsumer( opts instrument.TimerOptions, consumer string, ) messageWriterMetrics { - consumerScope := scope.Tagged(map[string]string{"consumer" : consumer}) + consumerScope := scope.Tagged(map[string]string{"consumer": consumer}) return messageWriterMetrics{ - scope: scope, - opts: opts, + scope: scope, + opts: opts, writeSuccess: consumerScope.Counter("write-success"), oneConsumerWriteError: scope.Counter("write-error-one-consumer"), allConsumersWriteError: consumerScope. @@ -196,13 +199,12 @@ func newMessageWriterMetricsWithConsumer( type messageWriterImpl struct { sync.RWMutex - replicatedShardID uint64 - mPool messagePool - opts Options - retryOpts retry.Options - r *rand.Rand - encoder proto.Encoder - numConnections int + replicatedShardID uint64 + mPool messagePool + opts Options + nextRetryAfterNanos func(int) int64 + encoder proto.Encoder + numConnections int msgID uint64 queue *list.List @@ -217,9 +219,9 @@ type messageWriterImpl struct { doneCh chan struct{} wg sync.WaitGroup // metrics can be updated when a consumer instance changes, so must be guarded with RLock - m *messageWriterMetrics - nextFullScan time.Time - lastNewWrite *list.Element + m *messageWriterMetrics + nextFullScan time.Time + lastNewWrite *list.Element nowFn clock.NowFn } @@ -235,23 +237,22 @@ func newMessageWriter( } nowFn := time.Now return &messageWriterImpl{ - replicatedShardID: replicatedShardID, - mPool: mPool, - opts: opts, - retryOpts: opts.MessageRetryOptions(), - r: rand.New(rand.NewSource(nowFn().UnixNano())), - encoder: proto.NewEncoder(opts.EncoderOptions()), - numConnections: opts.ConnectionOptions().NumConnections(), - msgID: 0, - queue: list.New(), - acks: newAckHelper(opts.InitialAckMapSize()), - cutOffNanos: 0, - cutOverNanos: 0, - msgsToWrite: make([]*message, 0, opts.MessageQueueScanBatchSize()), - isClosed: false, - doneCh: make(chan struct{}), - m: &m, - nowFn: nowFn, + replicatedShardID: replicatedShardID, + mPool: mPool, + opts: opts, + nextRetryAfterNanos: nextRetryNanosFn(opts.MessageRetryOptions()), + encoder: proto.NewEncoder(opts.EncoderOptions()), + numConnections: opts.ConnectionOptions().NumConnections(), + msgID: 0, + queue: list.New(), + acks: newAckHelper(opts.InitialAckMapSize()), + cutOffNanos: 0, + cutOverNanos: 0, + msgsToWrite: make([]*message, 0, opts.MessageQueueScanBatchSize()), + isClosed: false, + doneCh: make(chan struct{}), + m: &m, + nowFn: nowFn, } } @@ -317,20 +318,27 @@ func (w *messageWriterImpl) write( } var ( // NB(r): Always select the same connection index per shard. - connIndex = int(w.replicatedShardID % uint64(w.numConnections)) - written = false + connIndex = int(w.replicatedShardID % uint64(w.numConnections)) + writes int64 + writeErrors int64 ) + for i := len(iterationIndexes) - 1; i >= 0; i-- { consumerWriter := consumerWriters[randIndex(iterationIndexes, i)] if err := consumerWriter.Write(connIndex, w.encoder.Bytes()); err != nil { - metrics.oneConsumerWriteError.Inc(1) + writeErrors++ continue } - written = true - metrics.writeSuccess.Inc(1) + writes++ break } - if written { + + if writeErrors > 0 { + metrics.oneConsumerWriteError.Inc(writeErrors) + } + + if writes > 0 { + metrics.writeSuccess.Inc(writes) return nil } // Could not be written to any consumer, will retry later. @@ -339,25 +347,13 @@ func (w *messageWriterImpl) write( } func randIndex(iterationIndexes []int, i int) int { - j := rand.Intn(i + 1) + j := int(unsafe.Fastrandn(uint32(i + 1))) // NB: we should only mutate the order in the iteration indexes and // keep the order of consumer writers unchanged to prevent data race. iterationIndexes[i], iterationIndexes[j] = iterationIndexes[j], iterationIndexes[i] return iterationIndexes[i] } -func (w *messageWriterImpl) nextRetryNanos(writeTimes int, nowNanos int64) int64 { - backoff := retry.BackoffNanos( - writeTimes, - w.retryOpts.Jitter(), - w.retryOpts.BackoffFactor(), - w.retryOpts.InitialBackoff(), - w.retryOpts.MaxBackoff(), - w.r.Int63n, - ) - return nowNanos + backoff -} - func (w *messageWriterImpl) Ack(meta metadata) bool { acked, initNanos := w.acks.ack(meta) if acked { @@ -381,7 +377,10 @@ func (w *messageWriterImpl) Init() { func (w *messageWriterImpl) scanMessageQueueUntilClose() { var ( interval = w.opts.MessageQueueNewWritesScanInterval() - jitter = time.Duration(w.r.Int63n(int64(interval))) + jitter = time.Duration( + // approx ~40 days of jitter at millisecond precision - more than enough + unsafe.Fastrandn(uint32(interval.Milliseconds())), + ) * time.Millisecond ) // NB(cw): Add some jitter before the tick starts to reduce // some contention between all the message writers. @@ -407,30 +406,32 @@ func (w *messageWriterImpl) scanMessageQueue() { m := w.m w.RUnlock() var ( + nowFn = w.nowFn msgsToWrite []*message - beforeScan = w.nowFn() + beforeScan = nowFn() + beforeBatchNanos = beforeScan.UnixNano() batchSize = w.opts.MessageQueueScanBatchSize() consumerWriters []consumerWriter iterationIndexes []int fullScan = isClosed || beforeScan.After(w.nextFullScan) + scanMetrics scanBatchMetrics skipWrites bool ) + defer scanMetrics.record(m) for e != nil { - beforeBatch := w.nowFn() - beforeBatchNanos := beforeBatch.UnixNano() w.Lock() - e, msgsToWrite = w.scanBatchWithLock(e, beforeBatchNanos, batchSize, fullScan) + e, msgsToWrite = w.scanBatchWithLock(e, beforeBatchNanos, batchSize, fullScan, &scanMetrics) consumerWriters = w.consumerWriters iterationIndexes = w.iterationIndexes w.Unlock() if !fullScan && len(msgsToWrite) == 0 { - m.scanBatchLatency.Record(w.nowFn().Sub(beforeBatch)) + m.scanBatchLatency.Record(time.Duration(nowFn().UnixNano() - beforeBatchNanos)) // If this is not a full scan, abort after the iteration batch // that no new messages were found. break } if skipWrites { - m.scanBatchLatency.Record(w.nowFn().Sub(beforeBatch)) + m.scanBatchLatency.Record(time.Duration(nowFn().UnixNano() - beforeBatchNanos)) continue } if err := w.writeBatch(iterationIndexes, consumerWriters, m, msgsToWrite); err != nil { @@ -438,9 +439,11 @@ func (w *messageWriterImpl) scanMessageQueue() { // to avoid meaningless attempts but continue to clean up the queue. skipWrites = true } - m.scanBatchLatency.Record(w.nowFn().Sub(beforeBatch)) + nowNanos := nowFn().UnixNano() + m.scanBatchLatency.Record(time.Duration(nowNanos - beforeBatchNanos)) + beforeBatchNanos = nowNanos } - afterScan := w.nowFn() + afterScan := nowFn() m.scanTotalLatency.Record(afterScan.Sub(beforeScan)) if fullScan { w.nextFullScan = afterScan.Add(w.opts.MessageQueueFullScanInterval()) @@ -458,11 +461,15 @@ func (w *messageWriterImpl) writeBatch( metrics.noWritersError.Inc(int64(len(messages))) return errNoWriters } - for _, m := range messages { - if err := w.write(iterationIndexes, consumerWriters, metrics, m); err != nil { + delay := metrics.messageWriteDelay + nowFn := w.nowFn + for i := range messages { + if err := w.write(iterationIndexes, consumerWriters, metrics, messages[i]); err != nil { return err } - metrics.messageWriteDelay.Record(time.Duration(w.nowFn().UnixNano() - m.InitNanos())) + if i%_recordMessageDelayEvery == 0 { + delay.Record(time.Duration(nowFn().UnixNano() - messages[i].InitNanos())) + } } return nil } @@ -475,6 +482,7 @@ func (w *messageWriterImpl) scanBatchWithLock( nowNanos int64, batchSize int, fullScan bool, + scanMetrics *scanBatchMetrics, ) (*list.Element, []*message) { var ( iterated int @@ -489,7 +497,7 @@ func (w *messageWriterImpl) scanBatchWithLock( next = e.Next() m := e.Value.(*message) if w.isClosed { - w.m.processedClosed.Inc(1) + scanMetrics[_processedClosed]++ // Simply ack the messages here to mark them as consumed for this // message writer, this is useful when user removes a consumer service // during runtime that may be unhealthy to consume the messages. @@ -498,11 +506,11 @@ func (w *messageWriterImpl) scanBatchWithLock( // NB: The message must be added to the ack map to be acked here. w.acks.ack(m.Metadata()) w.removeFromQueueWithLock(e, m) - w.m.messageClosed.Inc(1) + scanMetrics[_messageClosed]++ continue } if m.RetryAtNanos() >= nowNanos { - w.m.processedNotReady.Inc(1) + scanMetrics[_processedNotReady]++ if !fullScan { // If this is not a full scan, bail after the first element that // is not a new write. @@ -513,22 +521,22 @@ func (w *messageWriterImpl) scanBatchWithLock( // If the message exceeded its allowed ttl of the consumer service, // remove it from the buffer. if w.messageTTLNanos > 0 && m.InitNanos()+w.messageTTLNanos <= nowNanos { - w.m.processedTTL.Inc(1) + scanMetrics[_processedTTL]++ // There is a chance the message was acked right before the ack is // called, in which case just remove it from the queue. if acked, _ := w.acks.ack(m.Metadata()); acked { - w.m.messageDroppedTTLExpire.Inc(1) + scanMetrics[_messageDroppedTTLExpire]++ } w.removeFromQueueWithLock(e, m) continue } if m.IsAcked() { - w.m.processedAck.Inc(1) + scanMetrics[_processedAck]++ w.removeFromQueueWithLock(e, m) continue } if m.IsDroppedOrConsumed() { - w.m.processedDrop.Inc(1) + scanMetrics[_processedDrop]++ // There is a chance the message could be acked between m.Acked() // and m.IsDroppedOrConsumed() check, in which case we should not // mark it as dropped, just continue and next tick will remove it @@ -538,16 +546,16 @@ func (w *messageWriterImpl) scanBatchWithLock( } w.acks.remove(m.Metadata()) w.removeFromQueueWithLock(e, m) - w.m.messageDroppedBufferFull.Inc(1) + scanMetrics[_messageDroppedBufferFull]++ continue } m.IncWriteTimes() writeTimes := m.WriteTimes() - m.SetRetryAtNanos(w.nextRetryNanos(writeTimes, nowNanos)) + m.SetRetryAtNanos(w.nextRetryAfterNanos(writeTimes) + nowNanos) if writeTimes > 1 { - w.m.messageRetry.Inc(1) + scanMetrics[_messageRetry]++ } - w.m.processedWrite.Inc(1) + scanMetrics[_processedWrite]++ w.msgsToWrite = append(w.msgsToWrite, m) } return next, w.msgsToWrite @@ -747,3 +755,75 @@ func (a *acks) size() int { a.Unlock() return l } + +type metricIdx byte + +const ( + _messageClosed metricIdx = iota + _messageDroppedBufferFull + _messageDroppedTTLExpire + _messageRetry + _processedAck + _processedClosed + _processedDrop + _processedNotReady + _processedTTL + _processedWrite + _lastMetricIdx +) + +type scanBatchMetrics [_lastMetricIdx]int32 + +func (m *scanBatchMetrics) record(metrics *messageWriterMetrics) { + m.recordNonzeroCounter(_messageClosed, metrics.messageClosed) + m.recordNonzeroCounter(_messageDroppedBufferFull, metrics.messageDroppedBufferFull) + m.recordNonzeroCounter(_messageDroppedTTLExpire, metrics.messageDroppedTTLExpire) + m.recordNonzeroCounter(_messageRetry, metrics.messageRetry) + m.recordNonzeroCounter(_processedAck, metrics.processedAck) + m.recordNonzeroCounter(_processedClosed, metrics.processedClosed) + m.recordNonzeroCounter(_processedDrop, metrics.processedDrop) + m.recordNonzeroCounter(_processedNotReady, metrics.processedNotReady) + m.recordNonzeroCounter(_processedTTL, metrics.processedTTL) + m.recordNonzeroCounter(_processedWrite, metrics.processedWrite) +} + +func (m *scanBatchMetrics) recordNonzeroCounter(idx metricIdx, c tally.Counter) { + if m[idx] > 0 { + c.Inc(int64(m[idx])) + } +} + +func nextRetryNanosFn(retryOpts retry.Options) func(int) int64 { + var ( + jitter = retryOpts.Jitter() + backoffFactor = retryOpts.BackoffFactor() + initialBackoff = retryOpts.InitialBackoff() + maxBackoff = retryOpts.MaxBackoff() + initialBackoffFloat = float64(initialBackoff) + ) + + // inlined and specialized retry function that does not have any state that needs to be kept + // between tries + return func(writeTimes int) int64 { + backoff := initialBackoff.Nanoseconds() + if writeTimes >= 1 { + backoffFloat64 := initialBackoffFloat * math.Pow(backoffFactor, float64(writeTimes-1)) + backoff = int64(backoffFloat64) + } + // Validate the value of backoff to make sure Fastrandn() does not panic and + // check for overflow from the exponentiation op - unlikely, but prevents weird side effects. + halfInMicros := (backoff / 2) / int64(time.Microsecond) + if jitter && backoff >= 2 && halfInMicros < math.MaxUint32 { + // Jitter can be only up to ~1 hour in microseconds, but it's not a limitation here + jitterInMicros := unsafe.Fastrandn(uint32(halfInMicros)) + jitterInNanos := time.Duration(jitterInMicros) * time.Microsecond + halfInNanos := time.Duration(halfInMicros) * time.Microsecond + backoff = int64(halfInNanos + jitterInNanos) + } + // Clamp backoff to maxBackoff + if maxBackoff := maxBackoff.Nanoseconds(); backoff > maxBackoff { + backoff = maxBackoff + } + return backoff + } +} diff --git a/src/msg/producer/writer/message_writer_benchmark_test.go b/src/msg/producer/writer/message_writer_benchmark_test.go new file mode 100644 index 0000000000..3ff371e935 --- /dev/null +++ b/src/msg/producer/writer/message_writer_benchmark_test.go @@ -0,0 +1,71 @@ +// Copyright (c) 2021 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 writer + +import ( + "testing" + + "github.com/m3db/m3/src/msg/producer" + "github.com/m3db/m3/src/x/instrument" +) + +func BenchmarkScanMessageQueue(b *testing.B) { + opts := testOptions(). + SetMessageQueueScanBatchSize(128). + SetInstrumentOptions( + instrument.NewOptions().SetTimerOptions( + instrument.TimerOptions{ + Type: instrument.StandardTimerType, + StandardSampleRate: 0.05, + }, + ), + ) + + b.RunParallel(func(pb *testing.PB) { + w := newMessageWriter( + 200, + testMessagePool(opts), + opts, + testMessageWriterMetrics(), + ).(*messageWriterImpl) + + w.consumerWriters = append(w.consumerWriters, noopWriter{}) + + for i := 0; i < 1024; i++ { + w.Write(producer.NewRefCountedMessage(emptyMessage{}, nil)) + } + b.ResetTimer() + + for pb.Next() { + w.scanMessageQueue() + if w.QueueSize() != 1024 { + b.Fatalf("expected queue len to be 1024, got %v", w.QueueSize()) + } + } + }) +} + +type noopWriter struct{} + +func (noopWriter) Address() string { return "" } +func (noopWriter) Write(int, []byte) error { return nil } +func (noopWriter) Init() {} +func (noopWriter) Close() {} diff --git a/src/msg/producer/writer/message_writer_test.go b/src/msg/producer/writer/message_writer_test.go index 05d62f2cda..fbfc494c93 100644 --- a/src/msg/producer/writer/message_writer_test.go +++ b/src/msg/producer/writer/message_writer_test.go @@ -529,7 +529,7 @@ func TestMessageWriterKeepNewWritesInOrderInFrontOfTheQueue(t *testing.T) { mm2.EXPECT().Bytes().Return([]byte("2")).AnyTimes() w.Write(rm2) validateMessages(t, []*producer.RefCountedMessage{rm1, rm2}, w) - w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), 2, true) + w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), 2, true, &scanBatchMetrics{}) w.lastNewWrite = nil mm3 := producer.NewMockMessage(ctrl) @@ -589,7 +589,7 @@ func TestMessageWriterRetryIterateBatchFullScan(t *testing.T) { mm1.EXPECT().Finalize(gomock.Eq(producer.Dropped)) rm1.Drop() require.Equal(t, 4, w.queue.Len()) - e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true) + e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true, &scanBatchMetrics{}) require.Equal(t, 1, len(toBeRetried)) require.Equal(t, 3, w.queue.Len()) @@ -597,21 +597,22 @@ func TestMessageWriterRetryIterateBatchFullScan(t *testing.T) { require.Equal(t, rm3, e.Value.(*message).RefCountedMessage) require.Equal(t, 3, w.queue.Len()) - e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano(), retryBatchSize, true) + e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano(), retryBatchSize, true, &scanBatchMetrics{}) require.Nil(t, e) require.Equal(t, 2, len(toBeRetried)) require.Equal(t, 3, w.queue.Len()) - e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true) + e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true, &scanBatchMetrics{}) // Make sure it stopped at rm4. require.Equal(t, rm4, e.Value.(*message).RefCountedMessage) require.Equal(t, 0, len(toBeRetried)) - e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano(), retryBatchSize, true) + e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano(), retryBatchSize, true, &scanBatchMetrics{}) require.Nil(t, e) require.Equal(t, 0, len(toBeRetried)) } +//nolint:lll func TestMessageWriterRetryIterateBatchFullScanWithMessageTTL(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -652,7 +653,7 @@ func TestMessageWriterRetryIterateBatchFullScanWithMessageTTL(t *testing.T) { mm1.EXPECT().Finalize(gomock.Eq(producer.Dropped)) rm1.Drop() require.Equal(t, 4, w.queue.Len()) - e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true) + e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, true, &scanBatchMetrics{}) require.Equal(t, 1, len(toBeRetried)) require.Equal(t, 3, w.queue.Len()) @@ -661,18 +662,19 @@ func TestMessageWriterRetryIterateBatchFullScanWithMessageTTL(t *testing.T) { w.SetMessageTTLNanos(int64(time.Minute)) mm4.EXPECT().Finalize(gomock.Eq(producer.Consumed)) mm3.EXPECT().Finalize(gomock.Eq(producer.Consumed)) - e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano()+int64(time.Hour), retryBatchSize, true) + e, toBeRetried = w.scanBatchWithLock(e, w.nowFn().UnixNano()+int64(time.Hour), retryBatchSize, true, &scanBatchMetrics{}) require.Equal(t, 0, len(toBeRetried)) require.Equal(t, 1, w.queue.Len()) require.Nil(t, e) mm2.EXPECT().Finalize(gomock.Eq(producer.Consumed)) - e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano()+int64(time.Hour), retryBatchSize, true) + e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano()+int64(time.Hour), retryBatchSize, true, &scanBatchMetrics{}) require.Equal(t, 0, len(toBeRetried)) require.Equal(t, 0, w.queue.Len()) require.Nil(t, e) } +//nolint:lll func TestMessageWriterRetryIterateBatchNotFullScan(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -713,7 +715,7 @@ func TestMessageWriterRetryIterateBatchNotFullScan(t *testing.T) { mm1.EXPECT().Finalize(gomock.Eq(producer.Dropped)) rm1.Drop() require.Equal(t, 4, w.queue.Len()) - e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false) + e, toBeRetried := w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false, &scanBatchMetrics{}) require.Equal(t, 3, len(toBeRetried)) require.Equal(t, 3, w.queue.Len()) require.Nil(t, e) @@ -723,7 +725,7 @@ func TestMessageWriterRetryIterateBatchNotFullScan(t *testing.T) { mm4.EXPECT().Finalize(gomock.Eq(producer.Dropped)) rm4.Drop() require.Equal(t, 3, w.queue.Len()) - e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false) + e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false, &scanBatchMetrics{}) require.Equal(t, rm2, e.Value.(*message).RefCountedMessage) require.Equal(t, 0, len(toBeRetried)) require.Equal(t, 3, w.queue.Len()) @@ -739,14 +741,14 @@ func TestMessageWriterRetryIterateBatchNotFullScan(t *testing.T) { validateMessages(t, []*producer.RefCountedMessage{rm5, rm2, rm3, rm4}, w) require.Equal(t, 4, w.queue.Len()) - e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false) + e, toBeRetried = w.scanBatchWithLock(w.queue.Front(), w.nowFn().UnixNano(), retryBatchSize, false, &scanBatchMetrics{}) require.Equal(t, rm2, e.Value.(*message).RefCountedMessage) require.Equal(t, 1, len(toBeRetried)) require.Equal(t, rm5, toBeRetried[0].RefCountedMessage) require.Equal(t, 4, w.queue.Len()) } -func TestNextRetryNanos(t *testing.T) { +func TestNextRetryAfterNanos(t *testing.T) { backoffDuration := time.Minute opts := testOptions().SetMessageRetryOptions( retry.NewOptions().SetInitialBackoff(backoffDuration).SetMaxBackoff(2 * backoffDuration).SetJitter(true), @@ -756,17 +758,17 @@ func TestNextRetryNanos(t *testing.T) { nowNanos := time.Now().UnixNano() m := newMessage() m.IncWriteTimes() - retryAtNanos := w.nextRetryNanos(m.WriteTimes(), nowNanos) + retryAtNanos := w.nextRetryAfterNanos(m.WriteTimes()) + nowNanos require.True(t, retryAtNanos > nowNanos) require.True(t, retryAtNanos < nowNanos+int64(backoffDuration)) m.IncWriteTimes() - retryAtNanos = w.nextRetryNanos(m.WriteTimes(), nowNanos) + retryAtNanos = w.nextRetryAfterNanos(m.WriteTimes()) + nowNanos require.True(t, retryAtNanos >= nowNanos+int64(backoffDuration)) require.True(t, retryAtNanos < nowNanos+2*int64(backoffDuration)) m.IncWriteTimes() - retryAtNanos = w.nextRetryNanos(m.WriteTimes(), nowNanos) + retryAtNanos = w.nextRetryAfterNanos(m.WriteTimes()) + nowNanos require.True(t, retryAtNanos == nowNanos+2*int64(backoffDuration)) } diff --git a/src/msg/protocol/proto/roundtrip_test.go b/src/msg/protocol/proto/roundtrip_test.go index de5202c6c8..cdb591b410 100644 --- a/src/msg/protocol/proto/roundtrip_test.go +++ b/src/msg/protocol/proto/roundtrip_test.go @@ -197,7 +197,7 @@ func getBytesPool(bucketSizes int, bucketCaps []int) pool.BytesPool { buckets := make([]pool.Bucket, len(bucketCaps)) for i, cap := range bucketCaps { buckets[i] = pool.Bucket{ - Count: bucketSizes, + Count: pool.Size(bucketSizes), Capacity: cap, } } diff --git a/src/query/api/v1/handler/ready.go b/src/query/api/v1/handler/ready.go index 887f24c62d..5870eb1cc8 100644 --- a/src/query/api/v1/handler/ready.go +++ b/src/query/api/v1/handler/ready.go @@ -90,10 +90,12 @@ func (h *ReadyHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - var ( + var namespaces m3.ClusterNamespaces + if h.clusters != nil { namespaces = h.clusters.ClusterNamespaces() - result = &readyResult{} - ) + } + + result := &readyResult{} for _, ns := range namespaces { attrs := ns.Options().Attributes() nsResult := readyResultNamespace{ diff --git a/src/query/api/v1/handler/ready_test.go b/src/query/api/v1/handler/ready_test.go index b616a78ae0..fcd94f30e3 100644 --- a/src/query/api/v1/handler/ready_test.go +++ b/src/query/api/v1/handler/ready_test.go @@ -211,3 +211,28 @@ func TestReadyHandler(t *testing.T) { }) } } + +func TestReadyHandlerNoClusters(t *testing.T) { + ctrl := xtest.NewController(t) + defer ctrl.Finish() + + opts := options.EmptyHandlerOptions() + readyHandler := NewReadyHandler(opts) + + w := httptest.NewRecorder() + url := ReadyURL + req := httptest.NewRequest(ReadyHTTPMethod, url, nil) + + readyHandler.ServeHTTP(w, req) + + resp := w.Result() + defer resp.Body.Close() + body, err := ioutil.ReadAll(resp.Body) + assert.NoError(t, err) + assert.Equal(t, http.StatusOK, resp.StatusCode) + + expected := "{}" + actual := xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, xtest.Diff(expected, actual)) +} diff --git a/src/query/generated/mocks/generate.go b/src/query/generated/mocks/generate.go index 199b0eac0b..607d57c1e1 100644 --- a/src/query/generated/mocks/generate.go +++ b/src/query/generated/mocks/generate.go @@ -21,7 +21,7 @@ // mockgen rules for generating mocks for exported interfaces (reflection mode). //go:generate sh -c "mockgen -package=downsample $PACKAGE/src/cmd/services/m3coordinator/downsample Downsampler,MetricsAppender,SamplesAppender | genclean -pkg $PACKAGE/src/cmd/services/m3coordinator/downsample -out $GOPATH/src/$PACKAGE/src/cmd/services/m3coordinator/downsample/downsample_mock.go" //go:generate sh -c "mockgen -package=storage -destination=$GOPATH/src/$PACKAGE/src/query/storage/storage_mock.go $PACKAGE/src/query/storage Storage" -//go:generate sh -c "mockgen -package=m3 -destination=$GOPATH/src/$PACKAGE/src/query/storage/m3/m3_mock.go $PACKAGE/src/query/storage/m3 Storage" +//go:generate sh -c "mockgen -package=m3 -destination=$GOPATH/src/$PACKAGE/src/query/storage/m3/m3_mock.go $PACKAGE/src/query/storage/m3 Storage,ClusterNamespace" //go:generate sh -c "mockgen -package=ts -destination=$GOPATH/src/$PACKAGE/src/query/ts/ts_mock.go $PACKAGE/src/query/ts Values" //go:generate sh -c "mockgen -package=block -destination=$GOPATH/src/$PACKAGE/src/query/block/block_mock.go $PACKAGE/src/query/block Block,StepIter,Builder,Step,SeriesIter" //go:generate sh -c "mockgen -package=ingest -destination=$GOPATH/src/$PACKAGE/src/cmd/services/m3coordinator/ingest/write_mock.go $PACKAGE/src/cmd/services/m3coordinator/ingest DownsamplerAndWriter" diff --git a/src/query/storage/m3/cluster.go b/src/query/storage/m3/cluster.go index 7818188926..2a71f78d49 100644 --- a/src/query/storage/m3/cluster.go +++ b/src/query/storage/m3/cluster.go @@ -100,6 +100,17 @@ type ClusterNamespaceOptions struct { downsample *ClusterNamespaceDownsampleOptions } +// NewClusterNamespaceOptions creates new cluster namespace options. +func NewClusterNamespaceOptions( + attributes storagemetadata.Attributes, + downsample *ClusterNamespaceDownsampleOptions, +) ClusterNamespaceOptions { + return ClusterNamespaceOptions{ + attributes: attributes, + downsample: downsample, + } +} + // Attributes returns the storage attributes of the cluster namespace. func (o ClusterNamespaceOptions) Attributes() storagemetadata.Attributes { return o.attributes diff --git a/src/query/storage/m3/m3_mock.go b/src/query/storage/m3/m3_mock.go index 6ef74722b0..8030d39120 100644 --- a/src/query/storage/m3/m3_mock.go +++ b/src/query/storage/m3/m3_mock.go @@ -1,7 +1,7 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/query/storage/m3 (interfaces: Storage) +// Source: github.com/m3db/m3/src/query/storage/m3 (interfaces: Storage,ClusterNamespace) -// Copyright (c) 2020 Uber Technologies, Inc. +// Copyright (c) 2021 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 @@ -28,9 +28,11 @@ import ( "context" "reflect" + "github.com/m3db/m3/src/dbnode/client" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/storage/m3/consolidators" + "github.com/m3db/m3/src/x/ident" "github.com/golang/mock/gomock" ) @@ -234,3 +236,68 @@ func (mr *MockStorageMockRecorder) Write(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MockStorage)(nil).Write), arg0, arg1) } + +// MockClusterNamespace is a mock of ClusterNamespace interface +type MockClusterNamespace struct { + ctrl *gomock.Controller + recorder *MockClusterNamespaceMockRecorder +} + +// MockClusterNamespaceMockRecorder is the mock recorder for MockClusterNamespace +type MockClusterNamespaceMockRecorder struct { + mock *MockClusterNamespace +} + +// NewMockClusterNamespace creates a new mock instance +func NewMockClusterNamespace(ctrl *gomock.Controller) *MockClusterNamespace { + mock := &MockClusterNamespace{ctrl: ctrl} + mock.recorder = &MockClusterNamespaceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockClusterNamespace) EXPECT() *MockClusterNamespaceMockRecorder { + return m.recorder +} + +// NamespaceID mocks base method +func (m *MockClusterNamespace) NamespaceID() ident.ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NamespaceID") + ret0, _ := ret[0].(ident.ID) + return ret0 +} + +// NamespaceID indicates an expected call of NamespaceID +func (mr *MockClusterNamespaceMockRecorder) NamespaceID() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NamespaceID", reflect.TypeOf((*MockClusterNamespace)(nil).NamespaceID)) +} + +// Options mocks base method +func (m *MockClusterNamespace) Options() ClusterNamespaceOptions { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Options") + ret0, _ := ret[0].(ClusterNamespaceOptions) + return ret0 +} + +// Options indicates an expected call of Options +func (mr *MockClusterNamespaceMockRecorder) Options() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Options", reflect.TypeOf((*MockClusterNamespace)(nil).Options)) +} + +// Session mocks base method +func (m *MockClusterNamespace) Session() client.Session { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Session") + ret0, _ := ret[0].(client.Session) + return ret0 +} + +// Session indicates an expected call of Session +func (mr *MockClusterNamespaceMockRecorder) Session() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Session", reflect.TypeOf((*MockClusterNamespace)(nil).Session)) +} diff --git a/src/query/storage/m3/storage_test.go b/src/query/storage/m3/storage_test.go index b7c2af81d9..ac930ed1fc 100644 --- a/src/query/storage/m3/storage_test.go +++ b/src/query/storage/m3/storage_test.go @@ -851,6 +851,6 @@ func TestInvalidBlockTypes(t *testing.T) { query := &storage.FetchQuery{} fetchOpts := &storage.FetchOptions{BlockType: models.TypeMultiBlock} - _, err = s.FetchBlocks(context.TODO(), query, fetchOpts) - assert.Error(t, err) + defer instrument.SetShouldPanicEnvironmentVariable(true)() + require.Panics(t, func() { _, _ = s.FetchBlocks(context.TODO(), query, fetchOpts) }) } diff --git a/src/query/ts/m3db/options.go b/src/query/ts/m3db/options.go index 153df2aac7..03606790cd 100644 --- a/src/query/ts/m3db/options.go +++ b/src/query/ts/m3db/options.go @@ -70,7 +70,7 @@ type encodedBlockOptions struct { func NewOptions() Options { bytesPool := pool.NewCheckedBytesPool([]pool.Bucket{{ Capacity: defaultCapacity, - Count: defaultCount, + Count: pool.Size(defaultCount), }}, nil, func(s []pool.Bucket) pool.BytesPool { return pool.NewBytesPool(s, nil) }) diff --git a/src/x/instrument/extended.go b/src/x/instrument/extended.go index 2c6d435801..560da5d1f3 100644 --- a/src/x/instrument/extended.go +++ b/src/x/instrument/extended.go @@ -63,6 +63,10 @@ const ( // - GC pause times DetailedExtendedMetrics + // DetailedGoRuntimeMetrics reports all detailed metrics, sans FD metrics to save CPU + // if in-use file descriptors are measured by an external system, like cAdvisor. + DetailedGoRuntimeMetrics + // DefaultExtendedMetricsType is the default extended metrics level. DefaultExtendedMetricsType = SimpleExtendedMetrics ) @@ -73,6 +77,7 @@ var ( SimpleExtendedMetrics, ModerateExtendedMetrics, DetailedExtendedMetrics, + DetailedGoRuntimeMetrics, } ) @@ -86,6 +91,8 @@ func (t ExtendedMetricsType) String() string { return "moderate" case DetailedExtendedMetrics: return "detailed" + case DetailedGoRuntimeMetrics: + return "runtime" } return "unknown" } @@ -191,20 +198,35 @@ func NewExtendedMetricsReporter( reportInterval time.Duration, metricsType ExtendedMetricsType, ) Reporter { - r := new(extendedMetricsReporter) + var ( + r = new(extendedMetricsReporter) + enableProcessReporter bool + ) + r.metricsType = metricsType r.init(reportInterval, func() { r.runtime.report(r.metricsType) }) - if r.metricsType >= ModerateExtendedMetrics { + + if r.metricsType == NoExtendedMetrics { + return r + } + + switch r.metricsType { + case ModerateExtendedMetrics: + enableProcessReporter = true + case DetailedExtendedMetrics: + enableProcessReporter = true + default: + enableProcessReporter = false + } + + if enableProcessReporter { // ProcessReporter can be quite slow in some situations (specifically // counting FDs for processes that have many of them) so it runs on // its own report loop. r.processReporter = NewProcessReporter(scope, reportInterval) } - if r.metricsType == NoExtendedMetrics { - return r - } runtimeScope := scope.SubScope("runtime") r.runtime.NumGoRoutines = runtimeScope.Gauge("num-goroutines") diff --git a/src/x/instrument/invariant.go b/src/x/instrument/invariant.go index baae4be76a..fde5a8a18a 100644 --- a/src/x/instrument/invariant.go +++ b/src/x/instrument/invariant.go @@ -23,6 +23,7 @@ package instrument import ( "fmt" "os" + "strconv" "strings" "go.uber.org/zap" @@ -88,6 +89,16 @@ func InvariantErrorf(format string, a ...interface{}) error { return err } +// SetShouldPanicEnvironmentVariable sets the env variable and returns a func to reset to the previous value. +// Useful for tests to use a defer statement when they need to test a specific value. +func SetShouldPanicEnvironmentVariable(value bool) func() { + restoreValue := os.Getenv(ShouldPanicEnvironmentVariableName) + _ = os.Setenv(ShouldPanicEnvironmentVariableName, strconv.FormatBool(value)) + return func() { + _ = os.Setenv(ShouldPanicEnvironmentVariableName, restoreValue) + } +} + func panicIfEnvSet() { panicIfEnvSetWithMessage("") } diff --git a/src/x/instrument/invariant_test.go b/src/x/instrument/invariant_test.go index 6fef670d21..e2c51180e4 100644 --- a/src/x/instrument/invariant_test.go +++ b/src/x/instrument/invariant_test.go @@ -22,7 +22,6 @@ package instrument_test import ( "fmt" - "os" "testing" "github.com/m3db/m3/src/x/instrument" @@ -33,6 +32,7 @@ import ( ) func ExampleInvariantViolatedMetricInvocation() { + defer instrument.SetShouldPanicEnvironmentVariable(false)() testScope := tally.NewTestScope("", nil) opts := instrument.NewOptions().SetMetricsScope(testScope) @@ -44,6 +44,7 @@ func ExampleInvariantViolatedMetricInvocation() { } func TestEmitInvariantViolationDoesNotPanicIfEnvNotSet(t *testing.T) { + defer instrument.SetShouldPanicEnvironmentVariable(false)() opts := instrument.NewOptions() require.NotPanics(t, func() { instrument.EmitInvariantViolation(opts) @@ -51,6 +52,7 @@ func TestEmitInvariantViolationDoesNotPanicIfEnvNotSet(t *testing.T) { } func TestEmitAndLogInvariantViolationDoesNotPanicIfEnvNotSet(t *testing.T) { + defer instrument.SetShouldPanicEnvironmentVariable(false)() opts := instrument.NewOptions() require.NotPanics(t, func() { instrument.EmitAndLogInvariantViolation(opts, func(l *zap.Logger) { @@ -60,6 +62,7 @@ func TestEmitAndLogInvariantViolationDoesNotPanicIfEnvNotSet(t *testing.T) { } func TestErrorfDoesNotPanicIfEnvNotSet(t *testing.T) { + defer instrument.SetShouldPanicEnvironmentVariable(false)() var ( format = "some error format: %s" err_msg = "error message" @@ -72,7 +75,7 @@ func TestErrorfDoesNotPanicIfEnvNotSet(t *testing.T) { } func TestEmitInvariantViolationPanicsIfEnvSet(t *testing.T) { - defer setShouldPanicEnvironmentVariable()() + defer instrument.SetShouldPanicEnvironmentVariable(true)() opts := instrument.NewOptions() require.Panics(t, func() { @@ -81,7 +84,7 @@ func TestEmitInvariantViolationPanicsIfEnvSet(t *testing.T) { } func TestEmitAndLogInvariantViolationPanicsIfEnvSet(t *testing.T) { - defer setShouldPanicEnvironmentVariable()() + defer instrument.SetShouldPanicEnvironmentVariable(true)() opts := instrument.NewOptions() require.Panics(t, func() { @@ -92,18 +95,8 @@ func TestEmitAndLogInvariantViolationPanicsIfEnvSet(t *testing.T) { } func TestErrorfPanicsIfEnvSet(t *testing.T) { - defer setShouldPanicEnvironmentVariable()() + defer instrument.SetShouldPanicEnvironmentVariable(true)() require.Panics(t, func() { instrument.InvariantErrorf("some_error") }) } - -type cleanupFn func() - -func setShouldPanicEnvironmentVariable() cleanupFn { - restoreValue := os.Getenv(instrument.ShouldPanicEnvironmentVariableName) - os.Setenv(instrument.ShouldPanicEnvironmentVariableName, "true") - return cleanupFn(func() { - os.Setenv(instrument.ShouldPanicEnvironmentVariableName, restoreValue) - }) -} diff --git a/src/x/pool/bucketized.go b/src/x/pool/bucketized.go index a564302cc4..84c575bd4b 100644 --- a/src/x/pool/bucketized.go +++ b/src/x/pool/bucketized.go @@ -21,8 +21,8 @@ package pool import ( - "fmt" "sort" + "strconv" "github.com/uber-go/tally" ) @@ -77,13 +77,21 @@ func (p *bucketizedObjectPool) Init(alloc BucketizedAllocator) { opts = perBucketOpts } - opts = opts.SetSize(size) + if size > 0 { + opts = opts.SetSize(int(size)) + } + opts = opts.SetDynamic(size.IsDynamic()) + iopts := opts.InstrumentOptions() if iopts.MetricsScope() != nil { + sz := strconv.Itoa(capacity) + if capacity <= 0 { + sz = "dynamic" + } opts = opts.SetInstrumentOptions(iopts.SetMetricsScope( iopts.MetricsScope().Tagged(map[string]string{ - "bucket-capacity": fmt.Sprintf("%d", capacity), + "bucket-capacity": sz, }))) } diff --git a/src/x/pool/bytes_test.go b/src/x/pool/bytes_test.go index 39a3c0c620..f151b039ac 100644 --- a/src/x/pool/bytes_test.go +++ b/src/x/pool/bytes_test.go @@ -89,7 +89,7 @@ func getBytesPool(bucketSizes int, bucketCaps []int) *bytesPool { buckets := make([]Bucket, len(bucketCaps)) for i, cap := range bucketCaps { buckets[i] = Bucket{ - Count: bucketSizes, + Count: Size(bucketSizes), Capacity: cap, } } diff --git a/src/x/pool/checked_bytes_test.go b/src/x/pool/checked_bytes_test.go index 7cecaf8158..dabb1f0a91 100644 --- a/src/x/pool/checked_bytes_test.go +++ b/src/x/pool/checked_bytes_test.go @@ -109,7 +109,7 @@ func getCheckedBytesPool( buckets := make([]Bucket, len(bucketCaps)) for i, cap := range bucketCaps { buckets[i] = Bucket{ - Count: bucketSizes, + Count: Size(bucketSizes), Capacity: cap, } } diff --git a/src/x/pool/config.go b/src/x/pool/config.go index 6e08286e98..fa967d7b41 100644 --- a/src/x/pool/config.go +++ b/src/x/pool/config.go @@ -20,12 +20,17 @@ package pool -import "github.com/m3db/m3/src/x/instrument" +import ( + "errors" + "strconv" + + "github.com/m3db/m3/src/x/instrument" +) // ObjectPoolConfiguration contains configuration for object pools. type ObjectPoolConfiguration struct { // The size of the pool. - Size int `yaml:"size"` + Size Size `yaml:"size"` // The watermark configuration. Watermark WatermarkConfiguration `yaml:"watermark"` @@ -35,13 +40,15 @@ type ObjectPoolConfiguration struct { func (c *ObjectPoolConfiguration) NewObjectPoolOptions( instrumentOpts instrument.Options, ) ObjectPoolOptions { - size := defaultSize - if c.Size != 0 { - size = c.Size + size := _defaultSize + if c.Size > 0 { + size = int(c.Size) } + return NewObjectPoolOptions(). SetInstrumentOptions(instrumentOpts). SetSize(size). + SetDynamic(c.Size.IsDynamic()). SetRefillLowWatermark(c.Watermark.RefillLowWatermark). SetRefillHighWatermark(c.Watermark.RefillHighWatermark) } @@ -78,7 +85,7 @@ func (c *BucketizedPoolConfiguration) NewBuckets() []Bucket { // BucketConfiguration contains configuration for a pool bucket. type BucketConfiguration struct { // The count of the items in the bucket. - Count int `yaml:"count"` + Count Size `yaml:"count"` // The capacity of each item in the bucket. Capacity int `yaml:"capacity"` @@ -100,3 +107,32 @@ type WatermarkConfiguration struct { // The high watermark to stop refilling the pool, if zero none. RefillHighWatermark float64 `yaml:"high" validate:"min=0.0,max=1.0"` } + +// Size stores pool capacity for pools that can be either dynamic or pre-allocated +type Size int + +// UnmarshalText unmarshals Size. +func (s *Size) UnmarshalText(b []byte) error { + if string(b) == "dynamic" { + *s = _dynamicPoolSize + return nil + } + + i, err := strconv.ParseInt(string(b), 10, 64) + if err != nil { + return err + } + + if i < 0 { + return errors.New("pool capacity cannot be negative") + } + + *s = Size(i) + + return nil +} + +// IsDynamic returns whether the pool should be fixed size or not. +func (s Size) IsDynamic() bool { + return s == _dynamicPoolSize +} diff --git a/src/x/pool/config_test.go b/src/x/pool/config_test.go index 7bbe5a08b5..0c0135f46d 100644 --- a/src/x/pool/config_test.go +++ b/src/x/pool/config_test.go @@ -23,9 +23,10 @@ package pool import ( "testing" - "github.com/m3db/m3/src/x/instrument" - "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" + + "github.com/m3db/m3/src/x/instrument" ) func TestObjectPoolConfiguration(t *testing.T) { @@ -38,10 +39,39 @@ func TestObjectPoolConfiguration(t *testing.T) { } opts := cfg.NewObjectPoolOptions(instrument.NewOptions()).(*objectPoolOptions) require.Equal(t, 1, opts.size) + require.False(t, opts.Dynamic()) require.Equal(t, 0.1, opts.refillLowWatermark) require.Equal(t, 0.5, opts.refillHighWatermark) } +func TestDynamicObjectPoolConfiguration(t *testing.T) { + cfg := ObjectPoolConfiguration{ + Size: _dynamicPoolSize, + } + opts := cfg.NewObjectPoolOptions(instrument.NewOptions()).(*objectPoolOptions) + require.Equal(t, _dynamicPoolSize, opts.Size()) + require.True(t, opts.Dynamic()) + + cfg, err := cfgFromStr(` +size: dynamic +`) + require.NoError(t, err) + + opts = cfg.NewObjectPoolOptions(instrument.NewOptions()).(*objectPoolOptions) + require.Equal(t, _dynamicPoolSize, opts.Size()) + require.True(t, opts.Dynamic()) + + _, err = cfgFromStr(` +size: invalid +`) + require.Error(t, err) + + _, err = cfgFromStr(` +size: -10 +`) + require.Error(t, err) +} + func TestBucketizedPoolConfiguration(t *testing.T) { cfg := BucketizedPoolConfiguration{ Buckets: []BucketConfiguration{ @@ -62,3 +92,9 @@ func TestBucketizedPoolConfiguration(t *testing.T) { require.Equal(t, 0.1, opts.refillLowWatermark) require.Equal(t, 0.5, opts.refillHighWatermark) } + +func cfgFromStr(config string) (ObjectPoolConfiguration, error) { + cfg := ObjectPoolConfiguration{} + err := yaml.Unmarshal([]byte(config), &cfg) + return cfg, err +} diff --git a/src/x/pool/floats_test.go b/src/x/pool/floats_test.go index 544a4f065a..aa92e82e9b 100644 --- a/src/x/pool/floats_test.go +++ b/src/x/pool/floats_test.go @@ -53,7 +53,7 @@ func getFloatsPool(bucketSizes int, bucketCaps []int) *floatsPool { buckets := make([]Bucket, len(bucketCaps)) for i, cap := range bucketCaps { buckets[i] = Bucket{ - Count: bucketSizes, + Count: Size(bucketSizes), Capacity: cap, } } diff --git a/src/x/pool/object.go b/src/x/pool/object.go index 2ef9c50b16..ebedf02ff3 100644 --- a/src/x/pool/object.go +++ b/src/x/pool/object.go @@ -23,6 +23,7 @@ package pool import ( "errors" "math" + "sync" "sync/atomic" "github.com/m3db/m3/src/x/unsafe" @@ -65,6 +66,21 @@ func NewObjectPool(opts ObjectPoolOptions) ObjectPool { opts = NewObjectPoolOptions() } + uninitializedAllocatorFn := func() interface{} { + fn := opts.OnPoolAccessErrorFn() + fn(errPoolAccessBeforeInitialized) + return nil + } + + if opts.Dynamic() { + return &dynamicPool{ + pool: sync.Pool{ + New: uninitializedAllocatorFn, + }, + onPoolAccessErrorFn: opts.OnPoolAccessErrorFn(), + } + } + m := opts.InstrumentOptions().MetricsScope() p := &objectPool{ @@ -80,11 +96,7 @@ func NewObjectPool(opts ObjectPoolOptions) ObjectPool { putOnFull: m.Counter("put-on-full"), }, onPoolAccessErrorFn: opts.OnPoolAccessErrorFn(), - alloc: func() interface{} { - fn := opts.OnPoolAccessErrorFn() - fn(errPoolAccessBeforeInitialized) - return nil - }, + alloc: uninitializedAllocatorFn, } p.setGauges() @@ -167,3 +179,28 @@ func (p *objectPool) tryFill() { } }() } + +var _ ObjectPool = (*dynamicPool)(nil) + +type dynamicPool struct { + pool sync.Pool + onPoolAccessErrorFn OnPoolAccessErrorFn + initialized int32 +} + +func (d *dynamicPool) Init(alloc Allocator) { + if !atomic.CompareAndSwapInt32(&d.initialized, 0, 1) { + d.onPoolAccessErrorFn(errPoolAlreadyInitialized) + return + } + + d.pool.New = alloc +} + +func (d *dynamicPool) Get() interface{} { + return d.pool.Get() +} + +func (d *dynamicPool) Put(x interface{}) { + d.pool.Put(x) +} diff --git a/src/x/pool/object_test.go b/src/x/pool/object_test.go index bf039943c4..3992b5420d 100644 --- a/src/x/pool/object_test.go +++ b/src/x/pool/object_test.go @@ -21,6 +21,7 @@ package pool import ( + "runtime" "strconv" "testing" "time" @@ -192,6 +193,40 @@ func BenchmarkObjectPoolParallelGetMultiPutContended(b *testing.B) { o := bufs[i] buf := *o buf = strconv.AppendInt(buf[:0], 12344321, 10) + runtime.KeepAlive(buf) + p.Put(o) + } + } + }) +} + +//nolint:dupl +func BenchmarkObjectPoolParallelGetMultiPutContendedDynamic(b *testing.B) { + opts := NewObjectPoolOptions(). + SetDynamic(true) + + p := NewObjectPool(opts) + p.Init(func() interface{} { + b := make([]byte, 0, 64) + return &b + }) + + b.ResetTimer() + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + bufs := make([]*[]byte, 16) + for i := 0; i < len(bufs); i++ { + o, ok := p.Get().(*[]byte) + if !ok { + b.Fail() + } + bufs[i] = o + } + for i := 0; i < len(bufs); i++ { + o := bufs[i] + buf := *o + buf = strconv.AppendInt(buf[:0], 12344321, 10) + runtime.KeepAlive(buf) p.Put(o) } } diff --git a/src/x/pool/options.go b/src/x/pool/options.go index fd70325d9c..de2440c9a4 100644 --- a/src/x/pool/options.go +++ b/src/x/pool/options.go @@ -23,9 +23,11 @@ package pool import "github.com/m3db/m3/src/x/instrument" const ( - defaultSize = 4096 - defaultRefillLowWatermark = 0.0 - defaultRefillHighWatermark = 0.0 + _defaultSize = 4096 + _defaultRefillLowWatermark = 0.0 + _defaultRefillHighWatermark = 0.0 + + _dynamicPoolSize = -1 ) type objectPoolOptions struct { @@ -34,14 +36,16 @@ type objectPoolOptions struct { refillHighWatermark float64 instrumentOpts instrument.Options onPoolAccessErrorFn OnPoolAccessErrorFn + dynamic bool } // NewObjectPoolOptions creates a new set of object pool options func NewObjectPoolOptions() ObjectPoolOptions { return &objectPoolOptions{ - size: defaultSize, - refillLowWatermark: defaultRefillLowWatermark, - refillHighWatermark: defaultRefillHighWatermark, + size: _defaultSize, + dynamic: false, + refillLowWatermark: _defaultRefillLowWatermark, + refillHighWatermark: _defaultRefillHighWatermark, instrumentOpts: instrument.NewOptions(), onPoolAccessErrorFn: func(err error) { panic(err) }, } @@ -54,9 +58,22 @@ func (o *objectPoolOptions) SetSize(value int) ObjectPoolOptions { } func (o *objectPoolOptions) Size() int { + if o.dynamic { + return _dynamicPoolSize + } return o.size } +func (o *objectPoolOptions) SetDynamic(dynamic bool) ObjectPoolOptions { + opts := *o + opts.dynamic = dynamic + return &opts +} + +func (o *objectPoolOptions) Dynamic() bool { + return o.dynamic +} + func (o *objectPoolOptions) SetRefillLowWatermark(value float64) ObjectPoolOptions { opts := *o opts.refillLowWatermark = value diff --git a/src/x/pool/types.go b/src/x/pool/types.go index 872189a619..4926139247 100644 --- a/src/x/pool/types.go +++ b/src/x/pool/types.go @@ -53,6 +53,12 @@ type ObjectPoolOptions interface { // Size returns the size of the object pool. Size() int + // SetDynamic creates a dynamically-sized, non-preallocated pool. + SetDynamic(value bool) ObjectPoolOptions + + // Dynamic returns if the pool is dynamic. + Dynamic() bool + // SetRefillLowWatermark sets the refill low watermark value between [0, 1), // if zero then no refills occur. SetRefillLowWatermark(value float64) ObjectPoolOptions @@ -90,7 +96,7 @@ type Bucket struct { Capacity int // Count is the number of fixed elements in the bucket. - Count int + Count Size // Options is an optional override to specify options to use for a bucket, // specify nil to use the options specified to the bucketized pool diff --git a/src/x/time/unit.go b/src/x/time/unit.go index f120def9d3..8d423e7f3c 100644 --- a/src/x/time/unit.go +++ b/src/x/time/unit.go @@ -108,11 +108,18 @@ func (tu Unit) String() string { // UnitFromDuration creates a time unit from a time duration. func UnitFromDuration(d time.Duration) (Unit, error) { - if unit, found := durationsToUnit[d]; found { - return unit, nil + i := 0 + // TODO: remove this once we're on go 1.16+, as for loops prevent inlining with older compilers +For: + if i >= len(unitLookupArray) { + return None, errConvertDurationToUnit } - return None, errConvertDurationToUnit + if unitLookupArray[i].duration == d { + return unitLookupArray[i].unit, nil + } + i++ + goto For } // DurationFromUnit creates a time duration from a time unit. @@ -184,13 +191,19 @@ var ( Day: time.Hour * 24, Year: time.Hour * 24 * 365, } - durationsToUnit = make(map[time.Duration]Unit) unitCount = len(unitsToDuration) unitsByDurationDesc []Unit ) +type unitLookupEntry struct { + duration time.Duration + unit Unit +} + +var unitLookupArray []unitLookupEntry + // byDurationDesc sorts time units by their durations in descending order. // The order is undefined if the units are invalid. type byDurationDesc []Unit @@ -206,13 +219,15 @@ func (b byDurationDesc) Less(i, j int) bool { func init() { unitsByDurationDesc = make([]Unit, 0, unitCount) + unitLookupArray = make([]unitLookupEntry, 0, unitCount) + for u, d := range unitsToDuration { unit := Unit(u) if unit == None { continue } - durationsToUnit[d] = unit + unitLookupArray = append(unitLookupArray, unitLookupEntry{unit: unit, duration: d}) unitsByDurationDesc = append(unitsByDurationDesc, unit) } sort.Sort(byDurationDesc(unitsByDurationDesc))