From 6fab578d9ad9e3b508a25282aa3fdcff02bc16ea Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Fri, 10 Apr 2020 11:03:11 -0400 Subject: [PATCH] [aggregator] Add compatibility for rollup rules with timed metrics (#2251) --- scripts/development/m3_stack/m3aggregator.yml | 2 +- .../aggregator/m3aggregator.yml | 8 +- .../aggregator/m3coordinator.yml | 14 + .../aggregator/test.sh | 157 +++++++- .../docker-integration-tests/carbon/test.sh | 2 +- .../coordinator_config_rules/test.sh | 2 +- src/aggregator/aggregation/counter.go | 23 +- src/aggregator/aggregation/gauge.go | 7 +- src/aggregator/aggregation/options.go | 24 +- src/aggregator/aggregation/timer.go | 36 +- .../aggregation/timer_benchmark_test.go | 4 +- src/aggregator/aggregation/timer_test.go | 11 +- src/aggregator/aggregator/aggregation.go | 8 +- src/aggregator/aggregator/aggregator.go | 22 + .../aggregator/capture/aggregator.go | 21 + src/aggregator/aggregator/counter_elem_gen.go | 58 ++- src/aggregator/aggregator/elem_base.go | 53 ++- src/aggregator/aggregator/elem_base_test.go | 81 ++-- src/aggregator/aggregator/elem_test.go | 92 ++--- src/aggregator/aggregator/entry.go | 129 +++++- src/aggregator/aggregator/entry_test.go | 9 +- src/aggregator/aggregator/gauge_elem_gen.go | 60 ++- src/aggregator/aggregator/generic_elem.go | 61 ++- src/aggregator/aggregator/map.go | 18 + src/aggregator/aggregator/shard.go | 46 ++- src/aggregator/aggregator/timer_elem_gen.go | 58 ++- src/aggregator/client/client.go | 27 +- src/aggregator/client/client_mock.go | 30 +- src/aggregator/client/payload.go | 15 +- src/aggregator/client/writer.go | 42 ++ src/aggregator/config/m3aggregator.yml | 2 +- .../integration/integration_data.go | 6 +- src/aggregator/server/rawtcp/server.go | 4 + .../m3aggregator/config/aggregator.go | 17 +- .../m3coordinator/downsample/downsampler.go | 18 +- .../downsample/downsampler_test.go | 282 ++++++++++--- .../downsample/metrics_appender.go | 98 +++-- .../m3coordinator/downsample/options.go | 83 +++- .../downsample/samples_appender.go | 24 +- src/metrics/encoding/protobuf/reset.go | 9 + .../encoding/protobuf/unaggregated_encoder.go | 14 + .../protobuf/unaggregated_iterator.go | 3 + src/metrics/encoding/types.go | 2 + .../generated/proto/metricpb/composite.pb.go | 378 ++++++++++++++---- .../generated/proto/metricpb/composite.proto | 7 + .../transformationpb/transformation.pb.go | 21 +- .../transformationpb/transformation.proto | 2 + src/metrics/metric/aggregated/types.go | 25 ++ src/metrics/transformation/binary.go | 33 ++ src/metrics/transformation/func.go | 26 +- src/metrics/transformation/type.go | 81 +++- src/metrics/transformation/type_string.go | 25 +- src/metrics/transformation/unary.go | 24 ++ 53 files changed, 1820 insertions(+), 484 deletions(-) diff --git a/scripts/development/m3_stack/m3aggregator.yml b/scripts/development/m3_stack/m3aggregator.yml index 46ffd43f85..b47dc791fb 100644 --- a/scripts/development/m3_stack/m3aggregator.yml +++ b/scripts/development/m3_stack/m3aggregator.yml @@ -320,7 +320,7 @@ aggregator: readBufferSize: 256 forwarding: maxSingleDelay: 5s - entryTTL: 6h + entryTTL: 1h entryCheckInterval: 10m maxTimerBatchSizePerWrite: 140 defaultStoragePolicies: diff --git a/scripts/docker-integration-tests/aggregator/m3aggregator.yml b/scripts/docker-integration-tests/aggregator/m3aggregator.yml index 41cc0e813a..16aed28011 100644 --- a/scripts/docker-integration-tests/aggregator/m3aggregator.yml +++ b/scripts/docker-integration-tests/aggregator/m3aggregator.yml @@ -154,6 +154,7 @@ aggregator: envVarName: M3AGGREGATOR_HOST_ID instanceID: type: host_id + verboseErrors: true metricPrefix: "" counterPrefix: "" timerPrefix: "" @@ -230,6 +231,7 @@ aggregator: hashType: murmur32 bufferDurationBeforeShardCutover: 10m bufferDurationAfterShardCutoff: 10m + bufferDurationForFutureTimedMetric: 10m # Allow test to write into future. resignTimeout: 1m flushTimesManager: kvConfig: @@ -319,12 +321,10 @@ aggregator: writeBufferSize: 16384 readBufferSize: 256 forwarding: - maxSingleDelay: 5s - entryTTL: 6h + maxConstDelay: 1m # Need to add some buffer window, since timed metrics by default are delayed by 1min. + entryTTL: 1h entryCheckInterval: 10m maxTimerBatchSizePerWrite: 140 - defaultStoragePolicies: - - 10s:2d maxNumCachedSourceSets: 2 discardNaNAggregatedValues: true entryPool: diff --git a/scripts/docker-integration-tests/aggregator/m3coordinator.yml b/scripts/docker-integration-tests/aggregator/m3coordinator.yml index 475afe191f..b93406ba0f 100644 --- a/scripts/docker-integration-tests/aggregator/m3coordinator.yml +++ b/scripts/docker-integration-tests/aggregator/m3coordinator.yml @@ -52,6 +52,20 @@ clusters: readConsistencyLevel: unstrict_majority downsample: + rules: + rollupRules: + - name: "requests per second by status code" + filter: "__name__:http_requests app:* status_code:* endpoint:*" + transforms: + - transform: + type: "PerSecond" + - rollup: + metricName: "http_requests_by_status_code" + groupBy: ["app", "status_code", "endpoint"] + aggregations: ["Sum"] + storagePolicies: + - resolution: 10s + retention: 6h remoteAggregator: client: placementKV: diff --git a/scripts/docker-integration-tests/aggregator/test.sh b/scripts/docker-integration-tests/aggregator/test.sh index e0b8d7ce13..339eefb41f 100755 --- a/scripts/docker-integration-tests/aggregator/test.sh +++ b/scripts/docker-integration-tests/aggregator/test.sh @@ -5,8 +5,15 @@ set -xe source $GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/common.sh REVISION=$(git rev-parse HEAD) COMPOSE_FILE=$GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/aggregator/docker-compose.yml +# quay.io/m3db/prometheus_remote_client_golang @ v0.4.3 +PROMREMOTECLI_IMAGE=quay.io/m3db/prometheus_remote_client_golang@sha256:fc56df819bff9a5a087484804acf3a584dd4a78c68900c31a28896ed66ca7e7b +JQ_IMAGE=realguess/jq:1.4@sha256:300c5d9fb1d74154248d155ce182e207cf6630acccbaadd0168e18b15bfaa786 export REVISION +echo "Pull containers required for test" +docker pull $PROMREMOTECLI_IMAGE +docker pull $JQ_IMAGE + echo "Run m3dbnode" docker-compose -f ${COMPOSE_FILE} up -d dbnode01 @@ -107,7 +114,7 @@ function read_carbon { end=$(date +%s) start=$(($end-1000)) RESPONSE=$(curl -sSfg "http://${COORDINATOR_API}/api/v1/graphite/render?target=$target&from=$start&until=$end") - test "$(echo "$RESPONSE" | jq ".[0].datapoints | .[][0] | select(. != null)" | tail -n 1)" = "$expected_val" + test "$(echo "$RESPONSE" | jq ".[0].datapoints | .[][0] | select(. != null)" | jq -s last)" = "$expected_val" return $? } @@ -118,8 +125,146 @@ bash -c 'while true; do t=$(date +%s); echo "foo.bar.baz 40 $t" | nc 0.0.0.0 720 # Track PID to kill on exit METRIC_EMIT_PID="$!" -# Read back the averaged averaged metric, we configured graphite -# aggregation policy to average each tile and we are emitting -# values 40 and 44 to get an average of 42 each tile -echo "Read back aggregated averaged metric" -ATTEMPTS=10 TIMEOUT=1 retry_with_backoff read_carbon foo.bar.* 42 +function test_aggregated_graphite_metric { + # Read back the averaged averaged metric, we configured graphite + # aggregation policy to average each tile and we are emitting + # values 40 and 44 to get an average of 42 each tile + echo "Read back aggregated averaged metric" + ATTEMPTS=40 TIMEOUT=1 MAX_TIMEOUT=4 retry_with_backoff read_carbon foo.bar.* 42 + + # echo "Finished with carbon metrics" + kill $METRIC_EMIT_PID + export METRIC_EMIT_PID="-1" +} + +function prometheus_remote_write { + local metric_name=$1 + local datapoint_timestamp=$2 + local datapoint_value=$3 + local expect_success=$4 + local expect_success_err=$5 + local expect_status=$6 + local expect_status_err=$7 + local label0_name=${label0_name:-label0} + local label0_value=${label0_value:-label0} + local label1_name=${label1_name:-label1} + local label1_value=${label1_value:-label1} + local label2_name=${label2_name:-label2} + local label2_value=${label2_value:-label2} + + network_name="aggregator" + network=$(docker network ls | fgrep $network_name | tr -s ' ' | cut -f 1 -d ' ') + out=$((docker run -it --rm --network $network \ + $PROMREMOTECLI_IMAGE \ + -u http://m3coordinator01:7202/api/v1/prom/remote/write \ + -t __name__:${metric_name} \ + -t ${label0_name}:${label0_value} \ + -t ${label1_name}:${label1_value} \ + -t ${label2_name}:${label2_value} \ + -d ${datapoint_timestamp},${datapoint_value} | grep -v promremotecli_log) || true) + success=$(echo $out | grep -v promremotecli_log | docker run --rm -i $JQ_IMAGE jq .success) + status=$(echo $out | grep -v promremotecli_log | docker run --rm -i $JQ_IMAGE jq .statusCode) + if [[ "$success" != "$expect_success" ]]; then + echo $expect_success_err + return 1 + fi + if [[ "$status" != "$expect_status" ]]; then + echo "${expect_status_err}: actual=${status}" + return 1 + fi + echo "Returned success=${success}, status=${status} as expected" + return 0 +} + +function prometheus_query_native { + local endpoint=${endpoint:-} + local query=${query:-} + local params=${params:-} + local metrics_type=${metrics_type:-} + local metrics_storage_policy=${metrics_storage_policy:-} + local jq_path=${jq_path:-} + local expected_value=${expected_value:-} + + params_prefixed="" + if [[ "$params" != "" ]]; then + params_prefixed='&'"${params}" + fi + + result=$(curl -s \ + -H "M3-Metrics-Type: ${metrics_type}" \ + -H "M3-Storage-Policy: ${metrics_storage_policy}" \ + "0.0.0.0:7202/api/v1/${endpoint}?query=${query}${params_prefixed}" | jq -r "${jq_path}" | jq -s last) + test "$result" = "$expected_value" + return $? +} + +function test_aggregated_rollup_rule { + resolution_seconds="10" + now=$(date +"%s") + now_truncate_by=$(( $now % $resolution_seconds )) + now_truncated=$(( $now - $now_truncate_by )) + + echo "Test write with rollup rule" + + # Emit values for endpoint /foo/bar (to ensure right values aggregated) + write_at="$now_truncated" + value="42" + value_rate="22" + value_inc_by=$(( $value_rate * $resolution_seconds )) + for i in $(seq 1 10); do + label0_name="app" label0_value="nginx_edge" \ + label1_name="status_code" label1_value="500" \ + label2_name="endpoint" label2_value="/foo/bar" \ + prometheus_remote_write \ + http_requests $write_at $value \ + true "Expected request to succeed" \ + 200 "Expected request to return status code 200" + write_at=$(( $write_at + $resolution_seconds )) + value=$(( $value + $value_inc_by )) + done + + # Emit values for endpoint /foo/baz (to ensure right values aggregated) + write_at="$now_truncated" + value="84" + value_rate="4" + value_inc_by=$(( $value_rate * $resolution_seconds )) + for i in $(seq 1 10); do + label0_name="app" label0_value="nginx_edge" \ + label1_name="status_code" label1_value="500" \ + label2_name="endpoint" label2_value="/foo/baz" \ + prometheus_remote_write \ + http_requests $write_at $value \ + true "Expected request to succeed" \ + 200 "Expected request to return status code 200" + write_at=$(( $write_at + $resolution_seconds )) + value=$(( $value + $value_inc_by )) + done + + start=$(( $now - 3600 )) + end=$(( $now + 3600 )) + step="30s" + params_range="start=${start}"'&'"end=${end}"'&'"step=30s" + jq_path=".data.result[0].values | .[][1] | select(. != null)" + + echo "Test query rollup rule" + + # Test by values are rolled up by second, then sum (for endpoint="/foo/bar") + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query_range query="http_requests_by_status_code\{endpoint=\"/foo/bar\"\}" \ + params="$params_range" \ + jq_path="$jq_path" expected_value="22" \ + metrics_type="aggregated" metrics_storage_policy="10s:6h" \ + retry_with_backoff prometheus_query_native + + # Test by values are rolled up by second, then sum (for endpoint="/foo/bar") + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query_range query="http_requests_by_status_code\{endpoint=\"/foo/baz\"\}" \ + params="$params_range" \ + jq_path="$jq_path" expected_value="4" \ + metrics_type="aggregated" metrics_storage_policy="10s:6h" \ + retry_with_backoff prometheus_query_native +} + +echo "Run tests" +test_aggregated_graphite_metric +test_aggregated_rollup_rule diff --git a/scripts/docker-integration-tests/carbon/test.sh b/scripts/docker-integration-tests/carbon/test.sh index 2bc019422c..d9cb74aa94 100755 --- a/scripts/docker-integration-tests/carbon/test.sh +++ b/scripts/docker-integration-tests/carbon/test.sh @@ -27,7 +27,7 @@ function read_carbon { end=$(date +%s) start=$(($end-1000)) RESPONSE=$(curl -sSfg "http://localhost:7201/api/v1/graphite/render?target=$target&from=$start&until=$end") - test "$(echo "$RESPONSE" | jq ".[0].datapoints | .[][0] | select(. != null)" | tail -n 1)" = "$expected_val" + test "$(echo "$RESPONSE" | jq ".[0].datapoints | .[][0] | select(. != null)" | jq -s last)" = "$expected_val" return $? } diff --git a/scripts/docker-integration-tests/coordinator_config_rules/test.sh b/scripts/docker-integration-tests/coordinator_config_rules/test.sh index 5e6e12090e..8f0811e737 100755 --- a/scripts/docker-integration-tests/coordinator_config_rules/test.sh +++ b/scripts/docker-integration-tests/coordinator_config_rules/test.sh @@ -83,7 +83,7 @@ function prometheus_query_native { result=$(curl -s \ -H "M3-Metrics-Type: ${metrics_type}" \ -H "M3-Storage-Policy: ${metrics_storage_policy}" \ - "0.0.0.0:7201/api/v1/${endpoint}?query=${query}${params_prefixed}" | jq -r "${jq_path}") + "0.0.0.0:7201/api/v1/${endpoint}?query=${query}${params_prefixed}" | jq -r "${jq_path}" | jq -s last) test "$result" = "$expected_value" return $? } diff --git a/src/aggregator/aggregation/counter.go b/src/aggregator/aggregation/counter.go index 4add082174..f739cc1a9e 100644 --- a/src/aggregator/aggregation/counter.go +++ b/src/aggregator/aggregation/counter.go @@ -31,11 +31,12 @@ import ( type Counter struct { Options - sum int64 - sumSq int64 - count int64 - max int64 - min int64 + lastAt time.Time + sum int64 + sumSq int64 + count int64 + max int64 + min int64 } // NewCounter creates a new counter. @@ -49,6 +50,15 @@ func NewCounter(opts Options) Counter { // Update updates the counter value. func (c *Counter) Update(timestamp time.Time, value int64) { + if c.lastAt.IsZero() || timestamp.After(c.lastAt) { + // NB(r): Only set the last value if this value arrives + // after the wall clock timestamp of previous values, not + // the arrival time (i.e. order received). + c.lastAt = timestamp + } else { + c.Options.Metrics.Counter.IncValuesOutOfOrder() + } + c.sum += value c.count++ @@ -64,6 +74,9 @@ func (c *Counter) Update(timestamp time.Time, value int64) { } } +// LastAt returns the time of the last value received. +func (c *Counter) LastAt() time.Time { return c.lastAt } + // Count returns the number of values received. func (c *Counter) Count() int64 { return c.count } diff --git a/src/aggregator/aggregation/gauge.go b/src/aggregator/aggregation/gauge.go index 4dbd7efcb6..4e381c81c2 100644 --- a/src/aggregator/aggregation/gauge.go +++ b/src/aggregator/aggregation/gauge.go @@ -35,8 +35,8 @@ const ( type Gauge struct { Options - last float64 lastAt time.Time + last float64 sum float64 sumSq float64 count int64 @@ -59,8 +59,8 @@ func (g *Gauge) Update(timestamp time.Time, value float64) { // NB(r): Only set the last value if this value arrives // after the wall clock timestamp of previous values, not // the arrival time (i.e. order received). - g.last = value g.lastAt = timestamp + g.last = value } else { g.Options.Metrics.Gauge.IncValuesOutOfOrder() } @@ -79,6 +79,9 @@ func (g *Gauge) Update(timestamp time.Time, value float64) { } } +// LastAt returns the time of the last value received. +func (g *Gauge) LastAt() time.Time { return g.lastAt } + // Last returns the last value received. func (g *Gauge) Last() float64 { return g.last } diff --git a/src/aggregator/aggregation/options.go b/src/aggregator/aggregation/options.go index d2c797454c..a6ca9f6f1e 100644 --- a/src/aggregator/aggregation/options.go +++ b/src/aggregator/aggregation/options.go @@ -42,7 +42,13 @@ type Options struct { // Metrics is a set of metrics that can be used by elements. type Metrics struct { - Gauge GaugeMetrics + Counter CounterMetrics + Gauge GaugeMetrics +} + +// CounterMetrics is a set of counter metrics can be used by all counters. +type CounterMetrics struct { + valuesOutOfOrder tally.Counter } // GaugeMetrics is a set of gauge metrics can be used by all gauges. @@ -54,7 +60,21 @@ type GaugeMetrics struct { func NewMetrics(scope tally.Scope) Metrics { scope = scope.SubScope("aggregation") return Metrics{ - Gauge: newGaugeMetrics(scope.SubScope("gauges")), + Counter: newCounterMetrics(scope.SubScope("counters")), + Gauge: newGaugeMetrics(scope.SubScope("gauges")), + } +} + +func newCounterMetrics(scope tally.Scope) CounterMetrics { + return CounterMetrics{ + valuesOutOfOrder: scope.Counter("values-out-of-order"), + } +} + +// IncValuesOutOfOrder increments value or if not initialized is a no-op. +func (m CounterMetrics) IncValuesOutOfOrder() { + if m.valuesOutOfOrder != nil { + m.valuesOutOfOrder.Inc(1) } } diff --git a/src/aggregator/aggregation/timer.go b/src/aggregator/aggregation/timer.go index 9ae2b2d122..6b7fc3d4f7 100644 --- a/src/aggregator/aggregation/timer.go +++ b/src/aggregator/aggregation/timer.go @@ -21,6 +21,8 @@ package aggregation import ( + "time" + "github.com/m3db/m3/src/aggregator/aggregation/quantile/cm" "github.com/m3db/m3/src/metrics/aggregation" ) @@ -29,6 +31,7 @@ import ( type Timer struct { Options + lastAt time.Time count int64 // Number of values received. sum float64 // Sum of the values. sumSq float64 // Sum of squared values. @@ -46,7 +49,30 @@ func NewTimer(quantiles []float64, streamOpts cm.Options, opts Options) Timer { } // Add adds a timer value. -func (t *Timer) Add(value float64) { +func (t *Timer) Add(timestamp time.Time, value float64) { + t.recordLastAt(timestamp) + t.addValue(value) +} + +// AddBatch adds a batch of timer values. +func (t *Timer) AddBatch(timestamp time.Time, values []float64) { + // Record last at just once. + t.recordLastAt(timestamp) + for _, v := range values { + t.addValue(v) + } +} + +func (t *Timer) recordLastAt(timestamp time.Time) { + if t.lastAt.IsZero() || timestamp.After(t.lastAt) { + // NB(r): Only set the last value if this value arrives + // after the wall clock timestamp of previous values, not + // the arrival time (i.e. order received). + t.lastAt = timestamp + } +} + +func (t *Timer) addValue(value float64) { t.count++ t.sum += value t.stream.Add(value) @@ -56,12 +82,8 @@ func (t *Timer) Add(value float64) { } } -// AddBatch adds a batch of timer values. -func (t *Timer) AddBatch(values []float64) { - for _, v := range values { - t.Add(v) - } -} +// LastAt returns the time of the last value received. +func (t *Timer) LastAt() time.Time { return t.lastAt } // Quantile returns the value at a given quantile. func (t *Timer) Quantile(q float64) float64 { diff --git a/src/aggregator/aggregation/timer_benchmark_test.go b/src/aggregator/aggregation/timer_benchmark_test.go index d494e90af1..103c15951c 100644 --- a/src/aggregator/aggregation/timer_benchmark_test.go +++ b/src/aggregator/aggregation/timer_benchmark_test.go @@ -22,6 +22,7 @@ package aggregation import ( "testing" + "time" "github.com/m3db/m3/src/aggregator/aggregation/quantile/cm" "github.com/m3db/m3/src/x/instrument" @@ -31,10 +32,11 @@ func getTimer() Timer { opts := NewOptions(instrument.NewOptions()) opts.ResetSetData(testAggTypes) + at := time.Now() timer := NewTimer(testQuantiles, cm.NewOptions(), opts) for i := 1; i <= 100; i++ { - timer.Add(float64(i)) + timer.Add(at, float64(i)) } return timer } diff --git a/src/aggregator/aggregation/timer_test.go b/src/aggregator/aggregation/timer_test.go index 4fc0850558..adc80336cc 100644 --- a/src/aggregator/aggregation/timer_test.go +++ b/src/aggregator/aggregation/timer_test.go @@ -23,6 +23,7 @@ package aggregation import ( "math" "testing" + "time" "github.com/m3db/m3/src/aggregator/aggregation/quantile/cm" "github.com/m3db/m3/src/metrics/aggregation" @@ -58,13 +59,13 @@ func TestCreateTimerResetStream(t *testing.T) { // Add a value to the timer and close the timer, which returns the // underlying stream to the pool. timer := NewTimer(testQuantiles, streamOpts, NewOptions(instrument.NewOptions())) - timer.Add(1.0) + timer.Add(time.Now(), 1.0) require.Equal(t, 1.0, timer.Min()) timer.Close() // Create a new timer and assert the underlying stream has been closed. timer = NewTimer(testQuantiles, streamOpts, NewOptions(instrument.NewOptions())) - timer.Add(1.0) + timer.Add(time.Now(), 1.0) require.Equal(t, 1.0, timer.Min()) timer.Close() require.Equal(t, 0.0, timer.stream.Min()) @@ -90,8 +91,9 @@ func TestTimerAggregations(t *testing.T) { require.Equal(t, 0.0, timer.Quantile(0.99)) // Add values. + at := time.Now() for i := 1; i <= 100; i++ { - timer.Add(float64(i)) + timer.Add(at, float64(i)) } // Validate the timer values match expectations. @@ -153,8 +155,9 @@ func TestTimerAggregationsNotExpensive(t *testing.T) { require.False(t, timer.HasExpensiveAggregations) // Add values. + at := time.Now() for i := 1; i <= 100; i++ { - timer.Add(float64(i)) + timer.Add(at, float64(i)) } // All Non expensive calculations should be performed. diff --git a/src/aggregator/aggregator/aggregation.go b/src/aggregator/aggregator/aggregation.go index 352f496aa7..0f51d89eb7 100644 --- a/src/aggregator/aggregator/aggregation.go +++ b/src/aggregator/aggregator/aggregation.go @@ -53,12 +53,12 @@ func newTimerAggregation(t aggregation.Timer) timerAggregation { return timerAggregation{Timer: t} } -func (a *timerAggregation) Add(_ time.Time, value float64) { - a.Timer.Add(value) +func (a *timerAggregation) Add(timestamp time.Time, value float64) { + a.Timer.Add(timestamp, value) } -func (a *timerAggregation) AddUnion(_ time.Time, mu unaggregated.MetricUnion) { - a.Timer.AddBatch(mu.BatchTimerVal) +func (a *timerAggregation) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) { + a.Timer.AddBatch(timestamp, mu.BatchTimerVal) } // gaugeAggregation is a gauge aggregation. diff --git a/src/aggregator/aggregator/aggregator.go b/src/aggregator/aggregator/aggregator.go index 99406e59e1..84e7cc3dd2 100644 --- a/src/aggregator/aggregator/aggregator.go +++ b/src/aggregator/aggregator/aggregator.go @@ -71,6 +71,9 @@ type Aggregator interface { // AddTimed adds a timed metric with metadata. AddTimed(metric aggregated.Metric, metadata metadata.TimedMetadata) error + // AddTimedWithStagedMetadatas adds a timed metric with staged metadatas. + AddTimedWithStagedMetadatas(metric aggregated.Metric, metas metadata.StagedMetadatas) error + // AddForwarded adds a forwarded metric with metadata. AddForwarded(metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata) error @@ -209,6 +212,25 @@ func (agg *aggregator) AddTimed( return nil } +func (agg *aggregator) AddTimedWithStagedMetadatas( + metric aggregated.Metric, + metas metadata.StagedMetadatas, +) error { + callStart := agg.nowFn() + agg.metrics.timed.Inc(1) + shard, err := agg.shardFor(metric.ID) + if err != nil { + agg.metrics.addTimed.ReportError(err) + return err + } + if err = shard.AddTimedWithStagedMetadatas(metric, metas); err != nil { + agg.metrics.addTimed.ReportError(err) + return err + } + agg.metrics.addTimed.ReportSuccess(agg.nowFn().Sub(callStart)) + return nil +} + func (agg *aggregator) AddForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/aggregator/capture/aggregator.go b/src/aggregator/aggregator/capture/aggregator.go index 1695506e1f..8b9c0cc3a5 100644 --- a/src/aggregator/aggregator/capture/aggregator.go +++ b/src/aggregator/aggregator/capture/aggregator.go @@ -45,6 +45,7 @@ type aggregator struct { gaugesWithMetadatas []unaggregated.GaugeWithMetadatas forwardedMetricsWithMetadata []aggregated.ForwardedMetricWithMetadata timedMetricsWithMetadata []aggregated.TimedMetricWithMetadata + timedMetricsWithMetadatas []aggregated.TimedMetricWithMetadatas } // NewAggregator creates a new capturing aggregator. @@ -111,6 +112,26 @@ func (agg *aggregator) AddTimed( return nil } +func (agg *aggregator) AddTimedWithStagedMetadatas( + metric aggregated.Metric, + sm metadata.StagedMetadatas, +) error { + // Clone the metric and timed metadata to ensure it cannot be mutated externally. + metric = cloneTimedMetric(metric) + sm = cloneStagedMetadatas(sm) + + agg.Lock() + defer agg.Unlock() + + tms := aggregated.TimedMetricWithMetadatas{ + Metric: metric, + StagedMetadatas: sm, + } + agg.timedMetricsWithMetadatas = append(agg.timedMetricsWithMetadatas, tms) + agg.numMetricsAdded++ + return nil +} + func (agg *aggregator) AddForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 6758ed22b4..a6f3c720d3 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -63,10 +63,10 @@ type CounterElem struct { elemBase counterElemBase - values []timedCounter // metric aggregations sorted by time in ascending order - toConsume []timedCounter // small buffer to avoid memory allocations during consumption - lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds - lastConsumedValues []float64 // last consumed values + values []timedCounter // metric aggregations sorted by time in ascending order + toConsume []timedCounter // small buffer to avoid memory allocations during consumption + lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds + lastConsumedValues []transformation.Datapoint // last consumed values } // NewCounterElem creates a new element for the given metric type. @@ -132,11 +132,11 @@ func (e *CounterElem) ResetSetData( } numAggTypes := len(e.aggTypes) if cap(e.lastConsumedValues) < numAggTypes { - e.lastConsumedValues = make([]float64, numAggTypes) + e.lastConsumedValues = make([]transformation.Datapoint, numAggTypes) } e.lastConsumedValues = e.lastConsumedValues[:numAggTypes] for i := 0; i < len(e.lastConsumedValues); i++ { - e.lastConsumedValues[i] = nan + e.lastConsumedValues[i] = transformation.Datapoint{Value: nan} } return nil } @@ -412,28 +412,52 @@ func (e *CounterElem) processValueWithAggregationLock( ) for aggTypeIdx, aggType := range e.aggTypes { value := lockedAgg.aggregation.ValueOf(aggType) - for i := 0; i < transformations.Len(); i++ { - transformType := transformations.At(i).Transformation.Type - if transformType.IsUnaryTransform() { - fn := transformType.MustUnaryTransform() - res := fn(transformation.Datapoint{TimeNanos: timeNanos, Value: value}) + for _, transformOp := range transformations { + unaryOp, isUnaryOp := transformOp.UnaryTransform() + binaryOp, isBinaryOp := transformOp.BinaryTransform() + switch { + case isUnaryOp: + curr := transformation.Datapoint{ + TimeNanos: timeNanos, + Value: value, + } + + res := unaryOp.Evaluate(curr) + value = res.Value - } else { - fn := transformType.MustBinaryTransform() - prev := transformation.Datapoint{TimeNanos: e.lastConsumedAtNanos, Value: e.lastConsumedValues[aggTypeIdx]} - curr := transformation.Datapoint{TimeNanos: timeNanos, Value: value} - res := fn(prev, curr) + + case isBinaryOp: + lastTimeNanos := e.lastConsumedAtNanos + prev := transformation.Datapoint{ + TimeNanos: lastTimeNanos, + Value: e.lastConsumedValues[aggTypeIdx].Value, + } + + currTimeNanos := timeNanos + curr := transformation.Datapoint{ + TimeNanos: currTimeNanos, + Value: value, + } + + res := binaryOp.Evaluate(prev, curr) + // NB: we only need to record the value needed for derivative transformations. // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - e.lastConsumedValues[aggTypeIdx] = value + if !math.IsNaN(curr.Value) { + e.lastConsumedValues[aggTypeIdx] = curr + } + value = res.Value + } } + if discardNaNValues && math.IsNaN(value) { continue } + if !e.parsedPipeline.HasRollup { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index af6228fd3a..eeff7cc5b2 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -35,7 +35,9 @@ import ( mpipeline "github.com/m3db/m3/src/metrics/pipeline" "github.com/m3db/m3/src/metrics/pipeline/applied" "github.com/m3db/m3/src/metrics/policy" + "github.com/m3db/m3/src/metrics/transformation" "github.com/m3db/m3/src/x/pool" + "go.uber.org/zap" "github.com/willf/bitset" ) @@ -192,6 +194,8 @@ func (e *elemBase) resetSetData( ) error { parsed, err := newParsedPipeline(pipeline) if err != nil { + l := e.opts.InstrumentOptions().Logger() + l.Error("error parsing pipeline", zap.Error(err)) return err } e.id = id @@ -378,9 +382,9 @@ type parsedPipeline struct { // Whether the source pipeline contains derivative transformations at its head. HasDerivativeTransform bool - // Sub-pipline containing only transformation operations from the head - // of the source pipeline this parsed pipeline was derived from. - Transformations applied.Pipeline + // Transformation operations from the head of the source pipeline this + // parsed pipeline was derived from. + Transformations []transformation.Op // Whether the source pipeline contains a rollup operation that is either at the // head of the source pipeline or immediately following the transformation operations @@ -417,7 +421,8 @@ func newParsedPipeline(pipeline applied.Pipeline) (parsedPipeline, error) { for i := 0; i < numSteps; i++ { pipelineOp := pipeline.At(i) if pipelineOp.Type != mpipeline.TransformationOpType && pipelineOp.Type != mpipeline.RollupOpType { - return parsedPipeline{}, fmt.Errorf("pipeline %v step %d has invalid operation type %v", pipeline, i, pipelineOp.Type) + err := fmt.Errorf("pipeline %v step %d has invalid operation type %v", pipeline, i, pipelineOp.Type) + return parsedPipeline{}, err } if pipelineOp.Type == mpipeline.RollupOpType { if firstRollupOpIdx == -1 { @@ -433,9 +438,7 @@ func newParsedPipeline(pipeline applied.Pipeline) (parsedPipeline, error) { } } } - if firstRollupOpIdx == -1 { - return parsedPipeline{}, fmt.Errorf("pipeline %v has no rollup operations", pipeline) - } + // Pipelines that compute higher order derivatives require keeping more states including // the raw values and lower order derivatives. For example, a pipline such as `aggregate Last | // perSecond | perSecond` requires storing both the raw value and the first-order derivatives. @@ -445,11 +448,37 @@ func newParsedPipeline(pipeline applied.Pipeline) (parsedPipeline, error) { if transformationDerivativeOrder > maxSupportedTransformationDerivativeOrder { return parsedPipeline{}, fmt.Errorf("pipeline %v transformation derivative order is %d higher than supported %d", pipeline, transformationDerivativeOrder, maxSupportedTransformationDerivativeOrder) } + + var ( + hasRollup = firstRollupOpIdx != -1 + hasDerivativeTransform = transformationDerivativeOrder > 0 + transformPipeline applied.Pipeline + remainder applied.Pipeline + rollup applied.RollupOp + ) + if hasRollup { + transformPipeline = pipeline.SubPipeline(0, firstRollupOpIdx) + remainder = pipeline.SubPipeline(firstRollupOpIdx+1, numSteps) + rollup = pipeline.At(firstRollupOpIdx).Rollup + } else { + transformPipeline = pipeline + } + + transformations := make([]transformation.Op, 0, transformPipeline.Len()) + for i := 0; i < transformPipeline.Len(); i++ { + op, err := transformPipeline.At(i).Transformation.Type.NewOp() + if err != nil { + err := fmt.Errorf("transform could not construct op: %v", err) + return parsedPipeline{}, err + } + transformations = append(transformations, op) + } + return parsedPipeline{ - HasDerivativeTransform: transformationDerivativeOrder > 0, - Transformations: pipeline.SubPipeline(0, firstRollupOpIdx), - HasRollup: true, - Rollup: pipeline.At(firstRollupOpIdx).Rollup, - Remainder: pipeline.SubPipeline(firstRollupOpIdx+1, numSteps), + HasDerivativeTransform: hasDerivativeTransform, + HasRollup: hasRollup, + Transformations: transformations, + Remainder: remainder, + Rollup: rollup, }, nil } diff --git a/src/aggregator/aggregator/elem_base_test.go b/src/aggregator/aggregator/elem_base_test.go index 1ab9948e21..a929227a26 100644 --- a/src/aggregator/aggregator/elem_base_test.go +++ b/src/aggregator/aggregator/elem_base_test.go @@ -37,25 +37,40 @@ import ( "github.com/stretchr/testify/require" ) +func mustNewOp(t require.TestingT, ttype transformation.Type) transformation.Op { + op, err := ttype.NewOp() + require.NoError(t, err) + return op +} + func TestElemBaseID(t *testing.T) { e := &elemBase{} e.resetSetData(testCounterID, testStoragePolicy, maggregation.DefaultTypes, true, applied.DefaultPipeline, 0, NoPrefixNoSuffix) require.Equal(t, testCounterID, e.ID()) } +func requirePipelinesMatch(t require.TestingT, expected, actual parsedPipeline) { + // Compare transform types + require.Equal(t, len(expected.Transformations), len(actual.Transformations)) + for i := range expected.Transformations { + require.Equal(t, expected.Transformations[i].Type(), actual.Transformations[i].Type()) + } + + // Note: transformations are now constructed each time, so not equal, nil out before comparison + expectedWithoutTransforms := expected + expectedWithoutTransforms.Transformations = nil + actualWithoutTransforms := actual + actualWithoutTransforms.Transformations = nil + require.Equal(t, expectedWithoutTransforms, actualWithoutTransforms) +} + func TestElemBaseResetSetData(t *testing.T) { expectedParsedPipeline := parsedPipeline{ HasDerivativeTransform: true, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, - }, - }), + Transformations: []transformation.Op{ + mustNewOp(t, transformation.Absolute), + mustNewOp(t, transformation.PerSecond), + }, HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo.bar"), @@ -78,24 +93,25 @@ func TestElemBaseResetSetData(t *testing.T) { require.Equal(t, testAggregationTypesExpensive, e.aggTypes) require.False(t, e.useDefaultAggregation) require.True(t, e.aggOpts.HasExpensiveAggregations) - require.Equal(t, expectedParsedPipeline, e.parsedPipeline) + + requirePipelinesMatch(t, expectedParsedPipeline, e.parsedPipeline) + require.Equal(t, 3, e.numForwardedTimes) require.False(t, e.tombstoned) require.False(t, e.closed) require.Equal(t, WithPrefixWithSuffix, e.idPrefixSuffixType) } -func TestElemBaseResetSetDataInvalidPipeline(t *testing.T) { - invalidPipeline := applied.NewPipeline([]applied.OpUnion{ +func TestElemBaseResetSetDataNoRollup(t *testing.T) { + pipelineNoRollup := applied.NewPipeline([]applied.OpUnion{ { Type: pipeline.TransformationOpType, Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) e := &elemBase{} - err := e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypes, false, invalidPipeline, 0, WithPrefixWithSuffix) - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "has no rollup operations")) + err := e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypes, false, pipelineNoRollup, 0, WithPrefixWithSuffix) + require.NoError(t, err) } func TestElemBaseForwardedIDWithDefaultPipeline(t *testing.T) { @@ -338,7 +354,6 @@ func TestParsePipelineNoTransformation(t *testing.T) { }) expected := parsedPipeline{ HasDerivativeTransform: false, - Transformations: applied.NewPipeline([]applied.OpUnion{}), HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo"), @@ -367,7 +382,7 @@ func TestParsePipelineNoTransformation(t *testing.T) { } parsed, err := newParsedPipeline(p) require.NoError(t, err) - require.Equal(t, expected, parsed) + requirePipelinesMatch(t, expected, parsed) } func TestParsePipelineWithNonDerivativeTransformation(t *testing.T) { @@ -404,13 +419,8 @@ func TestParsePipelineWithNonDerivativeTransformation(t *testing.T) { }) expected := parsedPipeline{ HasDerivativeTransform: false, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - }), - HasRollup: true, + Transformations: []transformation.Op{mustNewOp(t, transformation.Absolute)}, + HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo"), AggregationID: maggregation.MustCompressTypes(maggregation.Count), @@ -438,7 +448,7 @@ func TestParsePipelineWithNonDerivativeTransformation(t *testing.T) { } parsed, err := newParsedPipeline(p) require.NoError(t, err) - require.Equal(t, expected, parsed) + requirePipelinesMatch(t, expected, parsed) } func TestParsePipelineWithDerivativeTransformation(t *testing.T) { @@ -479,16 +489,10 @@ func TestParsePipelineWithDerivativeTransformation(t *testing.T) { }) expected := parsedPipeline{ HasDerivativeTransform: true, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, - }, - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - }), + Transformations: []transformation.Op{ + mustNewOp(t, transformation.PerSecond), + mustNewOp(t, transformation.Absolute), + }, HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo"), @@ -517,7 +521,7 @@ func TestParsePipelineWithDerivativeTransformation(t *testing.T) { } parsed, err := newParsedPipeline(p) require.NoError(t, err) - require.Equal(t, expected, parsed) + requirePipelinesMatch(t, expected, parsed) } func TestParsePipelineInvalidOperationType(t *testing.T) { @@ -543,8 +547,7 @@ func TestParsePipelineNoRollupOperation(t *testing.T) { }, }) _, err := newParsedPipeline(p) - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "has no rollup operations")) + require.NoError(t, err) } func TestParsePipelineTransformationDerivativeOrderTooHigh(t *testing.T) { diff --git a/src/aggregator/aggregator/elem_test.go b/src/aggregator/aggregator/elem_test.go index 9818b0cc83..f3e223897b 100644 --- a/src/aggregator/aggregator/elem_test.go +++ b/src/aggregator/aggregator/elem_test.go @@ -128,16 +128,10 @@ func TestCounterResetSetData(t *testing.T) { // Reset element with a pipeline containing a derivative transformation. expectedParsedPipeline := parsedPipeline{ HasDerivativeTransform: true, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, - }, - }), + Transformations: []transformation.Op{ + mustNewOp(t, transformation.Absolute), + mustNewOp(t, transformation.PerSecond), + }, HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo.bar"), @@ -155,10 +149,10 @@ func TestCounterResetSetData(t *testing.T) { } err = ce.ResetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0, NoPrefixNoSuffix) require.NoError(t, err) - require.Equal(t, expectedParsedPipeline, ce.parsedPipeline) + requirePipelinesMatch(t, expectedParsedPipeline, ce.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(ce.lastConsumedValues)) for i := 0; i < len(ce.lastConsumedValues); i++ { - require.True(t, math.IsNaN(ce.lastConsumedValues[i])) + require.True(t, math.IsNaN(ce.lastConsumedValues[i].Value)) } } @@ -169,18 +163,18 @@ func TestCounterResetSetDataInvalidAggregationType(t *testing.T) { require.Error(t, err) } -func TestCounterResetSetDataInvalidPipeline(t *testing.T) { +func TestCounterResetSetDataNoRollup(t *testing.T) { opts := NewOptions() ce := MustNewCounterElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NoPrefixNoSuffix, opts) - invalidPipeline := applied.NewPipeline([]applied.OpUnion{ + pipelineNoRollup := applied.NewPipeline([]applied.OpUnion{ { Type: pipeline.TransformationOpType, Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) - err := ce.ResetSetData(testCounterID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0, NoPrefixNoSuffix) - require.Error(t, err) + err := ce.ResetSetData(testCounterID, testStoragePolicy, maggregation.DefaultTypes, pipelineNoRollup, 0, NoPrefixNoSuffix) + require.NoError(t, err) } func TestCounterElemAddUnion(t *testing.T) { @@ -535,7 +529,8 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 2, len(e.values)) require.Equal(t, time.Unix(220, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{123.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 123.0, e.lastConsumedValues[0].Value) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -559,7 +554,8 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(e.values)) require.Equal(t, time.Unix(240, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{589.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 589.0, e.lastConsumedValues[0].Value) // Tombstone the element and discard all values. e.tombstoned = true @@ -683,16 +679,10 @@ func TestTimerResetSetData(t *testing.T) { // Reset element with a pipeline containing a derivative transformation. expectedParsedPipeline := parsedPipeline{ HasDerivativeTransform: true, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, - }, - }), + Transformations: []transformation.Op{ + mustNewOp(t, transformation.Absolute), + mustNewOp(t, transformation.PerSecond), + }, HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo.bar"), @@ -710,10 +700,10 @@ func TestTimerResetSetData(t *testing.T) { } err = te.ResetSetData(testBatchTimerID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0, NoPrefixNoSuffix) require.NoError(t, err) - require.Equal(t, expectedParsedPipeline, te.parsedPipeline) + requirePipelinesMatch(t, expectedParsedPipeline, te.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(te.lastConsumedValues)) for i := 0; i < len(te.lastConsumedValues); i++ { - require.True(t, math.IsNaN(te.lastConsumedValues[i])) + require.True(t, math.IsNaN(te.lastConsumedValues[i].Value)) } } @@ -724,18 +714,18 @@ func TestTimerResetSetDataInvalidAggregationType(t *testing.T) { require.Error(t, err) } -func TestTimerResetSetDataInvalidPipeline(t *testing.T) { +func TestTimerResetSetDataNoRollup(t *testing.T) { opts := NewOptions() te := MustNewTimerElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NoPrefixNoSuffix, opts) - invalidPipeline := applied.NewPipeline([]applied.OpUnion{ + pipelineNoRollup := applied.NewPipeline([]applied.OpUnion{ { Type: pipeline.TransformationOpType, Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) - err := te.ResetSetData(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0, NoPrefixNoSuffix) - require.Error(t, err) + err := te.ResetSetData(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, pipelineNoRollup, 0, NoPrefixNoSuffix) + require.NoError(t, err) } func TestTimerElemAddUnion(t *testing.T) { @@ -1033,7 +1023,8 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 2, len(e.values)) require.Equal(t, time.Unix(220, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{123.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 123.0, e.lastConsumedValues[0].Value) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -1057,7 +1048,8 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(e.values)) require.Equal(t, time.Unix(240, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{589.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 589.0, e.lastConsumedValues[0].Value) // Tombstone the element and discard all values. e.tombstoned = true @@ -1186,16 +1178,10 @@ func TestGaugeResetSetData(t *testing.T) { // Reset element with a pipeline containing a derivative transformation. expectedParsedPipeline := parsedPipeline{ HasDerivativeTransform: true, - Transformations: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, - }, - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{Type: transformation.PerSecond}, - }, - }), + Transformations: []transformation.Op{ + mustNewOp(t, transformation.Absolute), + mustNewOp(t, transformation.PerSecond), + }, HasRollup: true, Rollup: applied.RollupOp{ ID: []byte("foo.bar"), @@ -1213,10 +1199,10 @@ func TestGaugeResetSetData(t *testing.T) { } err = ge.ResetSetData(testGaugeID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0, NoPrefixNoSuffix) require.NoError(t, err) - require.Equal(t, expectedParsedPipeline, ge.parsedPipeline) + requirePipelinesMatch(t, expectedParsedPipeline, ge.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(ge.lastConsumedValues)) for i := 0; i < len(ge.lastConsumedValues); i++ { - require.True(t, math.IsNaN(ge.lastConsumedValues[i])) + require.True(t, math.IsNaN(ge.lastConsumedValues[i].Value)) } } @@ -1576,7 +1562,8 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 2, len(e.values)) require.Equal(t, time.Unix(220, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{123.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 123.0, e.lastConsumedValues[0].Value) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -1600,7 +1587,8 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(e.values)) require.Equal(t, time.Unix(240, 0).UnixNano(), e.lastConsumedAtNanos) - require.Equal(t, []float64{589.0}, e.lastConsumedValues) + require.Equal(t, 1, len(e.lastConsumedValues)) + require.Equal(t, 589.0, e.lastConsumedValues[0].Value) // Tombstone the element and discard all values. e.tombstoned = true @@ -1824,7 +1812,7 @@ func testTimerElem( for i, aligned := range alignedstartAtNanos { newTimer := raggregation.NewTimer(opts.AggregationTypesOptions().Quantiles(), opts.StreamOptions(), e.aggOpts) timer := &lockedTimerAggregation{aggregation: newTimerAggregation(newTimer)} - timer.aggregation.AddBatch(timerBatches[i]) + timer.aggregation.AddBatch(time.Now(), timerBatches[i]) e.values = append(e.values, timedTimer{ startAtNanos: aligned, lockedAgg: timer, @@ -1843,7 +1831,7 @@ func testGaugeElem( e := MustNewGaugeElem(testGaugeID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, WithPrefixWithSuffix, opts) for i, aligned := range alignedstartAtNanos { gauge := &lockedGaugeAggregation{aggregation: newGaugeAggregation(raggregation.NewGauge(e.aggOpts))} - gauge.aggregation.Update(time.Now(), gaugeVals[i]) + gauge.aggregation.Update(time.Unix(0, aligned), gaugeVals[i]) e.values = append(e.values, timedGauge{ startAtNanos: aligned, lockedAgg: gauge, diff --git a/src/aggregator/aggregator/entry.go b/src/aggregator/aggregator/entry.go index 4729dddba3..0c4f5cdd02 100644 --- a/src/aggregator/aggregator/entry.go +++ b/src/aggregator/aggregator/entry.go @@ -35,6 +35,7 @@ import ( "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric" "github.com/m3db/m3/src/metrics/metric/aggregated" + "github.com/m3db/m3/src/metrics/metric/id" metricid "github.com/m3db/m3/src/metrics/metric/id" "github.com/m3db/m3/src/metrics/metric/unaggregated" "github.com/m3db/m3/src/metrics/policy" @@ -100,18 +101,24 @@ func newUntimedEntryMetrics(scope tally.Scope) untimedEntryMetrics { } type timedEntryMetrics struct { - rateLimit rateLimitEntryMetrics - tooFarInTheFuture tally.Counter - tooFarInThePast tally.Counter - metadataUpdates tally.Counter + rateLimit rateLimitEntryMetrics + tooFarInTheFuture tally.Counter + tooFarInThePast tally.Counter + noPipelinesInMetadata tally.Counter + tombstonedMetadata tally.Counter + metadataUpdates tally.Counter + metadatasUpdates tally.Counter } func newTimedEntryMetrics(scope tally.Scope) timedEntryMetrics { return timedEntryMetrics{ - rateLimit: newRateLimitEntryMetrics(scope), - tooFarInTheFuture: scope.Counter("too-far-in-the-future"), - tooFarInThePast: scope.Counter("too-far-in-the-past"), - metadataUpdates: scope.Counter("metadata-updates"), + rateLimit: newRateLimitEntryMetrics(scope), + tooFarInTheFuture: scope.Counter("too-far-in-the-future"), + tooFarInThePast: scope.Counter("too-far-in-the-past"), + noPipelinesInMetadata: scope.Counter("no-pipelines-in-metadata"), + tombstonedMetadata: scope.Counter("tombstoned-metadata"), + metadataUpdates: scope.Counter("metadata-updates"), + metadatasUpdates: scope.Counter("metadatas-updates"), } } @@ -254,7 +261,18 @@ func (e *Entry) AddTimed( if err := e.applyValueRateLimit(1, e.metrics.timed.rateLimit); err != nil { return err } - return e.addTimed(metric, metadata) + return e.addTimed(metric, metadata, nil) +} + +// AddTimedWithStagedMetadatas adds a timed metric with staged metadatas. +func (e *Entry) AddTimedWithStagedMetadatas( + metric aggregated.Metric, + metas metadata.StagedMetadatas, +) error { + if err := e.applyValueRateLimit(1, e.metrics.timed.rateLimit); err != nil { + return err + } + return e.addTimed(metric, metadata.TimedMetadata{}, metas) } // AddForwarded adds a forwarded metric alongside its metadata. @@ -414,13 +432,16 @@ func (e *Entry) addUntimed( } if e.shouldUpdateStagedMetadatasWithLock(sm) { - if err = e.updateStagedMetadatasWithLock(metric, hasDefaultMetadatas, sm); err != nil { + err := e.updateStagedMetadatasWithLock(metric.ID, metric.Type, + hasDefaultMetadatas, sm) + if err != nil { // NB(xichen): if an error occurred during policy update, the policies // will remain as they are, i.e., there are no half-updated policies. e.Unlock() timeLock.RUnlock() return err } + e.metrics.untimed.metadatasUpdates.Inc(1) } err = e.addUntimedWithLock(currTime, metric) @@ -565,12 +586,13 @@ func (e *Entry) removeOldAggregations(newAggregations aggregationValues) { } func (e *Entry) updateStagedMetadatasWithLock( - metric unaggregated.MetricUnion, + metricID id.RawID, + metricType metric.Type, hasDefaultMetadatas bool, sm metadata.StagedMetadata, ) error { var ( - elemID = e.maybeCopyIDWithLock(metric.ID) + elemID = e.maybeCopyIDWithLock(metricID) newAggregations = make(aggregationValues, 0, initialAggregationCapacity) ) @@ -588,7 +610,7 @@ func (e *Entry) updateStagedMetadatasWithLock( resolution: storagePolicy.Resolution().Window, }.toMetricListID() var err error - newAggregations, err = e.addNewAggregationKeyWithLock(metric.Type, elemID, key, listID, newAggregations) + newAggregations, err = e.addNewAggregationKeyWithLock(metricType, elemID, key, listID, newAggregations) if err != nil { return err } @@ -602,7 +624,6 @@ func (e *Entry) updateStagedMetadatasWithLock( e.aggregations = newAggregations e.hasDefaultMetadatas = hasDefaultMetadatas e.cutoverNanos = sm.CutoverNanos - e.metrics.untimed.metadatasUpdates.Inc(1) return nil } @@ -620,6 +641,7 @@ func (e *Entry) addUntimedWithLock(timestamp time.Time, mu unaggregated.MetricUn func (e *Entry) addTimed( metric aggregated.Metric, metadata metadata.TimedMetadata, + stagedMetadatas metadata.StagedMetadatas, ) error { timeLock := e.opts.TimeLock() timeLock.RLock() @@ -651,6 +673,72 @@ func (e *Entry) addTimed( return err } + // Only apply processing of staged metadatas if has sent staged metadatas + // that isn't the default staged metadatas. + hasDefaultMetadatas := stagedMetadatas.IsDefault() + if len(stagedMetadatas) > 0 && !hasDefaultMetadatas { + sm, err := e.activeStagedMetadataWithLock(currTime, stagedMetadatas) + if err != nil { + e.RUnlock() + timeLock.RUnlock() + return err + } + + // If the metadata indicates the (rollup) metric has been tombstoned, the metric is + // not ingested for aggregation. However, we do not update the policies asssociated + // with this entry and mark it tombstoned because there may be a different raw metric + // generating this same (rollup) metric that is actively emitting, meaning this entry + // may still be very much alive. + if sm.Tombstoned { + e.RUnlock() + timeLock.RUnlock() + e.metrics.timed.tombstonedMetadata.Inc(1) + return nil + } + + // It is expected that there is at least one pipeline in the metadata. + if len(sm.Pipelines) == 0 { + e.RUnlock() + timeLock.RUnlock() + e.metrics.timed.noPipelinesInMetadata.Inc(1) + return errNoPipelinesInMetadata + } + + if !e.shouldUpdateStagedMetadatasWithLock(sm) { + err = e.addTimedWithStagedMetadatasAndLock(metric) + e.RUnlock() + timeLock.RUnlock() + return err + } + e.RUnlock() + + e.Lock() + if e.closed { + e.Unlock() + timeLock.RUnlock() + return errEntryClosed + } + + if e.shouldUpdateStagedMetadatasWithLock(sm) { + err := e.updateStagedMetadatasWithLock(metric.ID, metric.Type, + hasDefaultMetadatas, sm) + if err != nil { + // NB(xichen): if an error occurred during policy update, the policies + // will remain as they are, i.e., there are no half-updated policies. + e.Unlock() + timeLock.RUnlock() + return err + } + e.metrics.timed.metadatasUpdates.Inc(1) + } + + err = e.addTimedWithStagedMetadatasAndLock(metric) + e.Unlock() + timeLock.RUnlock() + + return err + } + // Check if we should update metadata, and add metric if not. key := aggregationKey{ aggregationID: metadata.AggregationID, @@ -774,6 +862,19 @@ func (e *Entry) addTimedWithLock( return value.elem.Value.(metricElem).AddValue(timestamp, metric.Value) } +func (e *Entry) addTimedWithStagedMetadatasAndLock( + metric aggregated.Metric, +) error { + timestamp := time.Unix(0, metric.TimeNanos) + multiErr := xerrors.NewMultiError() + for _, val := range e.aggregations { + if err := val.elem.Value.(metricElem).AddValue(timestamp, metric.Value); err != nil { + multiErr = multiErr.Add(err) + } + } + return multiErr.FinalError() +} + func (e *Entry) addForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/aggregator/entry_test.go b/src/aggregator/aggregator/entry_test.go index 0d25154f8a..f03fb94b25 100644 --- a/src/aggregator/aggregator/entry_test.go +++ b/src/aggregator/aggregator/entry_test.go @@ -999,11 +999,11 @@ func TestEntryAddUntimedWithInvalidAggregationType(t *testing.T) { } } -func TestEntryAddUntimedWithInvalidPipeline(t *testing.T) { +func TestEntryAddUntimedWithNoRollup(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - invalidPipeline := metadata.PipelineMetadata{ + pipelineNoRollup := metadata.PipelineMetadata{ Pipeline: applied.NewPipeline([]applied.OpUnion{ { Type: pipeline.TransformationOpType, @@ -1013,11 +1013,10 @@ func TestEntryAddUntimedWithInvalidPipeline(t *testing.T) { } e, _, _ := testEntry(ctrl, testEntryOptions{}) metadatas := metadata.StagedMetadatas{ - {Metadata: metadata.Metadata{Pipelines: []metadata.PipelineMetadata{invalidPipeline}}}, + {Metadata: metadata.Metadata{Pipelines: []metadata.PipelineMetadata{pipelineNoRollup}}}, } err := e.AddUntimed(testCounter, metadatas) - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "has no rollup operations")) + require.NoError(t, err) } func TestShouldUpdateStagedMetadataWithLock(t *testing.T) { diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index 536ff03992..0a7de840b7 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -1,4 +1,4 @@ -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -63,10 +63,10 @@ type GaugeElem struct { elemBase gaugeElemBase - values []timedGauge // metric aggregations sorted by time in ascending order - toConsume []timedGauge // small buffer to avoid memory allocations during consumption - lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds - lastConsumedValues []float64 // last consumed values + values []timedGauge // metric aggregations sorted by time in ascending order + toConsume []timedGauge // small buffer to avoid memory allocations during consumption + lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds + lastConsumedValues []transformation.Datapoint // last consumed values } // NewGaugeElem creates a new element for the given metric type. @@ -132,11 +132,11 @@ func (e *GaugeElem) ResetSetData( } numAggTypes := len(e.aggTypes) if cap(e.lastConsumedValues) < numAggTypes { - e.lastConsumedValues = make([]float64, numAggTypes) + e.lastConsumedValues = make([]transformation.Datapoint, numAggTypes) } e.lastConsumedValues = e.lastConsumedValues[:numAggTypes] for i := 0; i < len(e.lastConsumedValues); i++ { - e.lastConsumedValues[i] = nan + e.lastConsumedValues[i] = transformation.Datapoint{Value: nan} } return nil } @@ -412,28 +412,52 @@ func (e *GaugeElem) processValueWithAggregationLock( ) for aggTypeIdx, aggType := range e.aggTypes { value := lockedAgg.aggregation.ValueOf(aggType) - for i := 0; i < transformations.Len(); i++ { - transformType := transformations.At(i).Transformation.Type - if transformType.IsUnaryTransform() { - fn := transformType.MustUnaryTransform() - res := fn(transformation.Datapoint{TimeNanos: timeNanos, Value: value}) + for _, transformOp := range transformations { + unaryOp, isUnaryOp := transformOp.UnaryTransform() + binaryOp, isBinaryOp := transformOp.BinaryTransform() + switch { + case isUnaryOp: + curr := transformation.Datapoint{ + TimeNanos: timeNanos, + Value: value, + } + + res := unaryOp.Evaluate(curr) + value = res.Value - } else { - fn := transformType.MustBinaryTransform() - prev := transformation.Datapoint{TimeNanos: e.lastConsumedAtNanos, Value: e.lastConsumedValues[aggTypeIdx]} - curr := transformation.Datapoint{TimeNanos: timeNanos, Value: value} - res := fn(prev, curr) + + case isBinaryOp: + lastTimeNanos := e.lastConsumedAtNanos + prev := transformation.Datapoint{ + TimeNanos: lastTimeNanos, + Value: e.lastConsumedValues[aggTypeIdx].Value, + } + + currTimeNanos := timeNanos + curr := transformation.Datapoint{ + TimeNanos: currTimeNanos, + Value: value, + } + + res := binaryOp.Evaluate(prev, curr) + // NB: we only need to record the value needed for derivative transformations. // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - e.lastConsumedValues[aggTypeIdx] = value + if !math.IsNaN(curr.Value) { + e.lastConsumedValues[aggTypeIdx] = curr + } + value = res.Value + } } + if discardNaNValues && math.IsNaN(value) { continue } + if !e.parsedPipeline.HasRollup { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index 49ba630ca2..f5c57e2a54 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -51,6 +51,9 @@ type typeSpecificAggregation interface { // ValueOf returns the value for the given aggregation type. ValueOf(aggType maggregation.Type) float64 + // LastAt returns the time for last received value. + LastAt() time.Time + // Close closes the aggregation object. Close() } @@ -117,10 +120,10 @@ type GenericElem struct { elemBase typeSpecificElemBase - values []timedAggregation // metric aggregations sorted by time in ascending order - toConsume []timedAggregation // small buffer to avoid memory allocations during consumption - lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds - lastConsumedValues []float64 // last consumed values + values []timedAggregation // metric aggregations sorted by time in ascending order + toConsume []timedAggregation // small buffer to avoid memory allocations during consumption + lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds + lastConsumedValues []transformation.Datapoint // last consumed values } // NewGenericElem creates a new element for the given metric type. @@ -186,11 +189,11 @@ func (e *GenericElem) ResetSetData( } numAggTypes := len(e.aggTypes) if cap(e.lastConsumedValues) < numAggTypes { - e.lastConsumedValues = make([]float64, numAggTypes) + e.lastConsumedValues = make([]transformation.Datapoint, numAggTypes) } e.lastConsumedValues = e.lastConsumedValues[:numAggTypes] for i := 0; i < len(e.lastConsumedValues); i++ { - e.lastConsumedValues[i] = nan + e.lastConsumedValues[i] = transformation.Datapoint{Value: nan} } return nil } @@ -466,28 +469,52 @@ func (e *GenericElem) processValueWithAggregationLock( ) for aggTypeIdx, aggType := range e.aggTypes { value := lockedAgg.aggregation.ValueOf(aggType) - for i := 0; i < transformations.Len(); i++ { - transformType := transformations.At(i).Transformation.Type - if transformType.IsUnaryTransform() { - fn := transformType.MustUnaryTransform() - res := fn(transformation.Datapoint{TimeNanos: timeNanos, Value: value}) + for _, transformOp := range transformations { + unaryOp, isUnaryOp := transformOp.UnaryTransform() + binaryOp, isBinaryOp := transformOp.BinaryTransform() + switch { + case isUnaryOp: + curr := transformation.Datapoint{ + TimeNanos: timeNanos, + Value: value, + } + + res := unaryOp.Evaluate(curr) + value = res.Value - } else { - fn := transformType.MustBinaryTransform() - prev := transformation.Datapoint{TimeNanos: e.lastConsumedAtNanos, Value: e.lastConsumedValues[aggTypeIdx]} - curr := transformation.Datapoint{TimeNanos: timeNanos, Value: value} - res := fn(prev, curr) + + case isBinaryOp: + lastTimeNanos := e.lastConsumedAtNanos + prev := transformation.Datapoint{ + TimeNanos: lastTimeNanos, + Value: e.lastConsumedValues[aggTypeIdx].Value, + } + + currTimeNanos := timeNanos + curr := transformation.Datapoint{ + TimeNanos: currTimeNanos, + Value: value, + } + + res := binaryOp.Evaluate(prev, curr) + // NB: we only need to record the value needed for derivative transformations. // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - e.lastConsumedValues[aggTypeIdx] = value + if !math.IsNaN(curr.Value) { + e.lastConsumedValues[aggTypeIdx] = curr + } + value = res.Value + } } + if discardNaNValues && math.IsNaN(value) { continue } + if !e.parsedPipeline.HasRollup { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: diff --git a/src/aggregator/aggregator/map.go b/src/aggregator/aggregator/map.go index cc5d0f3877..130c20d680 100644 --- a/src/aggregator/aggregator/map.go +++ b/src/aggregator/aggregator/map.go @@ -178,6 +178,24 @@ func (m *metricMap) AddTimed( return err } +func (m *metricMap) AddTimedWithStagedMetadatas( + metric aggregated.Metric, + metas metadata.StagedMetadatas, +) error { + key := entryKey{ + metricCategory: timedMetric, + metricType: metric.Type, + idHash: hash.Murmur3Hash128(metric.ID), + } + entry, err := m.findOrCreate(key) + if err != nil { + return err + } + err = entry.AddTimedWithStagedMetadatas(metric, metas) + entry.DecWriter() + return err +} + func (m *metricMap) AddForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/aggregator/shard.go b/src/aggregator/aggregator/shard.go index 1acef7f3f8..712f717568 100644 --- a/src/aggregator/aggregator/shard.go +++ b/src/aggregator/aggregator/shard.go @@ -50,6 +50,11 @@ type addTimedFn func( metadata metadata.TimedMetadata, ) error +type addTimedWithStagedMetadatasFn func( + metric aggregated.Metric, + metas metadata.StagedMetadatas, +) error + type addForwardedFn func( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, @@ -80,12 +85,13 @@ type aggregatorShard struct { earliestWritableNanos int64 latestWriteableNanos int64 - closed bool - metricMap *metricMap - metrics aggregatorShardMetrics - addUntimedFn addUntimedFn - addTimedFn addTimedFn - addForwardedFn addForwardedFn + closed bool + metricMap *metricMap + metrics aggregatorShardMetrics + addUntimedFn addUntimedFn + addTimedFn addTimedFn + addTimedWithStagedMetadatasFn addTimedWithStagedMetadatasFn + addForwardedFn addForwardedFn } func newAggregatorShard(shard uint32, opts Options) *aggregatorShard { @@ -97,8 +103,8 @@ func newAggregatorShard(shard uint32, opts Options) *aggregatorShard { map[string]string{"shard": strconv.Itoa(int(shard))}, ) s := &aggregatorShard{ - shard: shard, - nowFn: opts.ClockOptions().NowFn(), + shard: shard, + nowFn: opts.ClockOptions().NowFn(), bufferDurationBeforeShardCutover: opts.BufferDurationBeforeShardCutover(), bufferDurationAfterShardCutoff: opts.BufferDurationAfterShardCutoff(), metricMap: newMetricMap(shard, opts), @@ -106,6 +112,7 @@ func newAggregatorShard(shard uint32, opts Options) *aggregatorShard { } s.addUntimedFn = s.metricMap.AddUntimed s.addTimedFn = s.metricMap.AddTimed + s.addTimedWithStagedMetadatasFn = s.metricMap.AddTimedWithStagedMetadatas s.addForwardedFn = s.metricMap.AddForwarded return s } @@ -201,6 +208,29 @@ func (s *aggregatorShard) AddTimed( return nil } +func (s *aggregatorShard) AddTimedWithStagedMetadatas( + metric aggregated.Metric, + metas metadata.StagedMetadatas, +) error { + s.RLock() + if s.closed { + s.RUnlock() + return errAggregatorShardClosed + } + if !s.isWritableWithLock() { + s.RUnlock() + s.metrics.notWriteableErrors.Inc(1) + return errAggregatorShardNotWriteable + } + err := s.addTimedWithStagedMetadatasFn(metric, metas) + s.RUnlock() + if err != nil { + return err + } + s.metrics.writeSucccess.Inc(1) + return nil +} + func (s *aggregatorShard) AddForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index 40e7dd0df7..a40370f3ac 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -63,10 +63,10 @@ type TimerElem struct { elemBase timerElemBase - values []timedTimer // metric aggregations sorted by time in ascending order - toConsume []timedTimer // small buffer to avoid memory allocations during consumption - lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds - lastConsumedValues []float64 // last consumed values + values []timedTimer // metric aggregations sorted by time in ascending order + toConsume []timedTimer // small buffer to avoid memory allocations during consumption + lastConsumedAtNanos int64 // last consumed at in Unix nanoseconds + lastConsumedValues []transformation.Datapoint // last consumed values } // NewTimerElem creates a new element for the given metric type. @@ -132,11 +132,11 @@ func (e *TimerElem) ResetSetData( } numAggTypes := len(e.aggTypes) if cap(e.lastConsumedValues) < numAggTypes { - e.lastConsumedValues = make([]float64, numAggTypes) + e.lastConsumedValues = make([]transformation.Datapoint, numAggTypes) } e.lastConsumedValues = e.lastConsumedValues[:numAggTypes] for i := 0; i < len(e.lastConsumedValues); i++ { - e.lastConsumedValues[i] = nan + e.lastConsumedValues[i] = transformation.Datapoint{Value: nan} } return nil } @@ -412,28 +412,52 @@ func (e *TimerElem) processValueWithAggregationLock( ) for aggTypeIdx, aggType := range e.aggTypes { value := lockedAgg.aggregation.ValueOf(aggType) - for i := 0; i < transformations.Len(); i++ { - transformType := transformations.At(i).Transformation.Type - if transformType.IsUnaryTransform() { - fn := transformType.MustUnaryTransform() - res := fn(transformation.Datapoint{TimeNanos: timeNanos, Value: value}) + for _, transformOp := range transformations { + unaryOp, isUnaryOp := transformOp.UnaryTransform() + binaryOp, isBinaryOp := transformOp.BinaryTransform() + switch { + case isUnaryOp: + curr := transformation.Datapoint{ + TimeNanos: timeNanos, + Value: value, + } + + res := unaryOp.Evaluate(curr) + value = res.Value - } else { - fn := transformType.MustBinaryTransform() - prev := transformation.Datapoint{TimeNanos: e.lastConsumedAtNanos, Value: e.lastConsumedValues[aggTypeIdx]} - curr := transformation.Datapoint{TimeNanos: timeNanos, Value: value} - res := fn(prev, curr) + + case isBinaryOp: + lastTimeNanos := e.lastConsumedAtNanos + prev := transformation.Datapoint{ + TimeNanos: lastTimeNanos, + Value: e.lastConsumedValues[aggTypeIdx].Value, + } + + currTimeNanos := timeNanos + curr := transformation.Datapoint{ + TimeNanos: currTimeNanos, + Value: value, + } + + res := binaryOp.Evaluate(prev, curr) + // NB: we only need to record the value needed for derivative transformations. // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - e.lastConsumedValues[aggTypeIdx] = value + if !math.IsNaN(curr.Value) { + e.lastConsumedValues[aggTypeIdx] = curr + } + value = res.Value + } } + if discardNaNValues && math.IsNaN(value) { continue } + if !e.parsedPipeline.HasRollup { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: diff --git a/src/aggregator/client/client.go b/src/aggregator/client/client.go index 8900ec505c..2317b947ab 100644 --- a/src/aggregator/client/client.go +++ b/src/aggregator/client/client.go @@ -75,6 +75,12 @@ type Client interface { metadata metadata.TimedMetadata, ) error + // WriteTimedWithStagedMetadatas writes timed metrics with staged metadatas. + WriteTimedWithStagedMetadatas( + metric aggregated.Metric, + metadatas metadata.StagedMetadatas, + ) error + // Flush flushes any remaining data buffered by the client. Flush() error @@ -166,8 +172,8 @@ func NewClient(opts Options) Client { placementWatcher := placement.NewStagedPlacementWatcher(placementWatcherOpts) return &client{ - opts: opts, - nowFn: opts.ClockOptions().NowFn(), + opts: opts, + nowFn: opts.ClockOptions().NowFn(), shardCutoverWarmupDuration: opts.ShardCutoverWarmupDuration(), shardCutoffLingerDuration: opts.ShardCutoffLingerDuration(), writerMgr: writerMgr, @@ -256,6 +262,23 @@ func (c *client) WriteTimed( return err } +func (c *client) WriteTimedWithStagedMetadatas( + metric aggregated.Metric, + metadatas metadata.StagedMetadatas, +) error { + callStart := c.nowFn() + payload := payloadUnion{ + payloadType: timedWithStagedMetadatasType, + timedWithStagedMetadatas: timedWithStagedMetadatas{ + metric: metric, + metadatas: metadatas, + }, + } + err := c.write(metric.ID, metric.TimeNanos, payload) + c.metrics.writeForwarded.ReportSuccessOrError(err, c.nowFn().Sub(callStart)) + return err +} + func (c *client) WriteForwarded( metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata, diff --git a/src/aggregator/client/client_mock.go b/src/aggregator/client/client_mock.go index edadd51a6f..a36674b14b 100644 --- a/src/aggregator/client/client_mock.go +++ b/src/aggregator/client/client_mock.go @@ -1,7 +1,7 @@ // Code generated by MockGen. DO NOT EDIT. // Source: github.com/m3db/m3/src/aggregator/client (interfaces: Client,AdminClient) -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -113,6 +113,20 @@ func (mr *MockClientMockRecorder) WriteTimed(arg0, arg1 interface{}) *gomock.Cal return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTimed", reflect.TypeOf((*MockClient)(nil).WriteTimed), arg0, arg1) } +// WriteTimedWithStagedMetadatas mocks base method +func (m *MockClient) WriteTimedWithStagedMetadatas(arg0 aggregated.Metric, arg1 metadata.StagedMetadatas) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WriteTimedWithStagedMetadatas", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// WriteTimedWithStagedMetadatas indicates an expected call of WriteTimedWithStagedMetadatas +func (mr *MockClientMockRecorder) WriteTimedWithStagedMetadatas(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTimedWithStagedMetadatas", reflect.TypeOf((*MockClient)(nil).WriteTimedWithStagedMetadatas), arg0, arg1) +} + // WriteUntimedBatchTimer mocks base method func (m *MockClient) WriteUntimedBatchTimer(arg0 unaggregated.BatchTimer, arg1 metadata.StagedMetadatas) error { m.ctrl.T.Helper() @@ -248,6 +262,20 @@ func (mr *MockAdminClientMockRecorder) WriteTimed(arg0, arg1 interface{}) *gomoc return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTimed", reflect.TypeOf((*MockAdminClient)(nil).WriteTimed), arg0, arg1) } +// WriteTimedWithStagedMetadatas mocks base method +func (m *MockAdminClient) WriteTimedWithStagedMetadatas(arg0 aggregated.Metric, arg1 metadata.StagedMetadatas) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WriteTimedWithStagedMetadatas", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// WriteTimedWithStagedMetadatas indicates an expected call of WriteTimedWithStagedMetadatas +func (mr *MockAdminClientMockRecorder) WriteTimedWithStagedMetadatas(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTimedWithStagedMetadatas", reflect.TypeOf((*MockAdminClient)(nil).WriteTimedWithStagedMetadatas), arg0, arg1) +} + // WriteUntimedBatchTimer mocks base method func (m *MockAdminClient) WriteUntimedBatchTimer(arg0 unaggregated.BatchTimer, arg1 metadata.StagedMetadatas) error { m.ctrl.T.Helper() diff --git a/src/aggregator/client/payload.go b/src/aggregator/client/payload.go index dc66b37172..7e24a793e3 100644 --- a/src/aggregator/client/payload.go +++ b/src/aggregator/client/payload.go @@ -34,6 +34,7 @@ const ( untimedType forwardedType timedType + timedWithStagedMetadatasType ) type untimedPayload struct { @@ -51,9 +52,15 @@ type timedPayload struct { metadata metadata.TimedMetadata } +type timedWithStagedMetadatas struct { + metric aggregated.Metric + metadatas metadata.StagedMetadatas +} + type payloadUnion struct { - payloadType payloadType - untimed untimedPayload - forwarded forwardedPayload - timed timedPayload + payloadType payloadType + untimed untimedPayload + forwarded forwardedPayload + timed timedPayload + timedWithStagedMetadatas timedWithStagedMetadatas } diff --git a/src/aggregator/client/writer.go b/src/aggregator/client/writer.go index 279158af6b..266470e3d7 100644 --- a/src/aggregator/client/writer.go +++ b/src/aggregator/client/writer.go @@ -164,6 +164,9 @@ func (w *writer) encodeWithLock( return w.encodeForwardedWithLock(encoder, payload.forwarded.metric, payload.forwarded.metadata) case timedType: return w.encodeTimedWithLock(encoder, payload.timed.metric, payload.timed.metadata) + case timedWithStagedMetadatasType: + elem := payload.timedWithStagedMetadatas + return w.encodeTimedWithStagedMetadatasWithLock(encoder, elem.metric, elem.metadatas) default: return fmt.Errorf("unknown payload type: %v", payload.payloadType) } @@ -374,6 +377,45 @@ func (w *writer) encodeTimedWithLock( return w.enqueueBuffer(buffer) } +func (w *writer) encodeTimedWithStagedMetadatasWithLock( + encoder *lockedEncoder, + metric aggregated.Metric, + metadatas metadata.StagedMetadatas, +) error { + encoder.Lock() + + sizeBefore := encoder.Len() + msg := encoding.UnaggregatedMessageUnion{ + Type: encoding.TimedMetricWithMetadatasType, + TimedMetricWithMetadatas: aggregated.TimedMetricWithMetadatas{ + Metric: metric, + StagedMetadatas: metadatas, + }} + if err := encoder.EncodeMessage(msg); err != nil { + w.log.Error("encode timed metric error", + zap.Any("metric", metric), + zap.Any("metadatas", metadatas), + zap.Error(err), + ) + // Rewind buffer and clear out the encoder error. + encoder.Truncate(sizeBefore) + encoder.Unlock() + w.metrics.encodeErrors.Inc(1) + return err + } + + // If the buffer size is not big enough, do nothing. + if sizeAfter := encoder.Len(); sizeAfter < w.flushSize { + encoder.Unlock() + return nil + } + + // Otherwise we enqueue the current buffer. + buffer := w.prepareEnqueueBufferWithLock(encoder, sizeBefore) + encoder.Unlock() + return w.enqueueBuffer(buffer) +} + // prepareEnqueueBufferWithLock prepares the writer to enqueue a // buffer onto its instance queue. It gets a new buffer from pool, // copies the bytes exceeding sizeBefore to it, resets the encoder diff --git a/src/aggregator/config/m3aggregator.yml b/src/aggregator/config/m3aggregator.yml index 46ffd43f85..b47dc791fb 100644 --- a/src/aggregator/config/m3aggregator.yml +++ b/src/aggregator/config/m3aggregator.yml @@ -320,7 +320,7 @@ aggregator: readBufferSize: 256 forwarding: maxSingleDelay: 5s - entryTTL: 6h + entryTTL: 1h entryCheckInterval: 10m maxTimerBatchSizePerWrite: 140 defaultStoragePolicies: diff --git a/src/aggregator/integration/integration_data.go b/src/aggregator/integration/integration_data.go index 116a404346..c05a275294 100644 --- a/src/aggregator/integration/integration_data.go +++ b/src/aggregator/integration/integration_data.go @@ -427,7 +427,7 @@ func addUntimedMetricToAggregation( return v, nil case metric.TimerType: v := values.(aggregation.Timer) - v.AddBatch(mu.BatchTimerVal) + v.AddBatch(time.Now(), mu.BatchTimerVal) return v, nil case metric.GaugeType: v := values.(aggregation.Gauge) @@ -449,7 +449,7 @@ func addTimedMetricToAggregation( return v, nil case metric.TimerType: v := values.(aggregation.Timer) - v.AddBatch([]float64{mu.Value}) + v.AddBatch(time.Now(), []float64{mu.Value}) return v, nil case metric.GaugeType: v := values.(aggregation.Gauge) @@ -473,7 +473,7 @@ func addForwardedMetricToAggregation( return v, nil case metric.TimerType: v := values.(aggregation.Timer) - v.AddBatch(mu.Values) + v.AddBatch(time.Now(), mu.Values) return v, nil case metric.GaugeType: v := values.(aggregation.Gauge) diff --git a/src/aggregator/server/rawtcp/server.go b/src/aggregator/server/rawtcp/server.go index adca4872cd..cb6f4d9ba1 100644 --- a/src/aggregator/server/rawtcp/server.go +++ b/src/aggregator/server/rawtcp/server.go @@ -155,6 +155,10 @@ func (s *handler) Handle(conn net.Conn) { timedMetric = current.TimedMetricWithMetadata.Metric timedMetadata = current.TimedMetricWithMetadata.TimedMetadata err = toAddTimedError(s.aggregator.AddTimed(timedMetric, timedMetadata)) + case encoding.TimedMetricWithMetadatasType: + timedMetric = current.TimedMetricWithMetadatas.Metric + stagedMetadatas = current.TimedMetricWithMetadatas.StagedMetadatas + err = toAddTimedError(s.aggregator.AddTimedWithStagedMetadatas(timedMetric, stagedMetadatas)) default: err = newUnknownMessageTypeError(current.Type) } diff --git a/src/cmd/services/m3aggregator/config/aggregator.go b/src/cmd/services/m3aggregator/config/aggregator.go index 7773b1b9c4..8e18e28159 100644 --- a/src/cmd/services/m3aggregator/config/aggregator.go +++ b/src/cmd/services/m3aggregator/config/aggregator.go @@ -65,6 +65,10 @@ type AggregatorConfiguration struct { // InstanceID is the instance ID configuration. InstanceID InstanceIDConfiguration `yaml:"instanceID"` + // VerboseErrors sets whether or not to use verbose errors when + // value arrives too early, late, or other bad request like operation. + VerboseErrors bool `yaml:"verboseErrors"` + // AggregationTypes configs the aggregation types. AggregationTypes aggregation.TypesConfiguration `yaml:"aggregationTypes"` @@ -135,7 +139,7 @@ type AggregatorConfiguration struct { MaxTimerBatchSizePerWrite int `yaml:"maxTimerBatchSizePerWrite" validate:"min=0"` // Default storage policies. - DefaultStoragePolicies []policy.StoragePolicy `yaml:"defaultStoragePolicies" validate:"nonzero"` + DefaultStoragePolicies []policy.StoragePolicy `yaml:"defaultStoragePolicies"` // Maximum number of cached source sets. MaxNumCachedSourceSets *int `yaml:"maxNumCachedSourceSets"` @@ -237,7 +241,8 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( ) (aggregator.Options, error) { opts := aggregator.NewOptions(). SetInstrumentOptions(instrumentOpts). - SetRuntimeOptionsManager(runtimeOptsManager) + SetRuntimeOptionsManager(runtimeOptsManager). + SetVerboseErrors(c.VerboseErrors) // Set the aggregation types options. aggTypesOpts, err := c.AggregationTypes.NewOptions(instrumentOpts) @@ -565,12 +570,20 @@ func (c placementManagerConfiguration) NewPlacementManager( type forwardingConfiguration struct { // MaxSingleDelay is the maximum delay for a single forward step. MaxSingleDelay time.Duration `yaml:"maxSingleDelay"` + // MaxConstDelay is the maximum delay for a forward step as a constant + resolution*numForwardedTimes. + MaxConstDelay time.Duration `yaml:"maxConstDelay"` } func (c forwardingConfiguration) MaxAllowedForwardingDelayFn( jitterEnabled bool, maxJitterFn aggregator.FlushJitterFn, ) aggregator.MaxAllowedForwardingDelayFn { + if v := c.MaxConstDelay; v > 0 { + return func(resolution time.Duration, numForwardedTimes int) time.Duration { + return v + (resolution * time.Duration(numForwardedTimes)) + } + } + return func(resolution time.Duration, numForwardedTimes int) time.Duration { // If jittering is enabled, we use max jitter fn to determine the initial jitter. // Otherwise, flushing may start at any point within a resolution interval so we diff --git a/src/cmd/services/m3coordinator/downsample/downsampler.go b/src/cmd/services/m3coordinator/downsample/downsampler.go index aa57564ff0..e947575c44 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler.go @@ -98,14 +98,14 @@ func newDownsampler(opts downsamplerOptions) (*downsampler, error) { func (d *downsampler) NewMetricsAppender() (MetricsAppender, error) { return newMetricsAppender(metricsAppenderOptions{ - agg: d.agg.aggregator, - clientRemote: d.agg.clientRemote, - defaultStagedMetadatas: d.agg.defaultStagedMetadatas, - clockOpts: d.agg.clockOpts, - tagEncoder: d.agg.pools.tagEncoderPool.Get(), - matcher: d.agg.matcher, - metricTagsIteratorPool: d.agg.pools.metricTagsIteratorPool, - debugLogging: d.debugLogging, - logger: d.logger, + agg: d.agg.aggregator, + clientRemote: d.agg.clientRemote, + defaultStagedMetadatasProtos: d.agg.defaultStagedMetadatasProtos, + clockOpts: d.agg.clockOpts, + tagEncoder: d.agg.pools.tagEncoderPool.Get(), + matcher: d.agg.matcher, + metricTagsIteratorPool: d.agg.pools.metricTagsIteratorPool, + debugLogging: d.debugLogging, + logger: d.logger, }), nil } diff --git a/src/cmd/services/m3coordinator/downsample/downsampler_test.go b/src/cmd/services/m3coordinator/downsample/downsampler_test.go index 23d74efd7d..49111f853b 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler_test.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler_test.go @@ -41,6 +41,7 @@ import ( "github.com/m3db/m3/src/metrics/rules" ruleskv "github.com/m3db/m3/src/metrics/rules/store/kv" "github.com/m3db/m3/src/metrics/rules/view" + "github.com/m3db/m3/src/metrics/transformation" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/storage/mock" @@ -160,8 +161,8 @@ func TestDownsamplerAggregationWithRulesConfigMappingRules(t *testing.T) { expect: &testDownsamplerOptionsExpect{ writes: []testExpectedWrite{ { - tags: gaugeMetric.tags, - value: 30, + tags: gaugeMetric.tags, + values: []expectedValue{{value: 30}}, attributes: &storage.Attributes{ MetricsType: storage.AggregatedMetricsType, Resolution: 5 * time.Second, @@ -218,8 +219,8 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesPartialReplaceAutoMapp // Expect the max to be used and override the default auto // mapping rule for the storage policy 2s:24h. { - tags: gaugeMetric.tags, - value: 30, + tags: gaugeMetric.tags, + values: []expectedValue{{value: 30}}, attributes: &storage.Attributes{ MetricsType: storage.AggregatedMetricsType, Resolution: 2 * time.Second, @@ -229,8 +230,8 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesPartialReplaceAutoMapp // Expect the sum to still be used for the storage // policy 4s:48h. { - tags: gaugeMetric.tags, - value: 60, + tags: gaugeMetric.tags, + values: []expectedValue{{value: 60}}, attributes: &storage.Attributes{ MetricsType: storage.AggregatedMetricsType, Resolution: 4 * time.Second, @@ -286,8 +287,8 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesReplaceAutoMappingRule // Expect the max to be used and override the default auto // mapping rule for the storage policy 2s:24h. { - tags: gaugeMetric.tags, - value: 30, + tags: gaugeMetric.tags, + values: []expectedValue{{value: 30}}, attributes: &storage.Attributes{ MetricsType: storage.AggregatedMetricsType, Resolution: 2 * time.Second, @@ -302,7 +303,7 @@ func TestDownsamplerAggregationWithRulesConfigMappingRulesReplaceAutoMappingRule testDownsamplerAggregation(t, testDownsampler) } -func TestDownsamplerAggregationWithRulesConfigRollupRules(t *testing.T) { +func TestDownsamplerAggregationWithRulesConfigRollupRulesPerSecondSum(t *testing.T) { gaugeMetric := testGaugeMetric{ tags: map[string]string{ nameTag: "http_requests", @@ -311,18 +312,15 @@ func TestDownsamplerAggregationWithRulesConfigRollupRules(t *testing.T) { "endpoint": "/foo/bar", "not_rolled_up": "not_rolled_up_value", }, - samples: []float64{42, 64}, - // TODO: Make rollup rules work with timestamped samples (like below) - // instead of only with untimed samples (this requires being able to - // write staged metadatas instead of a single storage policy for a - // timed metric). - // timedSamples: []testGaugeMetricTimedSample{ - // {value: 42}, {value: 64}, - // }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 42}, + {value: 64, offset: 5 * time.Second}, + }, } res := 5 * time.Second ret := 30 * 24 * time.Hour testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + autoMappingRules: []AutoMappingRule{}, rulesConfig: &RulesConfiguration{ RollupRules: []RollupRuleConfiguration{ { @@ -330,15 +328,11 @@ func TestDownsamplerAggregationWithRulesConfigRollupRules(t *testing.T) { "%s:http_requests app:* status_code:* endpoint:*", nameTag), Transforms: []TransformConfiguration{ - // TODO: make multi-stage rollup rules work, for some reason - // when multiple transforms applied the HasRollup detection - // fails and hence metric is not forwarded for second stage - // aggregation. - // { - // Transform: &TransformOperationConfiguration{ - // Type: transformation.PerSecond, - // }, - // }, + { + Transform: &TransformOperationConfiguration{ + Type: transformation.PerSecond, + }, + }, { Rollup: &RollupOperationConfiguration{ MetricName: "http_requests_by_status_code", @@ -369,7 +363,109 @@ func TestDownsamplerAggregationWithRulesConfigRollupRules(t *testing.T) { "status_code": "500", "endpoint": "/foo/bar", }, - value: 106, + values: []expectedValue{{value: 4.4}}, + attributes: &storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Resolution: res, + Retention: ret, + }, + }, + }, + }, + }) + + // Test expected output + testDownsamplerAggregation(t, testDownsampler) +} + +func TestDownsamplerAggregationWithRulesConfigRollupRulesIncreaseAdd(t *testing.T) { + gaugeMetrics := []testGaugeMetric{ + 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", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 42, offset: 5 * time.Second}, // +42 (should not be accounted since is a reset) + // Explicit no value. + {value: 12, offset: 15 * time.Second}, // +12 - simulate a reset (should not be accounted) + {value: 33, offset: 20 * time.Second}, // +21 + }, + }, + testGaugeMetric{ + tags: map[string]string{ + nameTag: "http_requests", + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + "not_rolled_up": "not_rolled_up_value_2", + }, + timedSamples: []testGaugeMetricTimedSample{ + {value: 13, offset: 5 * time.Second}, // +13 (should not be accounted since is a reset) + {value: 27, offset: 10 * time.Second}, // +14 + // Explicit no value. + {value: 42, offset: 20 * time.Second}, // +15 + }, + }, + } + res := 5 * time.Second + ret := 30 * 24 * time.Hour + testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ + autoMappingRules: []AutoMappingRule{}, + rulesConfig: &RulesConfiguration{ + RollupRules: []RollupRuleConfiguration{ + { + 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"}, + Aggregations: []aggregation.Type{aggregation.Sum}, + }, + }, + { + Transform: &TransformOperationConfiguration{ + Type: transformation.Add, + }, + }, + }, + StoragePolicies: []StoragePolicyConfiguration{ + { + Resolution: res, + Retention: ret, + }, + }, + }, + }, + }, + ingest: &testDownsamplerOptionsIngest{ + gaugeMetrics: gaugeMetrics, + }, + expect: &testDownsamplerOptionsExpect{ + writes: []testExpectedWrite{ + { + tags: map[string]string{ + nameTag: "http_requests_by_status_code", + string(rollupTagName): string(rollupTagValue), + "app": "nginx_edge", + "status_code": "500", + "endpoint": "/foo/bar", + }, + values: []expectedValue{ + {value: 14}, + {value: 50, offset: 10 * time.Second}, + }, attributes: &storage.Attributes{ MetricsType: storage.AggregatedMetricsType, Resolution: res, @@ -413,10 +509,10 @@ func TestDownsamplerAggregationWithTimedSamples(t *testing.T) { func TestDownsamplerAggregationWithOverrideRules(t *testing.T) { counterMetrics, counterMetricsExpect := testCounterMetrics(testCounterMetricsOptions{}) - counterMetricsExpect[0].value = 2 + counterMetricsExpect[0].values = []expectedValue{{value: 2}} gaugeMetrics, gaugeMetricsExpect := testGaugeMetrics(testGaugeMetricsOptions{}) - gaugeMetricsExpect[0].value = 5 + gaugeMetricsExpect[0].values = []expectedValue{{value: 5}} testDownsampler := newTestDownsampler(t, testDownsamplerOptions{ sampleAppenderOpts: &SampleAppenderOptions{ @@ -474,9 +570,15 @@ func TestDownsamplerAggregationWithRemoteAggregatorClient(t *testing.T) { } type testExpectedWrite struct { - tags map[string]string - value float64 - attributes *storage.Attributes + tags map[string]string + values []expectedValue // use values for multi expected values + valueAllowedError float64 // use for allowing for slightly inexact values due to timing, etc + attributes *storage.Attributes +} + +type expectedValue struct { + offset time.Duration + value float64 } type testCounterMetric struct { @@ -486,8 +588,9 @@ type testCounterMetric struct { } type testCounterMetricTimedSample struct { - time time.Time - value int64 + time time.Time + offset time.Duration + value int64 } type testGaugeMetric struct { @@ -497,8 +600,9 @@ type testGaugeMetric struct { } type testGaugeMetricTimedSample struct { - time time.Time - value float64 + time time.Time + offset time.Duration + value float64 } type testCounterMetricsOptions struct { @@ -520,8 +624,8 @@ func testCounterMetrics(opts testCounterMetricsOptions) ( } } write := testExpectedWrite{ - tags: metric.tags, - value: 6, + tags: metric.tags, + values: []expectedValue{{value: 6}}, } return []testCounterMetric{metric}, []testExpectedWrite{write} } @@ -542,8 +646,8 @@ func testGaugeMetrics(opts testGaugeMetricsOptions) ([]testGaugeMetric, []testEx } } write := testExpectedWrite{ - tags: metric.tags, - value: 15, + tags: metric.tags, + values: []expectedValue{{value: 15}}, } return []testGaugeMetric{metric}, []testExpectedWrite{write} } @@ -578,18 +682,22 @@ func testDownsamplerAggregation( logger.Info("wait for test metrics to appear") logWritesAccumulated := os.Getenv("TEST_LOG_WRITES_ACCUMULATED") == "true" logWritesAccumulatedTicker := time.NewTicker(time.Second) + + logWritesMatch := os.Getenv("TEST_LOG_WRITES_MATCH") == "true" + logWritesMatchTicker := time.NewTicker(time.Second) CheckAllWritesArrivedLoop: for { - writes := testDownsampler.storage.Writes() + allWrites := testDownsampler.storage.Writes() if logWritesAccumulated { select { case <-logWritesAccumulatedTicker.C: logger.Info("logging accmulated writes", - zap.Int("numWrites", len(writes))) - for _, write := range writes { + zap.Int("numAllWrites", len(allWrites))) + for _, write := range allWrites { logger.Info("accumulated write", zap.ByteString("tags", write.Tags.ID()), - zap.Any("datapoints", write.Datapoints)) + zap.Any("datapoints", write.Datapoints), + zap.Any("attributes", write.Attributes)) } default: } @@ -597,8 +705,22 @@ CheckAllWritesArrivedLoop: for _, expectedWrite := range expectedWrites { name := expectedWrite.tags[nameTag] - _, ok := findWrite(t, writes, name, expectedWrite.attributes) - if !ok { + attrs := expectedWrite.attributes + writesForNameAndAttrs, _ := findWrites(t, allWrites, name, attrs) + if len(writesForNameAndAttrs) != len(expectedWrite.values) { + if logWritesMatch { + select { + case <-logWritesMatchTicker.C: + logger.Info("continuing wait for accumulated writes", + zap.String("name", name), + zap.Any("attributes", attrs), + zap.Int("numWritesForNameAndAttrs", len(writesForNameAndAttrs)), + zap.Int("numExpectedWriteValues", len(expectedWrite.values)), + ) + default: + } + } + time.Sleep(100 * time.Millisecond) continue CheckAllWritesArrivedLoop } @@ -608,11 +730,11 @@ CheckAllWritesArrivedLoop: // Verify writes logger.Info("verify test metrics") - writes := testDownsampler.storage.Writes() + allWrites := testDownsampler.storage.Writes() if logWritesAccumulated { logger.Info("logging accmulated writes to verify", - zap.Int("numWrites", len(writes))) - for _, write := range writes { + zap.Int("numAllWrites", len(allWrites))) + for _, write := range allWrites { logger.Info("accumulated write", zap.ByteString("tags", write.Tags.ID()), zap.Any("datapoints", write.Datapoints)) @@ -621,16 +743,44 @@ CheckAllWritesArrivedLoop: for _, expectedWrite := range expectedWrites { name := expectedWrite.tags[nameTag] - value := expectedWrite.value + expectedValues := expectedWrite.values + allowedError := expectedWrite.valueAllowedError - write, found := findWrite(t, writes, name, expectedWrite.attributes) + writesForNameAndAttrs, found := findWrites(t, allWrites, name, expectedWrite.attributes) require.True(t, found) - assert.Equal(t, expectedWrite.tags, tagsToStringMap(write.Tags)) - require.Equal(t, 1, len(write.Datapoints)) - assert.Equal(t, float64(value), write.Datapoints[0].Value) + require.Equal(t, len(expectedValues), len(writesForNameAndAttrs)) + for i, expectedValue := range expectedValues { + write := writesForNameAndAttrs[i] + + assert.Equal(t, expectedWrite.tags, tagsToStringMap(write.Tags)) + + require.Equal(t, 1, len(write.Datapoints)) + + actualValue := write.Datapoints[0].Value + if allowedError == 0 { + // Exact match value. + assert.Equal(t, expectedValue.value, actualValue) + } else { + // Fuzzy match value. + lower := expectedValue.value - allowedError + upper := expectedValue.value + allowedError + withinBounds := (lower <= actualValue) && (actualValue <= upper) + msg := fmt.Sprintf("expected within: lower=%f, upper=%f, actual=%f", + lower, upper, actualValue) + assert.True(t, withinBounds, msg) + } + + if expectedOffset := expectedValue.offset; expectedOffset > 0 { + // Check if distance between datapoints as expected (use + // absolute offset from first write). + firstTimestamp := writesForNameAndAttrs[0].Datapoints[0].Timestamp + actualOffset := write.Datapoints[0].Timestamp.Sub(firstTimestamp) + assert.Equal(t, expectedOffset, actualOffset) + } - if attrs := expectedWrite.attributes; attrs != nil { - assert.Equal(t, *attrs, write.Attributes) + if attrs := expectedWrite.attributes; attrs != nil { + assert.Equal(t, *attrs, write.Attributes) + } } } } @@ -768,9 +918,12 @@ func testDownsamplerAggregationIngest( require.NoError(t, err) } for _, sample := range metric.timedSamples { - if sample.time.Equal(time.Time{}) { + if sample.time.IsZero() { sample.time = time.Now() // Allow empty time to mean "now" } + if sample.offset > 0 { + sample.time = sample.time.Add(sample.offset) + } err = samplesAppender.AppendCounterTimedSample(sample.time, sample.value) require.NoError(t, err) } @@ -789,9 +942,12 @@ func testDownsamplerAggregationIngest( require.NoError(t, err) } for _, sample := range metric.timedSamples { - if sample.time.Equal(time.Time{}) { + if sample.time.IsZero() { sample.time = time.Now() // Allow empty time to mean "now" } + if sample.offset > 0 { + sample.time = sample.time.Add(sample.offset) + } err = samplesAppender.AppendGaugeTimedSample(sample.time, sample.value) require.NoError(t, err) } @@ -950,12 +1106,13 @@ func newTestID(t *testing.T, tags map[string]string) id.ID { return iter } -func findWrite( +func findWrites( t *testing.T, writes []*storage.WriteQuery, name string, optionalMatchAttrs *storage.Attributes, -) (*storage.WriteQuery, bool) { +) ([]*storage.WriteQuery, bool) { + var results []*storage.WriteQuery for _, w := range writes { if t, ok := w.Tags.Get([]byte(nameTag)); ok { if !bytes.Equal(t, []byte(name)) { @@ -966,11 +1123,12 @@ func findWrite( // Tried to match attributes and not matched. continue } + // Matches name and all optional lookups. - return w, true + results = append(results, w) } } - return nil, false + return results, len(results) > 0 } func testUpdateMetadata() rules.UpdateMetadata { diff --git a/src/cmd/services/m3coordinator/downsample/metrics_appender.go b/src/cmd/services/m3coordinator/downsample/metrics_appender.go index 3bde367251..d7b3e3baeb 100644 --- a/src/cmd/services/m3coordinator/downsample/metrics_appender.go +++ b/src/cmd/services/m3coordinator/downsample/metrics_appender.go @@ -44,8 +44,11 @@ import ( type metricsAppender struct { metricsAppenderOptions - tags *tags - multiSamplesAppender *multiSamplesAppender + tags *tags + multiSamplesAppender *multiSamplesAppender + currStagedMetadata metadata.StagedMetadata + defaultStagedMetadatasCopies []metadata.StagedMetadatas + mappingRuleStoragePolicies []policy.StoragePolicy } // metricsAppenderOptions will have one of agg or clientRemote set. @@ -53,12 +56,10 @@ type metricsAppenderOptions struct { agg aggregator.Aggregator clientRemote client.Client - defaultStagedMetadatas []metadata.StagedMetadatas - tagEncoder serialize.TagEncoder - matcher matcher.Matcher - metricTagsIteratorPool serialize.MetricTagsIteratorPool - - mappingRuleStoragePolicies []policy.StoragePolicy + defaultStagedMetadatasProtos []metricpb.StagedMetadatas + tagEncoder serialize.TagEncoder + matcher matcher.Matcher + metricTagsIteratorPool serialize.MetricTagsIteratorPool clockOpts clock.Options debugLogging bool @@ -66,10 +67,13 @@ type metricsAppenderOptions struct { } func newMetricsAppender(opts metricsAppenderOptions) *metricsAppender { + stagedMetadatasCopies := make([]metadata.StagedMetadatas, + len(opts.defaultStagedMetadatasProtos)) return &metricsAppender{ - metricsAppenderOptions: opts, - tags: newTags(), - multiSamplesAppender: newMultiSamplesAppender(), + metricsAppenderOptions: opts, + tags: newTags(), + multiSamplesAppender: newMultiSamplesAppender(), + defaultStagedMetadatasCopies: stagedMetadatasCopies, } } @@ -106,6 +110,9 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp id.Close() if opts.Override { + // Reuse a slice to keep the current staged metadatas we will apply. + a.currStagedMetadata.Pipelines = a.currStagedMetadata.Pipelines[:0] + for _, rule := range opts.OverrideRules.MappingRules { stagedMetadatas, err := rule.StagedMetadatas() if err != nil { @@ -115,14 +122,21 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp a.debugLogMatch("downsampler applying override mapping rule", debugLogMatchOptions{Meta: stagedMetadatas}) - a.multiSamplesAppender.addSamplesAppender(samplesAppender{ - agg: a.agg, - clientRemote: a.clientRemote, - unownedID: unownedID, - stagedMetadatas: stagedMetadatas, - }) + pipelines := stagedMetadatas[len(stagedMetadatas)-1] + a.currStagedMetadata.Pipelines = + append(a.currStagedMetadata.Pipelines, pipelines.Pipelines...) } + + a.multiSamplesAppender.addSamplesAppender(samplesAppender{ + agg: a.agg, + clientRemote: a.clientRemote, + unownedID: unownedID, + stagedMetadatas: []metadata.StagedMetadata{a.currStagedMetadata}, + }) } else { + // Reuse a slice to keep the current staged metadatas we will apply. + a.currStagedMetadata.Pipelines = a.currStagedMetadata.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 @@ -131,13 +145,13 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp // name and tags (i.e. overwriting each other). a.mappingRuleStoragePolicies = a.mappingRuleStoragePolicies[:0] - stagedMetadatas := matchResult.ForExistingIDAt(nowNanos) - if !stagedMetadatas.IsDefault() && len(stagedMetadatas) != 0 { + mappingRuleStagedMetadatas := matchResult.ForExistingIDAt(nowNanos) + if !mappingRuleStagedMetadatas.IsDefault() && len(mappingRuleStagedMetadatas) != 0 { a.debugLogMatch("downsampler applying matched mapping rule", - debugLogMatchOptions{Meta: stagedMetadatas}) + debugLogMatchOptions{Meta: mappingRuleStagedMetadatas}) // Collect all the current active mapping rules - for _, stagedMetadata := range stagedMetadatas { + for _, stagedMetadata := range mappingRuleStagedMetadatas { for _, pipe := range stagedMetadata.Pipelines { for _, sp := range pipe.StoragePolicies { a.mappingRuleStoragePolicies = @@ -147,20 +161,26 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp } // Only sample if going to actually aggregate - a.multiSamplesAppender.addSamplesAppender(samplesAppender{ - agg: a.agg, - clientRemote: a.clientRemote, - unownedID: unownedID, - stagedMetadatas: stagedMetadatas, - }) + pipelines := mappingRuleStagedMetadatas[len(mappingRuleStagedMetadatas)-1] + a.currStagedMetadata.Pipelines = + append(a.currStagedMetadata.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). - for _, stagedMetadatas := range a.defaultStagedMetadatas { - a.debugLogMatch("downsampler applying default mapping rule", - debugLogMatchOptions{Meta: stagedMetadatas}) + for idx, stagedMetadatasProto := range a.defaultStagedMetadatasProtos { + // NB(r): Need to take copy of default staged metadatas as we + // sometimes mutate it. + stagedMetadatas := a.defaultStagedMetadatasCopies[idx] + err := stagedMetadatas.FromProto(stagedMetadatasProto) + if err != nil { + return nil, + fmt.Errorf("unable to copy default staged metadatas: %v", err) + } + + // Save the staged metadatas back to the idx so all slices can be reused. + a.defaultStagedMetadatasCopies[idx] = stagedMetadatas stagedMetadataBeforeFilter := stagedMetadatas[:] if len(a.mappingRuleStoragePolicies) != 0 { @@ -223,11 +243,25 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp continue } + a.debugLogMatch("downsampler applying default mapping rule", + debugLogMatchOptions{Meta: stagedMetadatas}) + + pipelines := stagedMetadatas[len(stagedMetadatas)-1] + a.currStagedMetadata.Pipelines = + append(a.currStagedMetadata.Pipelines, pipelines.Pipelines...) + } + + if len(a.currStagedMetadata.Pipelines) > 0 { + // Send to downsampler if we have something in the pipeline. + mappingStagedMetadatas := []metadata.StagedMetadata{a.currStagedMetadata} + a.debugLogMatch("downsampler using built mapping staged metadatas", + debugLogMatchOptions{Meta: mappingStagedMetadatas}) + a.multiSamplesAppender.addSamplesAppender(samplesAppender{ agg: a.agg, clientRemote: a.clientRemote, unownedID: unownedID, - stagedMetadatas: stagedMetadatas, + stagedMetadatas: mappingStagedMetadatas, }) } @@ -236,7 +270,7 @@ func (a *metricsAppender) SamplesAppender(opts SampleAppenderOptions) (SamplesAp rollup := matchResult.ForNewRollupIDsAt(i, nowNanos) a.debugLogMatch("downsampler applying matched rollup rule", - debugLogMatchOptions{Meta: stagedMetadatas, RollupID: rollup.ID}) + debugLogMatchOptions{Meta: rollup.Metadatas, RollupID: rollup.ID}) a.multiSamplesAppender.addSamplesAppender(samplesAppender{ agg: a.agg, diff --git a/src/cmd/services/m3coordinator/downsample/options.go b/src/cmd/services/m3coordinator/downsample/options.go index fdae1af32c..2bba042ac5 100644 --- a/src/cmd/services/m3coordinator/downsample/options.go +++ b/src/cmd/services/m3coordinator/downsample/options.go @@ -39,6 +39,7 @@ import ( "github.com/m3db/m3/src/metrics/aggregation" "github.com/m3db/m3/src/metrics/filters" "github.com/m3db/m3/src/metrics/generated/proto/aggregationpb" + "github.com/m3db/m3/src/metrics/generated/proto/metricpb" "github.com/m3db/m3/src/metrics/generated/proto/pipelinepb" "github.com/m3db/m3/src/metrics/generated/proto/rulepb" "github.com/m3db/m3/src/metrics/generated/proto/transformationpb" @@ -177,10 +178,10 @@ type agg struct { aggregator aggregator.Aggregator clientRemote client.Client - defaultStagedMetadatas []metadata.StagedMetadatas - clockOpts clock.Options - matcher matcher.Matcher - pools aggPools + defaultStagedMetadatasProtos []metricpb.StagedMetadatas + clockOpts clock.Options + matcher matcher.Matcher + pools aggPools } // Configuration configurates a downsampler. @@ -549,13 +550,13 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { } var ( - storageFlushConcurrency = defaultStorageFlushConcurrency - clockOpts = o.ClockOptions - instrumentOpts = o.InstrumentOptions - scope = instrumentOpts.MetricsScope() - logger = instrumentOpts.Logger() - openTimeout = defaultOpenTimeout - defaultStagedMetadatas []metadata.StagedMetadatas + storageFlushConcurrency = defaultStorageFlushConcurrency + clockOpts = o.ClockOptions + instrumentOpts = o.InstrumentOptions + scope = instrumentOpts.MetricsScope() + logger = instrumentOpts.Logger() + openTimeout = defaultOpenTimeout + defaultStagedMetadatasProtos []metricpb.StagedMetadatas ) if o.StorageFlushConcurrency > 0 { storageFlushConcurrency = o.StorageFlushConcurrency @@ -568,7 +569,14 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { if err != nil { return agg{}, err } - defaultStagedMetadatas = append(defaultStagedMetadatas, metadatas) + + var metadatasProto metricpb.StagedMetadatas + if err := metadatas.ToProto(&metadatasProto); err != nil { + return agg{}, err + } + + defaultStagedMetadatasProtos = + append(defaultStagedMetadatasProtos, metadatasProto) } pools := o.newAggregatorPools() @@ -670,10 +678,10 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { } return agg{ - clientRemote: client, - defaultStagedMetadatas: defaultStagedMetadatas, - matcher: matcher, - pools: pools, + clientRemote: client, + defaultStagedMetadatasProtos: defaultStagedMetadatasProtos, + matcher: matcher, + pools: pools, }, nil } @@ -719,10 +727,15 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { return bufferForPastTimedMetric(bufferPastLimits, tile) } + maxAllowedForwardingDelayFn := func(tile time.Duration, numForwardedTimes int) time.Duration { + return maxAllowedForwardingDelay(bufferPastLimits, tile, numForwardedTimes) + } + // Finally construct all options. aggregatorOpts := aggregator.NewOptions(). SetClockOptions(clockOpts). SetInstrumentOptions(instrumentOpts). + SetDefaultStoragePolicies(nil). SetMetricPrefix(nil). SetCounterPrefix(nil). SetGaugePrefix(nil). @@ -734,6 +747,7 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { SetFlushHandler(flushHandler). SetBufferForPastTimedMetricFn(bufferForPastTimedMetricFn). SetBufferForFutureTimedMetric(defaultBufferFutureTimedMetric). + SetMaxAllowedForwardingDelayFn(maxAllowedForwardingDelayFn). SetVerboseErrors(defaultVerboseErrors) if cfg.EntryTTL != 0 { @@ -829,10 +843,10 @@ func (cfg Configuration) newAggregator(o DownsamplerOptions) (agg, error) { } return agg{ - aggregator: aggregatorInstance, - defaultStagedMetadatas: defaultStagedMetadatas, - matcher: matcher, - pools: pools, + aggregator: aggregatorInstance, + defaultStagedMetadatasProtos: defaultStagedMetadatasProtos, + matcher: matcher, + pools: pools, }, nil } @@ -1061,6 +1075,14 @@ func (c *aggregatorLocalAdminClient) WriteTimed( return c.agg.AddTimed(metric, metadata) } +// WriteTimedWithStagedMetadatas writes timed metrics with staged metadatas. +func (c *aggregatorLocalAdminClient) WriteTimedWithStagedMetadatas( + metric aggregated.Metric, + metadatas metadata.StagedMetadatas, +) error { + return c.agg.AddTimedWithStagedMetadatas(metric, metadatas) +} + // WriteForwarded writes forwarded metrics. func (c *aggregatorLocalAdminClient) WriteForwarded( metric aggregated.ForwardedMetric, @@ -1093,7 +1115,10 @@ var ( } ) -func bufferForPastTimedMetric(limits []bufferPastLimit, tile time.Duration) time.Duration { +func bufferForPastTimedMetric( + limits []bufferPastLimit, + tile time.Duration, +) time.Duration { bufferPast := limits[0].bufferPast for _, limit := range limits { if tile < limit.upperBound { @@ -1103,3 +1128,19 @@ func bufferForPastTimedMetric(limits []bufferPastLimit, tile time.Duration) time } return bufferPast } + +func maxAllowedForwardingDelay( + limits []bufferPastLimit, + tile time.Duration, + numForwardedTimes int, +) time.Duration { + resolutionForwardDelay := tile * time.Duration(numForwardedTimes) + bufferPast := limits[0].bufferPast + for _, limit := range limits { + if tile < limit.upperBound { + return bufferPast + resolutionForwardDelay + } + bufferPast = limit.bufferPast + } + return bufferPast + resolutionForwardDelay +} diff --git a/src/cmd/services/m3coordinator/downsample/samples_appender.go b/src/cmd/services/m3coordinator/downsample/samples_appender.go index 5e1ab4595f..f84ebc683a 100644 --- a/src/cmd/services/m3coordinator/downsample/samples_appender.go +++ b/src/cmd/services/m3coordinator/downsample/samples_appender.go @@ -96,27 +96,11 @@ func (a *samplesAppender) AppendGaugeTimedSample(t time.Time, value float64) err } func (a *samplesAppender) appendTimedSample(sample aggregated.Metric) error { - var multiErr xerrors.MultiError - for _, meta := range a.stagedMetadatas { - for _, pipeline := range meta.Pipelines { - for _, policy := range pipeline.StoragePolicies { - metadata := metadata.TimedMetadata{ - AggregationID: pipeline.AggregationID, - StoragePolicy: policy, - } - - if a.clientRemote != nil { - // Remote client write instead of local aggregation. - multiErr = multiErr.Add(a.clientRemote.WriteTimed(sample, metadata)) - continue - } - - // Add timed to local aggregator. - multiErr = multiErr.Add(a.agg.AddTimed(sample, metadata)) - } - } + if a.clientRemote != nil { + return a.clientRemote.WriteTimedWithStagedMetadatas(sample, a.stagedMetadatas) } - return multiErr.LastError() + + return a.agg.AddTimedWithStagedMetadatas(sample, a.stagedMetadatas) } // Ensure multiSamplesAppender implements SamplesAppender. diff --git a/src/metrics/encoding/protobuf/reset.go b/src/metrics/encoding/protobuf/reset.go index f546895465..bfd999ae14 100644 --- a/src/metrics/encoding/protobuf/reset.go +++ b/src/metrics/encoding/protobuf/reset.go @@ -48,6 +48,7 @@ func resetMetricWithMetadatasProto(pb *metricpb.MetricWithMetadatas) { resetGaugeWithMetadatasProto(pb.GaugeWithMetadatas) resetForwardedMetricWithMetadataProto(pb.ForwardedMetricWithMetadata) resetTimedMetricWithMetadataProto(pb.TimedMetricWithMetadata) + resetTimedMetricWithMetadatasProto(pb.TimedMetricWithMetadatas) } func resetCounterWithMetadatasProto(pb *metricpb.CounterWithMetadatas) { @@ -90,6 +91,14 @@ func resetTimedMetricWithMetadataProto(pb *metricpb.TimedMetricWithMetadata) { resetTimedMetadata(&pb.Metadata) } +func resetTimedMetricWithMetadatasProto(pb *metricpb.TimedMetricWithMetadatas) { + if pb == nil { + return + } + resetTimedMetric(&pb.Metric) + resetMetadatas(&pb.Metadatas) +} + func resetTimedMetricWithStoragePolicyProto(pb *metricpb.TimedMetricWithStoragePolicy) { if pb == nil { return diff --git a/src/metrics/encoding/protobuf/unaggregated_encoder.go b/src/metrics/encoding/protobuf/unaggregated_encoder.go index d2c4d89bad..8ee584b751 100644 --- a/src/metrics/encoding/protobuf/unaggregated_encoder.go +++ b/src/metrics/encoding/protobuf/unaggregated_encoder.go @@ -68,6 +68,7 @@ type unaggregatedEncoder struct { gm metricpb.GaugeWithMetadatas fm metricpb.ForwardedMetricWithMetadata tm metricpb.TimedMetricWithMetadata + tms metricpb.TimedMetricWithMetadatas buf []byte used int @@ -127,6 +128,8 @@ func (enc *unaggregatedEncoder) EncodeMessage(msg encoding.UnaggregatedMessageUn return enc.encodeForwardedMetricWithMetadata(msg.ForwardedMetricWithMetadata) case encoding.TimedMetricWithMetadataType: return enc.encodeTimedMetricWithMetadata(msg.TimedMetricWithMetadata) + case encoding.TimedMetricWithMetadatasType: + return enc.encodeTimedMetricWithMetadatas(msg.TimedMetricWithMetadatas) default: return fmt.Errorf("unknown message type: %v", msg.Type) } @@ -187,6 +190,17 @@ func (enc *unaggregatedEncoder) encodeTimedMetricWithMetadata(tm aggregated.Time return enc.encodeMetricWithMetadatas(mm) } +func (enc *unaggregatedEncoder) encodeTimedMetricWithMetadatas(tms aggregated.TimedMetricWithMetadatas) error { + if err := tms.ToProto(&enc.tms); err != nil { + return fmt.Errorf("timed metric with metadatas proto conversion failed: %v", err) + } + mm := metricpb.MetricWithMetadatas{ + Type: metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATAS, + TimedMetricWithMetadatas: &enc.tms, + } + return enc.encodeMetricWithMetadatas(mm) +} + func (enc *unaggregatedEncoder) encodeMetricWithMetadatas(pb metricpb.MetricWithMetadatas) error { msgSize := pb.Size() if msgSize > enc.maxMessageSize { diff --git a/src/metrics/encoding/protobuf/unaggregated_iterator.go b/src/metrics/encoding/protobuf/unaggregated_iterator.go index d04bd618b4..3cc73b489a 100644 --- a/src/metrics/encoding/protobuf/unaggregated_iterator.go +++ b/src/metrics/encoding/protobuf/unaggregated_iterator.go @@ -149,6 +149,9 @@ func (it *unaggregatedIterator) decodeMessage(size int) error { case metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATA: it.msg.Type = encoding.TimedMetricWithMetadataType it.err = it.msg.TimedMetricWithMetadata.FromProto(it.pb.TimedMetricWithMetadata) + case metricpb.MetricWithMetadatas_TIMED_METRIC_WITH_METADATAS: + it.msg.Type = encoding.TimedMetricWithMetadatasType + it.err = it.msg.TimedMetricWithMetadatas.FromProto(it.pb.TimedMetricWithMetadatas) default: it.err = fmt.Errorf("unrecognized message type: %v", it.pb.Type) } diff --git a/src/metrics/encoding/types.go b/src/metrics/encoding/types.go index 266ac05a8e..d66c4da2c9 100644 --- a/src/metrics/encoding/types.go +++ b/src/metrics/encoding/types.go @@ -38,6 +38,7 @@ const ( GaugeWithMetadatasType ForwardedMetricWithMetadataType TimedMetricWithMetadataType + TimedMetricWithMetadatasType ) // UnaggregatedMessageUnion is a union of different types of unaggregated messages. @@ -51,6 +52,7 @@ type UnaggregatedMessageUnion struct { GaugeWithMetadatas unaggregated.GaugeWithMetadatas ForwardedMetricWithMetadata aggregated.ForwardedMetricWithMetadata TimedMetricWithMetadata aggregated.TimedMetricWithMetadata + TimedMetricWithMetadatas aggregated.TimedMetricWithMetadatas } // ByteReadScanner is capable of reading and scanning bytes. diff --git a/src/metrics/generated/proto/metricpb/composite.pb.go b/src/metrics/generated/proto/metricpb/composite.pb.go index 8bdb013320..b42e69cf29 100644 --- a/src/metrics/generated/proto/metricpb/composite.pb.go +++ b/src/metrics/generated/proto/metricpb/composite.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: github.com/m3db/m3/src/metrics/generated/proto/metricpb/composite.proto -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -35,6 +35,7 @@ GaugeWithMetadatas ForwardedMetricWithMetadata TimedMetricWithMetadata + TimedMetricWithMetadatas TimedMetricWithStoragePolicy AggregatedMetric MetricWithMetadatas @@ -80,6 +81,7 @@ const ( MetricWithMetadatas_GAUGE_WITH_METADATAS MetricWithMetadatas_Type = 3 MetricWithMetadatas_FORWARDED_METRIC_WITH_METADATA MetricWithMetadatas_Type = 4 MetricWithMetadatas_TIMED_METRIC_WITH_METADATA MetricWithMetadatas_Type = 5 + MetricWithMetadatas_TIMED_METRIC_WITH_METADATAS MetricWithMetadatas_Type = 6 ) var MetricWithMetadatas_Type_name = map[int32]string{ @@ -89,6 +91,7 @@ var MetricWithMetadatas_Type_name = map[int32]string{ 3: "GAUGE_WITH_METADATAS", 4: "FORWARDED_METRIC_WITH_METADATA", 5: "TIMED_METRIC_WITH_METADATA", + 6: "TIMED_METRIC_WITH_METADATAS", } var MetricWithMetadatas_Type_value = map[string]int32{ "UNKNOWN": 0, @@ -97,13 +100,14 @@ var MetricWithMetadatas_Type_value = map[string]int32{ "GAUGE_WITH_METADATAS": 3, "FORWARDED_METRIC_WITH_METADATA": 4, "TIMED_METRIC_WITH_METADATA": 5, + "TIMED_METRIC_WITH_METADATAS": 6, } func (x MetricWithMetadatas_Type) String() string { return proto.EnumName(MetricWithMetadatas_Type_name, int32(x)) } func (MetricWithMetadatas_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorComposite, []int{7, 0} + return fileDescriptorComposite, []int{8, 0} } type CounterWithMetadatas struct { @@ -228,6 +232,32 @@ func (m *TimedMetricWithMetadata) GetMetadata() TimedMetadata { return TimedMetadata{} } +type TimedMetricWithMetadatas struct { + Metric TimedMetric `protobuf:"bytes,1,opt,name=metric" json:"metric"` + Metadatas StagedMetadatas `protobuf:"bytes,2,opt,name=metadatas" json:"metadatas"` +} + +func (m *TimedMetricWithMetadatas) Reset() { *m = TimedMetricWithMetadatas{} } +func (m *TimedMetricWithMetadatas) String() string { return proto.CompactTextString(m) } +func (*TimedMetricWithMetadatas) ProtoMessage() {} +func (*TimedMetricWithMetadatas) Descriptor() ([]byte, []int) { + return fileDescriptorComposite, []int{5} +} + +func (m *TimedMetricWithMetadatas) GetMetric() TimedMetric { + if m != nil { + return m.Metric + } + return TimedMetric{} +} + +func (m *TimedMetricWithMetadatas) GetMetadatas() StagedMetadatas { + if m != nil { + return m.Metadatas + } + return StagedMetadatas{} +} + type TimedMetricWithStoragePolicy struct { TimedMetric TimedMetric `protobuf:"bytes,1,opt,name=timed_metric,json=timedMetric" json:"timed_metric"` StoragePolicy policypb.StoragePolicy `protobuf:"bytes,2,opt,name=storage_policy,json=storagePolicy" json:"storage_policy"` @@ -237,7 +267,7 @@ func (m *TimedMetricWithStoragePolicy) Reset() { *m = TimedMetricWithSto func (m *TimedMetricWithStoragePolicy) String() string { return proto.CompactTextString(m) } func (*TimedMetricWithStoragePolicy) ProtoMessage() {} func (*TimedMetricWithStoragePolicy) Descriptor() ([]byte, []int) { - return fileDescriptorComposite, []int{5} + return fileDescriptorComposite, []int{6} } func (m *TimedMetricWithStoragePolicy) GetTimedMetric() TimedMetric { @@ -262,7 +292,7 @@ type AggregatedMetric struct { func (m *AggregatedMetric) Reset() { *m = AggregatedMetric{} } func (m *AggregatedMetric) String() string { return proto.CompactTextString(m) } func (*AggregatedMetric) ProtoMessage() {} -func (*AggregatedMetric) Descriptor() ([]byte, []int) { return fileDescriptorComposite, []int{6} } +func (*AggregatedMetric) Descriptor() ([]byte, []int) { return fileDescriptorComposite, []int{7} } func (m *AggregatedMetric) GetMetric() TimedMetricWithStoragePolicy { if m != nil { @@ -291,12 +321,13 @@ type MetricWithMetadatas struct { GaugeWithMetadatas *GaugeWithMetadatas `protobuf:"bytes,4,opt,name=gauge_with_metadatas,json=gaugeWithMetadatas" json:"gauge_with_metadatas,omitempty"` ForwardedMetricWithMetadata *ForwardedMetricWithMetadata `protobuf:"bytes,5,opt,name=forwarded_metric_with_metadata,json=forwardedMetricWithMetadata" json:"forwarded_metric_with_metadata,omitempty"` TimedMetricWithMetadata *TimedMetricWithMetadata `protobuf:"bytes,6,opt,name=timed_metric_with_metadata,json=timedMetricWithMetadata" json:"timed_metric_with_metadata,omitempty"` + TimedMetricWithMetadatas *TimedMetricWithMetadatas `protobuf:"bytes,7,opt,name=timed_metric_with_metadatas,json=timedMetricWithMetadatas" json:"timed_metric_with_metadatas,omitempty"` } func (m *MetricWithMetadatas) Reset() { *m = MetricWithMetadatas{} } func (m *MetricWithMetadatas) String() string { return proto.CompactTextString(m) } func (*MetricWithMetadatas) ProtoMessage() {} -func (*MetricWithMetadatas) Descriptor() ([]byte, []int) { return fileDescriptorComposite, []int{7} } +func (*MetricWithMetadatas) Descriptor() ([]byte, []int) { return fileDescriptorComposite, []int{8} } func (m *MetricWithMetadatas) GetType() MetricWithMetadatas_Type { if m != nil { @@ -340,12 +371,20 @@ func (m *MetricWithMetadatas) GetTimedMetricWithMetadata() *TimedMetricWithMetad return nil } +func (m *MetricWithMetadatas) GetTimedMetricWithMetadatas() *TimedMetricWithMetadatas { + if m != nil { + return m.TimedMetricWithMetadatas + } + return nil +} + func init() { proto.RegisterType((*CounterWithMetadatas)(nil), "metricpb.CounterWithMetadatas") proto.RegisterType((*BatchTimerWithMetadatas)(nil), "metricpb.BatchTimerWithMetadatas") proto.RegisterType((*GaugeWithMetadatas)(nil), "metricpb.GaugeWithMetadatas") proto.RegisterType((*ForwardedMetricWithMetadata)(nil), "metricpb.ForwardedMetricWithMetadata") proto.RegisterType((*TimedMetricWithMetadata)(nil), "metricpb.TimedMetricWithMetadata") + proto.RegisterType((*TimedMetricWithMetadatas)(nil), "metricpb.TimedMetricWithMetadatas") proto.RegisterType((*TimedMetricWithStoragePolicy)(nil), "metricpb.TimedMetricWithStoragePolicy") proto.RegisterType((*AggregatedMetric)(nil), "metricpb.AggregatedMetric") proto.RegisterType((*MetricWithMetadatas)(nil), "metricpb.MetricWithMetadatas") @@ -521,6 +560,40 @@ func (m *TimedMetricWithMetadata) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *TimedMetricWithMetadatas) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TimedMetricWithMetadatas) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintComposite(dAtA, i, uint64(m.Metric.Size())) + n11, err := m.Metric.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + dAtA[i] = 0x12 + i++ + i = encodeVarintComposite(dAtA, i, uint64(m.Metadatas.Size())) + n12, err := m.Metadatas.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 + return i, nil +} + func (m *TimedMetricWithStoragePolicy) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -539,19 +612,19 @@ func (m *TimedMetricWithStoragePolicy) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintComposite(dAtA, i, uint64(m.TimedMetric.Size())) - n11, err := m.TimedMetric.MarshalTo(dAtA[i:]) + n13, err := m.TimedMetric.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n13 dAtA[i] = 0x12 i++ i = encodeVarintComposite(dAtA, i, uint64(m.StoragePolicy.Size())) - n12, err := m.StoragePolicy.MarshalTo(dAtA[i:]) + n14, err := m.StoragePolicy.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n14 return i, nil } @@ -573,11 +646,11 @@ func (m *AggregatedMetric) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintComposite(dAtA, i, uint64(m.Metric.Size())) - n13, err := m.Metric.MarshalTo(dAtA[i:]) + n15, err := m.Metric.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n15 if m.EncodeNanos != 0 { dAtA[i] = 0x10 i++ @@ -610,51 +683,61 @@ func (m *MetricWithMetadatas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintComposite(dAtA, i, uint64(m.CounterWithMetadatas.Size())) - n14, err := m.CounterWithMetadatas.MarshalTo(dAtA[i:]) + n16, err := m.CounterWithMetadatas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n16 } if m.BatchTimerWithMetadatas != nil { dAtA[i] = 0x1a i++ i = encodeVarintComposite(dAtA, i, uint64(m.BatchTimerWithMetadatas.Size())) - n15, err := m.BatchTimerWithMetadatas.MarshalTo(dAtA[i:]) + n17, err := m.BatchTimerWithMetadatas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n17 } if m.GaugeWithMetadatas != nil { dAtA[i] = 0x22 i++ i = encodeVarintComposite(dAtA, i, uint64(m.GaugeWithMetadatas.Size())) - n16, err := m.GaugeWithMetadatas.MarshalTo(dAtA[i:]) + n18, err := m.GaugeWithMetadatas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n18 } if m.ForwardedMetricWithMetadata != nil { dAtA[i] = 0x2a i++ i = encodeVarintComposite(dAtA, i, uint64(m.ForwardedMetricWithMetadata.Size())) - n17, err := m.ForwardedMetricWithMetadata.MarshalTo(dAtA[i:]) + n19, err := m.ForwardedMetricWithMetadata.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n19 } if m.TimedMetricWithMetadata != nil { dAtA[i] = 0x32 i++ i = encodeVarintComposite(dAtA, i, uint64(m.TimedMetricWithMetadata.Size())) - n18, err := m.TimedMetricWithMetadata.MarshalTo(dAtA[i:]) + n20, err := m.TimedMetricWithMetadata.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n20 + } + if m.TimedMetricWithMetadatas != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintComposite(dAtA, i, uint64(m.TimedMetricWithMetadatas.Size())) + n21, err := m.TimedMetricWithMetadatas.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n21 } return i, nil } @@ -718,6 +801,16 @@ func (m *TimedMetricWithMetadata) Size() (n int) { return n } +func (m *TimedMetricWithMetadatas) Size() (n int) { + var l int + _ = l + l = m.Metric.Size() + n += 1 + l + sovComposite(uint64(l)) + l = m.Metadatas.Size() + n += 1 + l + sovComposite(uint64(l)) + return n +} + func (m *TimedMetricWithStoragePolicy) Size() (n int) { var l int _ = l @@ -765,6 +858,10 @@ func (m *MetricWithMetadatas) Size() (n int) { l = m.TimedMetricWithMetadata.Size() n += 1 + l + sovComposite(uint64(l)) } + if m.TimedMetricWithMetadatas != nil { + l = m.TimedMetricWithMetadatas.Size() + n += 1 + l + sovComposite(uint64(l)) + } return n } @@ -1331,6 +1428,116 @@ func (m *TimedMetricWithMetadata) Unmarshal(dAtA []byte) error { } return nil } +func (m *TimedMetricWithMetadatas) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowComposite + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TimedMetricWithMetadatas: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TimedMetricWithMetadatas: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metric", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowComposite + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthComposite + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Metric.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadatas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowComposite + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthComposite + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Metadatas.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipComposite(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthComposite + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *TimedMetricWithStoragePolicy) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -1753,6 +1960,39 @@ func (m *MetricWithMetadatas) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TimedMetricWithMetadatas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowComposite + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthComposite + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TimedMetricWithMetadatas == nil { + m.TimedMetricWithMetadatas = &TimedMetricWithMetadatas{} + } + if err := m.TimedMetricWithMetadatas.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipComposite(dAtA[iNdEx:]) @@ -1884,51 +2124,53 @@ func init() { } var fileDescriptorComposite = []byte{ - // 732 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x95, 0xcd, 0x6e, 0xd3, 0x5a, - 0x10, 0xc7, 0xeb, 0x36, 0x69, 0x7b, 0x27, 0xbd, 0xbd, 0xb9, 0xe7, 0xe6, 0x36, 0x21, 0xad, 0x0c, - 0xb5, 0x04, 0x42, 0x42, 0xc4, 0xa2, 0x91, 0xa8, 0x50, 0x05, 0x92, 0xf3, 0xd1, 0x34, 0x42, 0x4d, - 0x91, 0xeb, 0x2a, 0x12, 0x8b, 0x5a, 0xfe, 0xaa, 0x63, 0x44, 0x62, 0xcb, 0x3e, 0x51, 0x55, 0xb1, - 0x61, 0x09, 0x3b, 0x24, 0xc4, 0x1b, 0xb0, 0xe5, 0x3d, 0xba, 0x83, 0x27, 0x40, 0xa8, 0xbc, 0x08, - 0xb2, 0x7d, 0x1c, 0xdb, 0xc7, 0x0e, 0x42, 0xed, 0xce, 0x9e, 0x99, 0xff, 0x6f, 0xfe, 0x39, 0x9e, - 0x39, 0x81, 0x9e, 0x69, 0xe1, 0xd1, 0x54, 0x6d, 0x68, 0xf6, 0x98, 0x1f, 0x37, 0x75, 0x95, 0x1f, - 0x37, 0x79, 0xcf, 0xd5, 0xf8, 0xb1, 0x81, 0x5d, 0x4b, 0xf3, 0x78, 0xd3, 0x98, 0x18, 0xae, 0x82, - 0x0d, 0x9d, 0x77, 0x5c, 0x1b, 0xdb, 0x24, 0xee, 0xa8, 0xbc, 0x66, 0x8f, 0x1d, 0xdb, 0xb3, 0xb0, - 0xd1, 0x08, 0x12, 0x68, 0x35, 0xca, 0xd4, 0x1f, 0x26, 0x90, 0xa6, 0x6d, 0xda, 0xa1, 0x52, 0x9d, - 0x9e, 0x05, 0x6f, 0x21, 0xc6, 0x7f, 0x0a, 0x85, 0xf5, 0xce, 0x75, 0x1d, 0x84, 0x0f, 0x84, 0xb2, - 0x7f, 0x03, 0x8a, 0xa2, 0x2b, 0x58, 0xb9, 0xa6, 0x1b, 0xc7, 0x7e, 0x6d, 0x69, 0x17, 0x8e, 0x4a, - 0x1e, 0x42, 0x0a, 0xf7, 0x8e, 0x81, 0x4a, 0xdb, 0x9e, 0x4e, 0xb0, 0xe1, 0x0e, 0x2d, 0x3c, 0x3a, - 0x24, 0x3d, 0x3c, 0xf4, 0x08, 0x56, 0xb4, 0x30, 0x5e, 0x63, 0xee, 0x30, 0xf7, 0x4b, 0x3b, 0xff, - 0x36, 0x22, 0x27, 0x0d, 0x22, 0x68, 0x15, 0x2e, 0xbf, 0xdf, 0x5e, 0x10, 0xa3, 0x3a, 0xf4, 0x14, - 0xfe, 0x8a, 0x3c, 0x7a, 0xb5, 0xc5, 0x40, 0x74, 0x2b, 0x16, 0x1d, 0x63, 0xc5, 0x34, 0xf4, 0x59, - 0x03, 0x22, 0x8e, 0x15, 0xdc, 0x27, 0x06, 0xaa, 0x2d, 0x05, 0x6b, 0x23, 0xc9, 0x1a, 0xd3, 0x6e, - 0xf6, 0xa0, 0xa4, 0xfa, 0x29, 0x19, 0xfb, 0x39, 0xe2, 0xa8, 0x12, 0xc3, 0x63, 0x1d, 0xe1, 0x82, - 0x3a, 0x8b, 0xdc, 0xd4, 0xd7, 0x5b, 0x06, 0x50, 0x4f, 0x99, 0x9a, 0x46, 0xda, 0xd2, 0x03, 0x28, - 0x9a, 0x7e, 0x94, 0x98, 0xf9, 0x27, 0x26, 0x06, 0xc5, 0x84, 0x13, 0xd6, 0xdc, 0xd4, 0xc2, 0x47, - 0x06, 0x36, 0xf7, 0x6d, 0xf7, 0x5c, 0x71, 0xf5, 0xa0, 0xce, 0xb5, 0xb4, 0xa4, 0x19, 0xb4, 0x0b, - 0xcb, 0x21, 0x8c, 0x98, 0x49, 0xb0, 0x29, 0x19, 0x61, 0x93, 0x72, 0xb4, 0x07, 0xab, 0x51, 0x97, - 0xac, 0x2d, 0x22, 0x8d, 0xba, 0x10, 0xe9, 0x4c, 0xc0, 0xbd, 0x67, 0xa0, 0xea, 0x9f, 0x70, 0x9e, - 0xa3, 0x26, 0xe5, 0xe8, 0xff, 0x18, 0x9b, 0x90, 0x50, 0x6e, 0x9e, 0x64, 0xdc, 0x54, 0xb3, 0xb2, - 0x7c, 0x2f, 0x9f, 0x19, 0xd8, 0xa2, 0xbc, 0x1c, 0x63, 0xdb, 0x55, 0x4c, 0xe3, 0x45, 0x30, 0xee, - 0xe8, 0x19, 0xac, 0xf9, 0xb3, 0xa3, 0xcb, 0x7f, 0x6e, 0xab, 0x84, 0xe3, 0x10, 0xea, 0xc0, 0xba, - 0x17, 0x02, 0xe5, 0x70, 0x81, 0x66, 0x0e, 0xa3, 0xc5, 0x6a, 0xa4, 0x1a, 0x12, 0xc6, 0xdf, 0x5e, - 0x32, 0xc8, 0xbd, 0x81, 0xb2, 0x60, 0x9a, 0xae, 0x61, 0xfa, 0x8b, 0x39, 0x23, 0xa7, 0x8f, 0xea, - 0x5e, 0xae, 0xa7, 0xcc, 0x2f, 0xa2, 0xce, 0x6e, 0x1b, 0xd6, 0x8c, 0x89, 0x66, 0xeb, 0x86, 0x3c, - 0x51, 0x26, 0x76, 0x38, 0x64, 0x4b, 0x62, 0x29, 0x8c, 0x0d, 0xfc, 0x10, 0xf7, 0xb5, 0x08, 0xff, - 0x65, 0x3f, 0x95, 0x87, 0x1e, 0x43, 0x01, 0x5f, 0x38, 0xe1, 0x20, 0xaf, 0xef, 0x70, 0x71, 0xfb, - 0x9c, 0xe2, 0x86, 0x74, 0xe1, 0x18, 0x62, 0x50, 0x8f, 0x24, 0xd8, 0x20, 0xab, 0x2f, 0x9f, 0x5b, - 0x78, 0x24, 0xd3, 0x13, 0xce, 0x66, 0x6e, 0x8c, 0x14, 0x4a, 0xac, 0x68, 0x79, 0x17, 0xcf, 0x29, - 0xd4, 0x13, 0xab, 0x4e, 0x93, 0x97, 0x02, 0xf2, 0x76, 0xde, 0xe6, 0xa7, 0xe1, 0x55, 0x75, 0xce, - 0x55, 0x32, 0x80, 0x4a, 0xb0, 0x93, 0x34, 0xb9, 0x10, 0x90, 0xb7, 0xa8, 0x35, 0x4e, 0x43, 0x91, - 0x99, 0xbd, 0x07, 0x5e, 0x01, 0x7b, 0x16, 0xed, 0x18, 0x19, 0xae, 0x34, 0xba, 0x56, 0x0c, 0xc8, - 0x77, 0xe7, 0xee, 0x64, 0x92, 0x27, 0x6e, 0x9e, 0xfd, 0x66, 0xcf, 0x4f, 0xa1, 0x9e, 0x1c, 0x62, - 0xaa, 0xcf, 0x32, 0x7d, 0x36, 0x73, 0x96, 0x53, 0xac, 0xe2, 0xfc, 0x04, 0xf7, 0x85, 0x81, 0x82, - 0xff, 0x81, 0x51, 0x09, 0x56, 0x4e, 0x06, 0xcf, 0x07, 0x47, 0xc3, 0x41, 0x79, 0x01, 0xd5, 0x61, - 0xa3, 0x7d, 0x74, 0x32, 0x90, 0xba, 0xa2, 0x3c, 0xec, 0x4b, 0x07, 0xf2, 0x61, 0x57, 0x12, 0x3a, - 0x82, 0x24, 0x1c, 0x97, 0x19, 0xc4, 0x42, 0xbd, 0x25, 0x48, 0xed, 0x03, 0x59, 0xea, 0x1f, 0x66, - 0xf3, 0x8b, 0xa8, 0x06, 0x95, 0x9e, 0x70, 0xd2, 0xeb, 0xd2, 0x99, 0x25, 0xc4, 0x01, 0xbb, 0x7f, - 0x24, 0x0e, 0x05, 0xb1, 0xd3, 0xed, 0xf8, 0x09, 0xb1, 0xdf, 0x4e, 0x17, 0x95, 0x0b, 0x3e, 0xdd, - 0xe7, 0xce, 0xc9, 0x17, 0x5b, 0xfd, 0xcb, 0x2b, 0x96, 0xf9, 0x76, 0xc5, 0x32, 0x3f, 0xae, 0x58, - 0xe6, 0xc3, 0x4f, 0x76, 0xe1, 0xe5, 0xee, 0x35, 0xff, 0x5d, 0xd5, 0xe5, 0xe0, 0xbd, 0xf9, 0x2b, - 0x00, 0x00, 0xff, 0xff, 0x1a, 0x70, 0xed, 0x30, 0x67, 0x08, 0x00, 0x00, + // 768 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x96, 0xdf, 0x6b, 0xd3, 0x6c, + 0x14, 0xc7, 0xf7, 0x6c, 0x5d, 0xb7, 0xf7, 0x74, 0xef, 0xde, 0xbe, 0x8f, 0x75, 0xad, 0xed, 0xc8, + 0x5c, 0x40, 0x11, 0xc4, 0x16, 0x57, 0x70, 0xc8, 0x50, 0xe8, 0xaf, 0x75, 0x45, 0xd6, 0x49, 0x9a, + 0x51, 0xf0, 0x62, 0x21, 0x49, 0xb3, 0x34, 0x62, 0x9b, 0x92, 0x3c, 0x65, 0x0c, 0x6f, 0xbc, 0xd4, + 0x1b, 0x11, 0xc4, 0xff, 0xc0, 0x3f, 0x66, 0x57, 0x22, 0x78, 0x2f, 0x32, 0xff, 0x11, 0x49, 0xf2, + 0xa4, 0xf9, 0xad, 0xd2, 0xde, 0x25, 0xe7, 0x9c, 0xef, 0xe7, 0x7c, 0xfb, 0xe4, 0x39, 0x87, 0x42, + 0x5b, 0xd5, 0xc8, 0x70, 0x2a, 0x95, 0x65, 0x7d, 0x54, 0x19, 0x55, 0x07, 0x52, 0x65, 0x54, 0xad, + 0x98, 0x86, 0x5c, 0x19, 0x29, 0xc4, 0xd0, 0x64, 0xb3, 0xa2, 0x2a, 0x63, 0xc5, 0x10, 0x89, 0x32, + 0xa8, 0x4c, 0x0c, 0x9d, 0xe8, 0x34, 0x3e, 0x91, 0x2a, 0xb2, 0x3e, 0x9a, 0xe8, 0xa6, 0x46, 0x94, + 0xb2, 0x9d, 0xc0, 0xeb, 0x6e, 0xa6, 0xf8, 0xc0, 0x87, 0x54, 0x75, 0x55, 0x77, 0x94, 0xd2, 0xf4, + 0xdc, 0x7e, 0x73, 0x30, 0xd6, 0x93, 0x23, 0x2c, 0x36, 0xe7, 0x75, 0xe0, 0x3c, 0x50, 0xca, 0xe1, + 0x02, 0x14, 0x71, 0x20, 0x12, 0x71, 0x4e, 0x37, 0x13, 0xfd, 0x95, 0x26, 0x5f, 0x4e, 0x24, 0xfa, + 0xe0, 0x50, 0xd8, 0xb7, 0x08, 0x72, 0x0d, 0x7d, 0x3a, 0x26, 0x8a, 0xd1, 0xd7, 0xc8, 0xf0, 0x98, + 0xf6, 0x30, 0xf1, 0x43, 0x58, 0x93, 0x9d, 0x78, 0x01, 0xdd, 0x46, 0xf7, 0x32, 0x7b, 0xff, 0x97, + 0x5d, 0x27, 0x65, 0x2a, 0xa8, 0xa7, 0xae, 0xbe, 0xef, 0x2c, 0x71, 0x6e, 0x1d, 0x7e, 0x02, 0xff, + 0xb8, 0x1e, 0xcd, 0xc2, 0xb2, 0x2d, 0xba, 0xe5, 0x89, 0x7a, 0x44, 0x54, 0x95, 0xc1, 0xac, 0x01, + 0x15, 0x7b, 0x0a, 0xf6, 0x13, 0x82, 0x7c, 0x5d, 0x24, 0xf2, 0x90, 0xd7, 0x46, 0x61, 0x37, 0x07, + 0x90, 0x91, 0xac, 0x94, 0x40, 0xac, 0x1c, 0x75, 0x94, 0xf3, 0xe0, 0x9e, 0x8e, 0x72, 0x41, 0x9a, + 0x45, 0x16, 0xf5, 0xf5, 0x06, 0x01, 0x6e, 0x8b, 0x53, 0x55, 0x09, 0x5a, 0xba, 0x0f, 0xab, 0xaa, + 0x15, 0xa5, 0x66, 0xfe, 0xf3, 0x88, 0x76, 0x31, 0xe5, 0x38, 0x35, 0x8b, 0x5a, 0xf8, 0x88, 0xa0, + 0x74, 0xa8, 0x1b, 0x17, 0xa2, 0x31, 0xb0, 0xeb, 0x0c, 0x4d, 0xf6, 0x9b, 0xc1, 0xfb, 0x90, 0x76, + 0x60, 0xd4, 0x8c, 0x8f, 0x1d, 0x92, 0x51, 0x36, 0x2d, 0xc7, 0x07, 0xb0, 0xee, 0x76, 0x89, 0xda, + 0xa2, 0x52, 0xb7, 0x0b, 0x95, 0xce, 0x04, 0xec, 0x3b, 0x04, 0x79, 0xeb, 0x84, 0xe3, 0x1c, 0x55, + 0x43, 0x8e, 0x6e, 0x7a, 0x58, 0x9f, 0x24, 0xe4, 0xe6, 0x71, 0xc4, 0x4d, 0x3e, 0x2a, 0x8b, 0xf7, + 0xf2, 0x1e, 0x41, 0x21, 0xc1, 0x8b, 0x39, 0x9f, 0x99, 0x05, 0x3f, 0xd9, 0x67, 0x04, 0xdb, 0x21, + 0x43, 0x3d, 0xa2, 0x1b, 0xa2, 0xaa, 0x3c, 0xb7, 0xe7, 0x0f, 0x3f, 0x85, 0x0d, 0xeb, 0x32, 0x0f, + 0x84, 0xbf, 0xb7, 0x96, 0x21, 0x5e, 0x08, 0x37, 0x61, 0xd3, 0x74, 0x80, 0x82, 0x33, 0xd1, 0xb3, + 0x23, 0x73, 0x27, 0xbd, 0x1c, 0x68, 0x48, 0x19, 0xff, 0x9a, 0xfe, 0x20, 0xfb, 0x1a, 0xb2, 0x35, + 0x55, 0x35, 0x14, 0xd5, 0xda, 0x14, 0x33, 0x72, 0xf0, 0xb8, 0xee, 0xc6, 0x7a, 0x8a, 0xfc, 0xa2, + 0xd0, 0xf9, 0xed, 0xc2, 0x86, 0x32, 0x96, 0xf5, 0x81, 0x22, 0x8c, 0xc5, 0xb1, 0xee, 0x1c, 0xe1, + 0x0a, 0x97, 0x71, 0x62, 0x5d, 0x2b, 0xc4, 0x7e, 0x4b, 0xc3, 0x8d, 0xb8, 0xef, 0xf5, 0x08, 0x52, + 0xe4, 0x72, 0xe2, 0x4c, 0xd6, 0xe6, 0x1e, 0xeb, 0xb5, 0x8f, 0x29, 0x2e, 0xf3, 0x97, 0x13, 0x85, + 0xb3, 0xeb, 0x31, 0x0f, 0x5b, 0x74, 0x17, 0x09, 0x17, 0x1a, 0x19, 0x0a, 0xe1, 0xef, 0xc7, 0x44, + 0x56, 0x58, 0x00, 0xc5, 0xe5, 0xe4, 0xb8, 0x4d, 0x78, 0x06, 0x45, 0xdf, 0xee, 0x09, 0x93, 0x57, + 0x6c, 0xf2, 0x6e, 0xdc, 0x2a, 0x0a, 0xc2, 0xf3, 0x52, 0xc2, 0x6e, 0xeb, 0x42, 0xce, 0x5e, 0x12, + 0x61, 0x72, 0xca, 0x26, 0x6f, 0x87, 0xf6, 0x4a, 0x10, 0x8a, 0xd5, 0xe8, 0x62, 0x7a, 0x09, 0xcc, + 0xb9, 0x3b, 0xf4, 0xf4, 0x72, 0x05, 0xd1, 0x85, 0x55, 0x9b, 0x7c, 0x27, 0x71, 0x49, 0xf8, 0x79, + 0x5c, 0xe9, 0xfc, 0x37, 0x8b, 0xe7, 0x0c, 0x8a, 0xfe, 0x4b, 0x1c, 0xea, 0x93, 0x0e, 0x9f, 0x4d, + 0xc2, 0x84, 0x72, 0x79, 0x92, 0xb0, 0x46, 0x44, 0x28, 0x25, 0xf3, 0xcd, 0xc2, 0x9a, 0xdd, 0x80, + 0xfd, 0x63, 0x03, 0x93, 0x2b, 0x24, 0x74, 0x30, 0xd9, 0x2f, 0x08, 0x52, 0xd6, 0x1d, 0xc2, 0x19, + 0x58, 0x3b, 0xed, 0x3e, 0xeb, 0x9e, 0xf4, 0xbb, 0xd9, 0x25, 0x5c, 0x84, 0xad, 0xc6, 0xc9, 0x69, + 0x97, 0x6f, 0x71, 0x42, 0xbf, 0xc3, 0x1f, 0x09, 0xc7, 0x2d, 0xbe, 0xd6, 0xac, 0xf1, 0xb5, 0x5e, + 0x16, 0x61, 0x06, 0x8a, 0xf5, 0x1a, 0xdf, 0x38, 0x12, 0xf8, 0xce, 0x71, 0x34, 0xbf, 0x8c, 0x0b, + 0x90, 0x6b, 0xd7, 0x4e, 0xdb, 0xad, 0x70, 0x66, 0x05, 0xb3, 0xc0, 0x1c, 0x9e, 0x70, 0xfd, 0x1a, + 0xd7, 0x6c, 0x35, 0xad, 0x04, 0xd7, 0x69, 0x04, 0x8b, 0xb2, 0x29, 0x8b, 0x6e, 0x71, 0x13, 0xf2, + 0xab, 0x78, 0x07, 0x4a, 0xc9, 0xf9, 0x5e, 0x36, 0x5d, 0xef, 0x5c, 0x5d, 0x33, 0xe8, 0xeb, 0x35, + 0x83, 0x7e, 0x5c, 0x33, 0xe8, 0xc3, 0x4f, 0x66, 0xe9, 0xc5, 0xfe, 0x9c, 0x7f, 0x39, 0xa4, 0xb4, + 0xfd, 0x5e, 0xfd, 0x15, 0x00, 0x00, 0xff, 0xff, 0x00, 0x47, 0x7a, 0x2d, 0x7c, 0x09, 0x00, 0x00, } diff --git a/src/metrics/generated/proto/metricpb/composite.proto b/src/metrics/generated/proto/metricpb/composite.proto index b7b7076c3d..ad54e6c119 100644 --- a/src/metrics/generated/proto/metricpb/composite.proto +++ b/src/metrics/generated/proto/metricpb/composite.proto @@ -54,6 +54,11 @@ message TimedMetricWithMetadata { TimedMetadata metadata = 2 [(gogoproto.nullable) = false]; } +message TimedMetricWithMetadatas { + TimedMetric metric = 1 [(gogoproto.nullable) = false]; + StagedMetadatas metadatas = 2 [(gogoproto.nullable) = false]; +} + message TimedMetricWithStoragePolicy { TimedMetric timed_metric = 1 [(gogoproto.nullable) = false]; policypb.StoragePolicy storage_policy = 2 [(gogoproto.nullable) = false]; @@ -78,6 +83,7 @@ message MetricWithMetadatas { GAUGE_WITH_METADATAS = 3; FORWARDED_METRIC_WITH_METADATA = 4; TIMED_METRIC_WITH_METADATA = 5; + TIMED_METRIC_WITH_METADATAS = 6; } Type type = 1; CounterWithMetadatas counter_with_metadatas = 2; @@ -85,4 +91,5 @@ message MetricWithMetadatas { GaugeWithMetadatas gauge_with_metadatas = 4; ForwardedMetricWithMetadata forwarded_metric_with_metadata = 5; TimedMetricWithMetadata timed_metric_with_metadata = 6; + TimedMetricWithMetadatas timed_metric_with_metadatas = 7; } diff --git a/src/metrics/generated/proto/transformationpb/transformation.pb.go b/src/metrics/generated/proto/transformationpb/transformation.pb.go index c4fd69f38c..808d10685a 100644 --- a/src/metrics/generated/proto/transformationpb/transformation.pb.go +++ b/src/metrics/generated/proto/transformationpb/transformation.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: github.com/m3db/m3/src/metrics/generated/proto/transformationpb/transformation.proto -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -52,17 +52,23 @@ const ( TransformationType_UNKNOWN TransformationType = 0 TransformationType_ABSOLUTE TransformationType = 1 TransformationType_PERSECOND TransformationType = 2 + TransformationType_INCREASE TransformationType = 3 + TransformationType_ADD TransformationType = 4 ) var TransformationType_name = map[int32]string{ 0: "UNKNOWN", 1: "ABSOLUTE", 2: "PERSECOND", + 3: "INCREASE", + 4: "ADD", } var TransformationType_value = map[string]int32{ "UNKNOWN": 0, "ABSOLUTE": 1, "PERSECOND": 2, + "INCREASE": 3, + "ADD": 4, } func (x TransformationType) String() string { @@ -81,17 +87,18 @@ func init() { } var fileDescriptorTransformation = []byte{ - // 180 bytes of a gzipped FileDescriptorProto + // 201 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x0a, 0x49, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0xcf, 0x35, 0x4e, 0x49, 0xd2, 0xcf, 0x35, 0xd6, 0x2f, 0x2e, 0x4a, 0xd6, 0xcf, 0x4d, 0x2d, 0x29, 0xca, 0x4c, 0x2e, 0xd6, 0x4f, 0x4f, 0xcd, 0x4b, 0x2d, 0x4a, 0x2c, 0x49, 0x4d, 0xd1, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0xd7, 0x2f, 0x29, 0x4a, 0xcc, 0x2b, 0x4e, 0xcb, 0x2f, 0xca, 0x4d, 0x2c, 0xc9, 0xcc, 0xcf, 0x2b, 0x48, 0x42, 0x13, 0xd0, 0x03, 0xab, - 0x12, 0x12, 0x40, 0x57, 0xa6, 0x65, 0xc7, 0x25, 0x14, 0x82, 0x22, 0x16, 0x52, 0x59, 0x90, 0x2a, + 0x12, 0x12, 0x40, 0x57, 0xa6, 0x15, 0xca, 0x25, 0x14, 0x82, 0x22, 0x16, 0x52, 0x59, 0x90, 0x2a, 0xc4, 0xcd, 0xc5, 0x1e, 0xea, 0xe7, 0xed, 0xe7, 0x1f, 0xee, 0x27, 0xc0, 0x20, 0xc4, 0xc3, 0xc5, 0xe1, 0xe8, 0x14, 0xec, 0xef, 0x13, 0x1a, 0xe2, 0x2a, 0xc0, 0x28, 0xc4, 0xcb, 0xc5, 0x19, 0xe0, - 0x1a, 0x14, 0xec, 0xea, 0xec, 0xef, 0xe7, 0x22, 0xc0, 0xe4, 0x14, 0x78, 0xe2, 0x91, 0x1c, 0xe3, - 0x85, 0x47, 0x72, 0x8c, 0x0f, 0x1e, 0xc9, 0x31, 0x4e, 0x78, 0x2c, 0xc7, 0x10, 0x65, 0x4f, 0xa1, - 0xcb, 0x93, 0xd8, 0xc0, 0xe2, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0x1e, 0xe7, 0xc2, 0x13, - 0x03, 0x01, 0x00, 0x00, + 0x1a, 0x14, 0xec, 0xea, 0xec, 0xef, 0xe7, 0x22, 0xc0, 0x04, 0x92, 0xf4, 0xf4, 0x73, 0x0e, 0x72, + 0x75, 0x0c, 0x76, 0x15, 0x60, 0x16, 0x62, 0xe7, 0x62, 0x76, 0x74, 0x71, 0x11, 0x60, 0x71, 0x0a, + 0x3c, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, + 0x63, 0x88, 0xb2, 0xa7, 0xd0, 0x43, 0x49, 0x6c, 0x60, 0x71, 0x63, 0x40, 0x00, 0x00, 0x00, 0xff, + 0xff, 0xf3, 0x67, 0xfb, 0xf2, 0x1a, 0x01, 0x00, 0x00, } diff --git a/src/metrics/generated/proto/transformationpb/transformation.proto b/src/metrics/generated/proto/transformationpb/transformation.proto index 681a490b17..640f043bfc 100644 --- a/src/metrics/generated/proto/transformationpb/transformation.proto +++ b/src/metrics/generated/proto/transformationpb/transformation.proto @@ -28,4 +28,6 @@ enum TransformationType { UNKNOWN = 0; ABSOLUTE = 1; PERSECOND = 2; + INCREASE = 3; + ADD = 4; } diff --git a/src/metrics/metric/aggregated/types.go b/src/metrics/metric/aggregated/types.go index fad7dd952a..65f9723b25 100644 --- a/src/metrics/metric/aggregated/types.go +++ b/src/metrics/metric/aggregated/types.go @@ -229,3 +229,28 @@ func (tm *TimedMetricWithMetadata) FromProto(pb *metricpb.TimedMetricWithMetadat } return tm.TimedMetadata.FromProto(pb.Metadata) } + +// TimedMetricWithMetadatas is a timed metric with staged metadatas. +type TimedMetricWithMetadatas struct { + Metric + metadata.StagedMetadatas +} + +// ToProto converts the timed metric with metadata to a protobuf message in place. +func (tm TimedMetricWithMetadatas) ToProto(pb *metricpb.TimedMetricWithMetadatas) error { + if err := tm.Metric.ToProto(&pb.Metric); err != nil { + return err + } + return tm.StagedMetadatas.ToProto(&pb.Metadatas) +} + +// FromProto converts the protobuf message to a timed metric with metadata in place. +func (tm *TimedMetricWithMetadatas) FromProto(pb *metricpb.TimedMetricWithMetadatas) error { + if pb == nil { + return errNilTimedMetricWithMetadataProto + } + if err := tm.Metric.FromProto(pb.Metric); err != nil { + return err + } + return tm.StagedMetadatas.FromProto(pb.Metadatas) +} diff --git a/src/metrics/transformation/binary.go b/src/metrics/transformation/binary.go index 20b9810654..593b8fab53 100644 --- a/src/metrics/transformation/binary.go +++ b/src/metrics/transformation/binary.go @@ -29,8 +29,20 @@ const ( nanosPerSecond = time.Second / time.Nanosecond ) +var ( + // allows to use a single transform fn ref (instead of + // taking reference to it each time when converting to iface). + transformPerSecondFn = BinaryTransformFn(perSecond) + transformIncreaseFn = BinaryTransformFn(increase) +) + +func transformPerSecond() BinaryTransform { + return transformPerSecondFn +} + // perSecond computes the derivative between consecutive datapoints, taking into // account the time interval between the values. +// Note: // * It skips NaN values. // * It assumes the timestamps are monotonically increasing, and values are non-decreasing. // If either of the two conditions is not met, an empty datapoint is returned. @@ -45,3 +57,24 @@ func perSecond(prev, curr Datapoint) Datapoint { rate := diff * float64(nanosPerSecond) / float64(curr.TimeNanos-prev.TimeNanos) return Datapoint{TimeNanos: curr.TimeNanos, Value: rate} } + +func transformIncrease() BinaryTransform { + return transformIncreaseFn +} + +// increase computes the difference between consecutive datapoints, unlike +// perSecond it does not account for the time interval between the values. +// Note: +// * It skips NaN values. +// * It assumes the timestamps are monotonically increasing, and values are non-decreasing. +// If either of the two conditions is not met, an empty datapoint is returned. +func increase(prev, curr Datapoint) Datapoint { + if prev.TimeNanos >= curr.TimeNanos || math.IsNaN(prev.Value) || math.IsNaN(curr.Value) { + return emptyDatapoint + } + diff := curr.Value - prev.Value + if diff < 0 { + return emptyDatapoint + } + return Datapoint{TimeNanos: curr.TimeNanos, Value: diff} +} diff --git a/src/metrics/transformation/func.go b/src/metrics/transformation/func.go index 016fe92ffd..80016bb49b 100644 --- a/src/metrics/transformation/func.go +++ b/src/metrics/transformation/func.go @@ -38,9 +38,31 @@ func (dp Datapoint) IsEmpty() bool { return math.IsNaN(dp.Value) } // UnaryTransform is a unary transformation that takes a single // datapoint as input and transforms it into a datapoint as output. -type UnaryTransform func(dp Datapoint) Datapoint +// It can keep state if it requires. +type UnaryTransform interface { + Evaluate(dp Datapoint) Datapoint +} + +// UnaryTransformFn implements UnaryTransform as a function. +type UnaryTransformFn func(dp Datapoint) Datapoint + +// Evaluate implements UnaryTransform as a function. +func (fn UnaryTransformFn) Evaluate(dp Datapoint) Datapoint { + return fn(dp) +} // BinaryTransform is a binary transformation that takes the // previous and the current datapoint as input and produces // a single datapoint as the transformation result. -type BinaryTransform func(prev, curr Datapoint) Datapoint +// It can keep state if it requires. +type BinaryTransform interface { + Evaluate(prev, curr Datapoint) Datapoint +} + +// BinaryTransformFn implements BinaryTransform as a function. +type BinaryTransformFn func(prev, curr Datapoint) Datapoint + +// Evaluate implements BinaryTransform as a function. +func (fn BinaryTransformFn) Evaluate(prev, curr Datapoint) Datapoint { + return fn(prev, curr) +} diff --git a/src/metrics/transformation/type.go b/src/metrics/transformation/type.go index 56d1b2fd6e..fe0edae0f1 100644 --- a/src/metrics/transformation/type.go +++ b/src/metrics/transformation/type.go @@ -1,3 +1,4 @@ +// go:generate stringer -type=Type // Copyright (c) 2017 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy @@ -34,6 +35,8 @@ const ( UnknownType Type = iota Absolute PerSecond + Increase + Add ) // IsValid checks if the transformation type is valid. @@ -53,6 +56,32 @@ func (t Type) IsBinaryTransform() bool { return exists } +// NewOp returns a constructed operation that is allocated once and can be +// reused. +func (t Type) NewOp() (Op, error) { + var ( + err error + unary UnaryTransform + binary BinaryTransform + ) + switch { + case t.IsUnaryTransform(): + unary, err = t.UnaryTransform() + case t.IsBinaryTransform(): + binary, err = t.BinaryTransform() + default: + err = fmt.Errorf("unknown transformation type: %v", t) + } + if err != nil { + return Op{}, err + } + return Op{ + opType: t, + unary: unary, + binary: binary, + }, nil +} + // UnaryTransform returns the unary transformation function associated with // the transformation type if applicable, or an error otherwise. func (t Type) UnaryTransform() (UnaryTransform, error) { @@ -60,7 +89,7 @@ func (t Type) UnaryTransform() (UnaryTransform, error) { if !exists { return nil, fmt.Errorf("%v is not a unary transfomration", t) } - return tf, nil + return tf(), nil } // MustUnaryTransform returns the unary transformation function associated with @@ -80,7 +109,7 @@ func (t Type) BinaryTransform() (BinaryTransform, error) { if !exists { return nil, fmt.Errorf("%v is not a binary transfomration", t) } - return tf, nil + return tf(), nil } // MustBinaryTransform returns the binary transformation function associated with @@ -100,6 +129,10 @@ func (t Type) ToProto(pb *transformationpb.TransformationType) error { *pb = transformationpb.TransformationType_ABSOLUTE case PerSecond: *pb = transformationpb.TransformationType_PERSECOND + case Increase: + *pb = transformationpb.TransformationType_INCREASE + case Add: + *pb = transformationpb.TransformationType_ADD default: return fmt.Errorf("unknown transformation type: %v", t) } @@ -113,6 +146,10 @@ func (t *Type) FromProto(pb transformationpb.TransformationType) error { *t = Absolute case transformationpb.TransformationType_PERSECOND: *t = PerSecond + case transformationpb.TransformationType_INCREASE: + *t = Increase + case transformationpb.TransformationType_ADD: + *t = Add default: return fmt.Errorf("unknown transformation type in proto: %v", pb) } @@ -160,12 +197,44 @@ func ParseType(str string) (Type, error) { return t, nil } +// Op represents a transform operation. +type Op struct { + opType Type + + // might have either unary or binary + unary UnaryTransform + binary BinaryTransform +} + +// Type returns the op type. +func (o Op) Type() Type { + return o.opType +} + +// UnaryTransform returns the active unary transform if op is unary transform. +func (o Op) UnaryTransform() (UnaryTransform, bool) { + if !o.Type().IsUnaryTransform() { + return nil, false + } + return o.unary, true +} + +// BinaryTransform returns the active binary transform if op is binary transform. +func (o Op) BinaryTransform() (BinaryTransform, bool) { + if !o.Type().IsBinaryTransform() { + return nil, false + } + return o.binary, true +} + var ( - unaryTransforms = map[Type]UnaryTransform{ - Absolute: absolute, + unaryTransforms = map[Type]func() UnaryTransform{ + Absolute: transformAbsolute, + Add: transformAdd, } - binaryTransforms = map[Type]BinaryTransform{ - PerSecond: perSecond, + binaryTransforms = map[Type]func() BinaryTransform{ + PerSecond: transformPerSecond, + Increase: transformIncrease, } typeStringMap map[string]Type ) diff --git a/src/metrics/transformation/type_string.go b/src/metrics/transformation/type_string.go index 37cb879c5c..028d8a4bc1 100644 --- a/src/metrics/transformation/type_string.go +++ b/src/metrics/transformation/type_string.go @@ -1,4 +1,6 @@ -// Copyright (c) 2017 Uber Technologies, Inc. +// Code generated by "stringer -type=Type"; DO NOT EDIT. + +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -18,19 +20,28 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -// generated by stringer -type=Type; DO NOT EDIT - package transformation -import "fmt" +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[UnknownType-0] + _ = x[Absolute-1] + _ = x[PerSecond-2] + _ = x[Increase-3] + _ = x[Add-4] +} -const _Type_name = "UnknownTypeAbsolutePerSecond" +const _Type_name = "UnknownTypeAbsolutePerSecondIncreaseAdd" -var _Type_index = [...]uint8{0, 11, 19, 28} +var _Type_index = [...]uint8{0, 11, 19, 28, 36, 39} func (i Type) String() string { if i < 0 || i >= Type(len(_Type_index)-1) { - return fmt.Sprintf("Type(%d)", i) + return "Type(" + strconv.FormatInt(int64(i), 10) + ")" } return _Type_name[_Type_index[i]:_Type_index[i+1]] } diff --git a/src/metrics/transformation/unary.go b/src/metrics/transformation/unary.go index eec1d9a246..00f9d493f7 100644 --- a/src/metrics/transformation/unary.go +++ b/src/metrics/transformation/unary.go @@ -22,9 +22,33 @@ package transformation import "math" +var ( + // allows to use a single transform fn ref (instead of + // taking reference to it each time when converting to iface). + transformAbsoluteFn = UnaryTransformFn(absolute) +) + +func transformAbsolute() UnaryTransform { + return transformAbsoluteFn +} + func absolute(dp Datapoint) Datapoint { var res Datapoint res.TimeNanos = dp.TimeNanos res.Value = math.Abs(dp.Value) return res } + +// add will add add a datapoint to a running count and return the result, useful +// for computing a running sum of values (like a monotonic increasing counter). +// Note: +// * It treats NaN as zero value, i.e. 42 + NaN = 42. +func transformAdd() UnaryTransform { + var curr float64 + return UnaryTransformFn(func(dp Datapoint) Datapoint { + if !math.IsNaN(dp.Value) { + curr += dp.Value + } + return Datapoint{TimeNanos: dp.TimeNanos, Value: curr} + }) +}