From 733a8139dd481b11beb0651df3e102f67b89c242 Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Sun, 17 May 2020 21:43:45 -0700 Subject: [PATCH 1/6] filterprocessor for filtering (dropping) incoming metrics by various properties traceprocessor for spans to come in a separate PR. --- processor/filterprocessor/config.go | 39 +++ processor/filterprocessor/config_test.go | 241 ++++++++++++++++++ processor/filterprocessor/doc.go | 17 ++ processor/filterprocessor/factory.go | 67 +++++ processor/filterprocessor/factory_test.go | 80 ++++++ processor/filterprocessor/filter_processor.go | 123 +++++++++ .../filterprocessor/filter_processor_test.go | 195 ++++++++++++++ .../testdata/config_invalid.yaml | 26 ++ .../testdata/config_regexp.yaml | 79 ++++++ .../testdata/config_strict.yaml | 59 +++++ processor/processortest/caching_processor.go | 46 ++++ 11 files changed, 972 insertions(+) create mode 100644 processor/filterprocessor/config.go create mode 100644 processor/filterprocessor/config_test.go create mode 100644 processor/filterprocessor/doc.go create mode 100644 processor/filterprocessor/factory.go create mode 100644 processor/filterprocessor/factory_test.go create mode 100644 processor/filterprocessor/filter_processor.go create mode 100644 processor/filterprocessor/filter_processor_test.go create mode 100644 processor/filterprocessor/testdata/config_invalid.yaml create mode 100644 processor/filterprocessor/testdata/config_regexp.yaml create mode 100644 processor/filterprocessor/testdata/config_strict.yaml create mode 100644 processor/processortest/caching_processor.go diff --git a/processor/filterprocessor/config.go b/processor/filterprocessor/config.go new file mode 100644 index 00000000000..02889ce275f --- /dev/null +++ b/processor/filterprocessor/config.go @@ -0,0 +1,39 @@ +// Copyright 2020 OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "go.opentelemetry.io/collector/config/configmodels" + "go.opentelemetry.io/collector/internal/processor/filtermetric" +) + +// Config defines configuration for Resource processor. +type Config struct { + configmodels.ProcessorSettings `mapstructure:",squash"` + Metrics MetricFilters `mapstructure:"metrics"` +} + +// MetricFilter filters by Metric properties. +type MetricFilters struct { + // Include match properties describe metrics that should be included in the Collector Service pipeline, + // all other metrics should be dropped from further processing. + // If both Include and Exclude are specified, Include filtering occurs first. + Include *filtermetric.MatchProperties `mapstructure:"include"` + + // Exclude match properties describe metrics that should be excluded from the Collector Service pipeline, + // all other metrics should be included. + // If both Include and Exclude are specified, Include filtering occurs first. + Exclude *filtermetric.MatchProperties `mapstructure:"exclude"` +} diff --git a/processor/filterprocessor/config_test.go b/processor/filterprocessor/config_test.go new file mode 100644 index 00000000000..27deb965a88 --- /dev/null +++ b/processor/filterprocessor/config_test.go @@ -0,0 +1,241 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "path" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "go.opentelemetry.io/collector/config" + "go.opentelemetry.io/collector/config/configmodels" + "go.opentelemetry.io/collector/internal/processor/filtermetric" + "go.opentelemetry.io/collector/internal/processor/filterset" + fsregexp "go.opentelemetry.io/collector/internal/processor/filterset/regexp" +) + +// TestLoadingConfigRegexp tests loading testdata/config_strict.yaml +func TestLoadingConfigStrict(t *testing.T) { + // list of filters used repeatedly on testdata/config_strict.yaml + testDataFilters := []string{ + "hello_world", + "hello/world", + } + + testDataMetricProperties := &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + MetricNames: testDataFilters, + } + + factories, err := config.ExampleComponents() + assert.Nil(t, err) + + factory := &Factory{} + factories.Processors[configmodels.Type(typeStr)] = factory + config, err := config.LoadConfigFile(t, path.Join(".", "testdata", "config_strict.yaml"), factories) + + assert.Nil(t, err) + require.NotNil(t, config) + + tests := []struct { + filterName string + expCfg *Config + }{ + { + filterName: "filter/empty", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/empty", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + }, + }, + }, + }, { + filterName: "filter/include", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/include", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: testDataMetricProperties, + }, + }, + }, { + filterName: "filter/exclude", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/exclude", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Exclude: testDataMetricProperties, + }, + }, + }, { + filterName: "filter/includeexclude", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/includeexclude", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: testDataMetricProperties, + Exclude: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + MetricNames: []string{"hello_world"}, + }, + }, + }, + }, { + filterName: "filter/strictwithconfig", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/strictwithconfig", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: testDataMetricProperties, + }, + }, + }, + } + + for _, test := range tests { + t.Run(test.filterName, func(t *testing.T) { + cfg := config.Processors[test.filterName] + assert.Equal(t, test.expCfg, cfg) + }) + } +} + +// TestLoadingConfigRegexp tests loading testdata/config_regexp.yaml +func TestLoadingConfigRegexp(t *testing.T) { + // list of filters used repeatedly on testdata/config.yaml + testDataFilters := []string{ + "prefix/.*", + "prefix_.*", + ".*/suffix", + ".*_suffix", + ".*/contains/.*", + ".*_contains_.*", + "full/name/match", + "full_name_match", + } + + testDataMetricProperties := &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Regexp, + }, + MetricNames: testDataFilters, + } + + factories, err := config.ExampleComponents() + assert.Nil(t, err) + + factory := &Factory{} + factories.Processors[typeStr] = factory + config, err := config.LoadConfigFile(t, path.Join(".", "testdata", "config_regexp.yaml"), factories) + + assert.Nil(t, err) + require.NotNil(t, config) + + tests := []struct { + filterName string + expCfg *Config + }{ + { + filterName: "filter/include", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/include", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: testDataMetricProperties, + }, + }, + }, { + filterName: "filter/exclude", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/exclude", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Exclude: testDataMetricProperties, + }, + }, + }, { + filterName: "filter/unlimitedcache", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/unlimitedcache", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Include: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Regexp, + RegexpConfig: &fsregexp.Config{ + CacheEnabled: true, + }, + }, + MetricNames: testDataFilters, + }, + }, + }, + }, { + filterName: "filter/limitedcache", + expCfg: &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: "filter/limitedcache", + TypeVal: typeStr, + }, + Metrics: MetricFilters{ + Exclude: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Regexp, + RegexpConfig: &fsregexp.Config{ + CacheEnabled: true, + CacheMaxNumEntries: 10, + }, + }, + MetricNames: testDataFilters, + }, + }, + }, + }, + } + + for _, test := range tests { + t.Run(test.filterName, func(t *testing.T) { + cfg := config.Processors[test.filterName] + assert.Equal(t, test.expCfg, cfg) + }) + } +} diff --git a/processor/filterprocessor/doc.go b/processor/filterprocessor/doc.go new file mode 100644 index 00000000000..aa74b8715a1 --- /dev/null +++ b/processor/filterprocessor/doc.go @@ -0,0 +1,17 @@ +// Copyright 2019 OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package filterprocessor implements a processor for filtering +// (dropping) metrics and/or spans by various properties. +package filterprocessor diff --git a/processor/filterprocessor/factory.go b/processor/filterprocessor/factory.go new file mode 100644 index 00000000000..c5af4fdcf18 --- /dev/null +++ b/processor/filterprocessor/factory.go @@ -0,0 +1,67 @@ +// Copyright 2019 OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "go.uber.org/zap" + + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/config/configerror" + "go.opentelemetry.io/collector/config/configmodels" + "go.opentelemetry.io/collector/consumer" +) + +const ( + // The value of "type" key in configuration. + typeStr = "filter" +) + +// Factory is the factory for filter processor. +type Factory struct { +} + +// Type gets the type of the Option config created by this factory. +func (f Factory) Type() configmodels.Type { + return typeStr +} + +// CreateDefaultConfig creates the default configuration for processor. +func (f Factory) CreateDefaultConfig() configmodels.Processor { + return &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + TypeVal: typeStr, + NameVal: typeStr, + }, + } +} + +// CreateTraceProcessor creates a trace processor based on this config. +func (f Factory) CreateTraceProcessor( + logger *zap.Logger, + nextConsumer consumer.TraceConsumerOld, + c configmodels.Processor, +) (component.TraceProcessorOld, error) { + return nil, configerror.ErrDataTypeIsNotSupported +} + +// CreateMetricsProcessor creates a metrics processor based on this config. +func (f Factory) CreateMetricsProcessor( + logger *zap.Logger, + nextConsumer consumer.MetricsConsumerOld, + cfg configmodels.Processor, +) (component.MetricsProcessorOld, error) { + oCfg := cfg.(*Config) + return newFilterMetricProcessor(nextConsumer, oCfg) +} diff --git a/processor/filterprocessor/factory_test.go b/processor/filterprocessor/factory_test.go new file mode 100644 index 00000000000..787da7350ab --- /dev/null +++ b/processor/filterprocessor/factory_test.go @@ -0,0 +1,80 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "fmt" + "path" + "testing" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap" + + "go.opentelemetry.io/collector/config" + "go.opentelemetry.io/collector/config/configcheck" + "go.opentelemetry.io/collector/config/configmodels" +) + +func TestCreateDefaultConfig(t *testing.T) { + factory := Factory{} + cfg := factory.CreateDefaultConfig() + assert.Equal(t, cfg, &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + NameVal: typeStr, + TypeVal: typeStr, + }, + }) + assert.NoError(t, configcheck.ValidateConfig(cfg)) +} + +func TestCreateProcessors(t *testing.T) { + tests := []struct { + configName string + succeed bool + }{ + { + configName: "config_regexp.yaml", + succeed: true, + }, { + configName: "config_strict.yaml", + succeed: true, + }, { + configName: "config_invalid.yaml", + succeed: false, + }, + } + + for _, test := range tests { + factories, err := config.ExampleComponents() + assert.Nil(t, err) + + factory := &Factory{} + factories.Processors[typeStr] = factory + config, err := config.LoadConfigFile(t, path.Join(".", "testdata", test.configName), factories) + + for name, cfg := range config.Processors { + t.Run(fmt.Sprintf("%s/%s", test.configName, name), func(t *testing.T) { + tp, tErr := factory.CreateTraceProcessor(zap.NewNop(), nil, cfg) + // Not implemented error + assert.NotNil(t, tErr) + assert.Nil(t, tp) + + mp, mErr := factory.CreateMetricsProcessor(zap.NewNop(), nil, cfg) + assert.Equal(t, test.succeed, mp != (*filterMetricProcessor)(nil)) + assert.Equal(t, test.succeed, mErr == nil) + }) + } + } +} diff --git a/processor/filterprocessor/filter_processor.go b/processor/filterprocessor/filter_processor.go new file mode 100644 index 00000000000..89328d2e639 --- /dev/null +++ b/processor/filterprocessor/filter_processor.go @@ -0,0 +1,123 @@ +// Copyright 2020 OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "context" + + metricspb "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1" + + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/consumer/consumerdata" + "go.opentelemetry.io/collector/internal/processor/filtermetric" +) + +type filterMetricProcessor struct { + cfg *Config + capabilities component.ProcessorCapabilities + next consumer.MetricsConsumerOld + include *filtermetric.Matcher + exclude *filtermetric.Matcher +} + +var _ component.MetricsProcessorOld = (*filterMetricProcessor)(nil) + +func newFilterMetricProcessor(next consumer.MetricsConsumerOld, cfg *Config) (*filterMetricProcessor, error) { + inc, err := createMatcher(cfg.Metrics.Include) + if err != nil { + return nil, err + } + + exc, err := createMatcher(cfg.Metrics.Exclude) + if err != nil { + return nil, err + } + + return &filterMetricProcessor{ + cfg: cfg, + capabilities: component.ProcessorCapabilities{MutatesConsumedData: false}, + next: next, + include: inc, + exclude: exc, + }, nil +} + +func createMatcher(mp *filtermetric.MatchProperties) (*filtermetric.Matcher, error) { + // Nothing specified in configuration + if mp == nil { + return nil, nil + } + + matcher, err := filtermetric.NewMatcher(mp) + if err != nil { + return nil, err + } + + return &matcher, nil +} + +// GetCapabilities returns the Capabilities assocciated with the resource processor. +func (fmp *filterMetricProcessor) GetCapabilities() component.ProcessorCapabilities { + return fmp.capabilities +} + +// Start is invoked during service startup. +func (*filterMetricProcessor) Start(ctx context.Context, host component.Host) error { + return nil +} + +// Shutdown is invoked during service shutdown. +func (*filterMetricProcessor) Shutdown(ctx context.Context) error { + return nil +} + +// ConsumeMetricsData implements the MetricsProcessor interface +func (fmp *filterMetricProcessor) ConsumeMetricsData(ctx context.Context, md consumerdata.MetricsData) error { + return fmp.next.ConsumeMetricsData(ctx, consumerdata.MetricsData{ + Node: md.Node, + Resource: md.Resource, + Metrics: fmp.filterMetrics(md.Metrics), + }) +} + +// filterMetrics filters the given spans based off the filterMetricProcessor's filters. +func (fmp *filterMetricProcessor) filterMetrics(metrics []*metricspb.Metric) []*metricspb.Metric { + keep := []*metricspb.Metric{} + for _, m := range metrics { + if fmp.shouldKeepMetric(m) { + keep = append(keep, m) + } + } + + return keep +} + +// shouldKeepMetric determines whether a metric should be kept based off the filterMetricProcessor's filters. +func (fmp *filterMetricProcessor) shouldKeepMetric(metric *metricspb.Metric) bool { + if fmp.include != nil { + if !fmp.include.MatchMetric(metric) { + return false + } + } + + if fmp.exclude != nil { + if fmp.exclude.MatchMetric(metric) { + return false + } + } + + return true +} diff --git a/processor/filterprocessor/filter_processor_test.go b/processor/filterprocessor/filter_processor_test.go new file mode 100644 index 00000000000..b8dc988de3c --- /dev/null +++ b/processor/filterprocessor/filter_processor_test.go @@ -0,0 +1,195 @@ +// Copyright 2020 OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package filterprocessor + +import ( + "context" + "testing" + + metricspb "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "go.opentelemetry.io/collector/config/configmodels" + "go.opentelemetry.io/collector/consumer/consumerdata" + "go.opentelemetry.io/collector/internal/processor/filtermetric" + "go.opentelemetry.io/collector/internal/processor/filterset" + ptest "go.opentelemetry.io/collector/processor/processortest" +) + +var ( + validFilters = []string{ + "prefix/.*", + "prefix_.*", + ".*/suffix", + ".*_suffix", + ".*/contains/.*", + ".*_contains_.*", + "full/name/match", + "full_name_match", + } +) + +func createFilterProcessorConfig(matchType filterset.MatchType, inc []string, exc []string) *Config { + return &Config{} + +} + +func TestFilterMetricProcessor(t *testing.T) { + inMetricNames := []string{ + "full_name_match", + "not_exact_string_match", + "prefix/test/match", + "prefix_test_match", + "prefixprefix/test/match", + "test/match/suffix", + "test_match_suffix", + "test/match/suffixsuffix", + "test/contains/match", + "test_contains_match", + "random", + "full/name/match", + "full_name_match", // repeats + "not_exact_string_match", + } + + regexpMetricsFilterProperties := &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Regexp, + }, + MetricNames: validFilters, + } + + tests := []struct { + name string + cfg *Config + inc *filtermetric.MatchProperties + exc *filtermetric.MatchProperties + inMN []string // input Metric names + outMN []string // output Metric names + }{ + { + name: "includeFilter", + inc: regexpMetricsFilterProperties, + inMN: inMetricNames, + outMN: []string{ + "full_name_match", + "prefix/test/match", + "prefix_test_match", + "prefixprefix/test/match", + "test/match/suffix", + "test_match_suffix", + "test/match/suffixsuffix", + "test/contains/match", + "test_contains_match", + "full/name/match", + "full_name_match", + }, + }, { + name: "excludeFilter", + exc: regexpMetricsFilterProperties, + inMN: inMetricNames, + outMN: []string{ + "not_exact_string_match", + "random", + "not_exact_string_match", + }, + }, { + name: "includeAndExclude", + inc: regexpMetricsFilterProperties, + exc: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + MetricNames: []string{ + "prefix_test_match", + "test_contains_match", + }, + }, + inMN: inMetricNames, + outMN: []string{ + "full_name_match", + "prefix/test/match", + // "prefix_test_match", excluded by exclude filter + "prefixprefix/test/match", + "test/match/suffix", + "test_match_suffix", + "test/match/suffixsuffix", + "test/contains/match", + // "test_contains_match", excluded by exclude filter + "full/name/match", + "full_name_match", + }, + }, { + name: "emptyFilterInclude", + inc: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + }, + inMN: inMetricNames, + outMN: []string{}, + }, { + name: "emptyFilterExclude", + exc: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + }, + inMN: inMetricNames, + outMN: inMetricNames, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + // next stores the results of the filter metric processor + next := &ptest.CachingMetricsConsumer{} + cfg := &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + TypeVal: typeStr, + NameVal: typeStr, + }, + Metrics: MetricFilters{ + Include: test.inc, + Exclude: test.exc, + }, + } + fmp, err := newFilterMetricProcessor(next, cfg) + assert.NotNil(t, fmp) + assert.Nil(t, err) + + md := consumerdata.MetricsData{ + Metrics: make([]*metricspb.Metric, len(test.inMN)), + } + + for idx, in := range test.inMN { + md.Metrics[idx] = &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: in, + }, + } + } + + cErr := fmp.ConsumeMetricsData(context.Background(), md) + assert.Nil(t, cErr) + + require.Equal(t, len(test.outMN), len(next.Data.Metrics)) + for idx, out := range next.Data.Metrics { + assert.Equal(t, test.outMN[idx], out.MetricDescriptor.Name) + } + }) + } +} diff --git a/processor/filterprocessor/testdata/config_invalid.yaml b/processor/filterprocessor/testdata/config_invalid.yaml new file mode 100644 index 00000000000..d9be2ba2c82 --- /dev/null +++ b/processor/filterprocessor/testdata/config_invalid.yaml @@ -0,0 +1,26 @@ +receivers: + examplereceiver: + +processors: + filter/include: + # any names NOT matching filters are excluded from remainder of pipeline + metrics: + include: + match_type: regexp + metric_names: + # re2 regexp patterns + - (\W|^)stock\stips(\W|$ + +exporters: + exampleexporter: + +service: + pipelines: + traces: + receivers: [examplereceiver] + processors: [filter/include] + exporters: [exampleexporter] + metrics: + receivers: [examplereceiver] + processors: [filter/include] + exporters: [exampleexporter] \ No newline at end of file diff --git a/processor/filterprocessor/testdata/config_regexp.yaml b/processor/filterprocessor/testdata/config_regexp.yaml new file mode 100644 index 00000000000..db13c3f4643 --- /dev/null +++ b/processor/filterprocessor/testdata/config_regexp.yaml @@ -0,0 +1,79 @@ +receivers: + examplereceiver: + +processors: + filter: + filter/include: + # any names NOT matching filters are excluded from remainder of pipeline + metrics: + include: + match_type: regexp + metric_names: + # re2 regexp patterns + - prefix/.* + - prefix_.* + - .*/suffix + - .*_suffix + - .*/contains/.* + - .*_contains_.* + - full/name/match + - full_name_match + filter/exclude: + # any names matching filters are excluded from remainder of pipeline + metrics: + exclude: + match_type: regexp + metric_names: + - prefix/.* + - prefix_.* + - .*/suffix + - .*_suffix + - .*/contains/.* + - .*_contains_.* + - full/name/match + - full_name_match + filter/unlimitedcache: + metrics: + include: + match_type: regexp + regexp: + cacheenabled: true + metric_names: + - prefix/.* + - prefix_.* + - .*/suffix + - .*_suffix + - .*/contains/.* + - .*_contains_.* + - full/name/match + - full_name_match + filter/limitedcache: + metrics: + exclude: + match_type: regexp + metric_names: + - prefix/.* + - prefix_.* + - .*/suffix + - .*_suffix + - .*/contains/.* + - .*_contains_.* + - full/name/match + - full_name_match + regexp: + cacheenabled: true + cachemaxnumentries: 10 + +exporters: + exampleexporter: + +service: + pipelines: + traces: + receivers: [examplereceiver] + processors: [filter] + exporters: [exampleexporter] + metrics: + receivers: [examplereceiver] + processors: [filter] + exporters: [exampleexporter] \ No newline at end of file diff --git a/processor/filterprocessor/testdata/config_strict.yaml b/processor/filterprocessor/testdata/config_strict.yaml new file mode 100644 index 00000000000..e307f61b168 --- /dev/null +++ b/processor/filterprocessor/testdata/config_strict.yaml @@ -0,0 +1,59 @@ +receivers: + examplereceiver: + +processors: + filter/empty: + metrics: + include: + match_type: strict + filter/include: + metrics: + # any names NOT matching filters are excluded from remainder of pipeline + include: + match_type: strict + metric_names: + - hello_world + - hello/world + filter/exclude: + metrics: + # any names matching filters are excluded from remainder of pipeline + exclude: + match_type: strict + metric_names: + - hello_world + - hello/world + filter/includeexclude: + metrics: + # if both include and exclude are specified, include filters are applied first + # the following configuration would only allow metrics named "hello/world" to pass through + include: + match_type: strict + metric_names: + - hello_world + - hello/world + exclude: + match_type: strict + metric_names: + - hello_world + filter/strictwithconfig: + metrics: + include: + match_type: strict + strict: + metric_names: + - hello_world + - hello/world + +exporters: + exampleexporter: + +service: + pipelines: + traces: + receivers: [examplereceiver] + processors: [filter/empty] + exporters: [exampleexporter] + metrics: + receivers: [examplereceiver] + processors: [filter/empty] + exporters: [exampleexporter] \ No newline at end of file diff --git a/processor/processortest/caching_processor.go b/processor/processortest/caching_processor.go new file mode 100644 index 00000000000..8d41aaf63da --- /dev/null +++ b/processor/processortest/caching_processor.go @@ -0,0 +1,46 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package processortest + +import ( + "context" + + "go.opentelemetry.io/collector/consumer/consumerdata" +) + +// CachingTraceConsumer is a dummy consumer that simply stores the trace data. +// When testing processors, it can be specified as the next consumer of a processor +// to inspect the outputs of the processor being tested. +type CachingTraceConsumerOld struct { + Data consumerdata.TraceData +} + +// ConsumeTraceData stores the trace data and returns nil. +func (ctc *CachingTraceConsumerOld) ConsumeTraceData(ctx context.Context, td consumerdata.TraceData) error { + ctc.Data = td + return nil +} + +// CachingMetricsConsumer is a dummy consumer that simply stores the metrics data. +// When testing processors, it can be specified as the next consumer of a processor +// to inspect the outputs of the processor being tested. +type CachingMetricsConsumer struct { + Data consumerdata.MetricsData +} + +// ConsumeMetricData stores the metrics data and returns nil. +func (cmc *CachingMetricsConsumer) ConsumeMetricsData(ctx context.Context, md consumerdata.MetricsData) error { + cmc.Data = md + return nil +} From a84904d8e8bb7b856e221a9384dfce5ad0d7133e Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Tue, 2 Jun 2020 18:18:33 -0700 Subject: [PATCH 2/6] Add benchmark test and use new trace processor format --- processor/filterprocessor/factory.go | 13 ++- processor/filterprocessor/factory_test.go | 7 +- .../filterprocessor/filter_processor_test.go | 108 +++++++++++++++--- 3 files changed, 104 insertions(+), 24 deletions(-) diff --git a/processor/filterprocessor/factory.go b/processor/filterprocessor/factory.go index c5af4fdcf18..81e6fa4c716 100644 --- a/processor/filterprocessor/factory.go +++ b/processor/filterprocessor/factory.go @@ -15,6 +15,8 @@ package filterprocessor import ( + "context" + "go.uber.org/zap" "go.opentelemetry.io/collector/component" @@ -48,11 +50,12 @@ func (f Factory) CreateDefaultConfig() configmodels.Processor { } // CreateTraceProcessor creates a trace processor based on this config. -func (f Factory) CreateTraceProcessor( - logger *zap.Logger, - nextConsumer consumer.TraceConsumerOld, - c configmodels.Processor, -) (component.TraceProcessorOld, error) { +func (f *Factory) CreateTraceProcessor( + ctx context.Context, + params component.ProcessorCreateParams, + nextConsumer consumer.TraceConsumer, + cfg configmodels.Processor, +) (component.TraceProcessor, error) { return nil, configerror.ErrDataTypeIsNotSupported } diff --git a/processor/filterprocessor/factory_test.go b/processor/filterprocessor/factory_test.go index 787da7350ab..a0f11720d79 100644 --- a/processor/filterprocessor/factory_test.go +++ b/processor/filterprocessor/factory_test.go @@ -15,6 +15,7 @@ package filterprocessor import ( + "context" "fmt" "path" "testing" @@ -22,6 +23,7 @@ import ( "github.com/stretchr/testify/assert" "go.uber.org/zap" + "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/config" "go.opentelemetry.io/collector/config/configcheck" "go.opentelemetry.io/collector/config/configmodels" @@ -66,7 +68,10 @@ func TestCreateProcessors(t *testing.T) { for name, cfg := range config.Processors { t.Run(fmt.Sprintf("%s/%s", test.configName, name), func(t *testing.T) { - tp, tErr := factory.CreateTraceProcessor(zap.NewNop(), nil, cfg) + factory := &Factory{} + creationParams := component.ProcessorCreateParams{Logger: zap.NewNop()} + + tp, tErr := factory.CreateTraceProcessor(context.Background(), creationParams, nil, cfg) // Not implemented error assert.NotNil(t, tErr) assert.Nil(t, tp) diff --git a/processor/filterprocessor/filter_processor_test.go b/processor/filterprocessor/filter_processor_test.go index b8dc988de3c..c4cd98f0245 100644 --- a/processor/filterprocessor/filter_processor_test.go +++ b/processor/filterprocessor/filter_processor_test.go @@ -29,6 +29,15 @@ import ( ptest "go.opentelemetry.io/collector/processor/processortest" ) +type metricNameTest struct { + name string + cfg *Config + inc *filtermetric.MatchProperties + exc *filtermetric.MatchProperties + inMN []string // input Metric names + outMN []string // output Metric names +} + var ( validFilters = []string{ "prefix/.*", @@ -40,15 +49,8 @@ var ( "full/name/match", "full_name_match", } -) - -func createFilterProcessorConfig(matchType filterset.MatchType, inc []string, exc []string) *Config { - return &Config{} - -} -func TestFilterMetricProcessor(t *testing.T) { - inMetricNames := []string{ + inMetricNames = []string{ "full_name_match", "not_exact_string_match", "prefix/test/match", @@ -65,21 +67,14 @@ func TestFilterMetricProcessor(t *testing.T) { "not_exact_string_match", } - regexpMetricsFilterProperties := &filtermetric.MatchProperties{ + regexpMetricsFilterProperties = &filtermetric.MatchProperties{ Config: filterset.Config{ MatchType: filterset.Regexp, }, MetricNames: validFilters, } - tests := []struct { - name string - cfg *Config - inc *filtermetric.MatchProperties - exc *filtermetric.MatchProperties - inMN []string // input Metric names - outMN []string // output Metric names - }{ + standardTests = []metricNameTest{ { name: "includeFilter", inc: regexpMetricsFilterProperties, @@ -152,8 +147,15 @@ func TestFilterMetricProcessor(t *testing.T) { outMN: inMetricNames, }, } +) - for _, test := range tests { +func createFilterProcessorConfig(matchType filterset.MatchType, inc []string, exc []string) *Config { + return &Config{} + +} + +func TestFilterMetricProcessor(t *testing.T) { + for _, test := range standardTests { t.Run(test.name, func(t *testing.T) { // next stores the results of the filter metric processor next := &ptest.CachingMetricsConsumer{} @@ -193,3 +195,73 @@ func TestFilterMetricProcessor(t *testing.T) { }) } } + +func BenchmarkFilter_MetricNames(b *testing.B) { + // runs 1000 metrics through a filterprocessor with both include and exclude filters. + stressTest := metricNameTest{ + name: "includeAndExcludeFilter1000Metrics", + inc: regexpMetricsFilterProperties, + exc: &filtermetric.MatchProperties{ + Config: filterset.Config{ + MatchType: filterset.Strict, + }, + MetricNames: []string{ + "prefix_test_match", + "test_contains_match", + }, + }, + outMN: []string{ + "full_name_match", + "prefix/test/match", + // "prefix_test_match", excluded by exclude filter + "prefixprefix/test/match", + "test/match/suffix", + "test_match_suffix", + "test/match/suffixsuffix", + "test/contains/match", + // "test_contains_match", excluded by exclude filter + "full/name/match", + "full_name_match", + }, + } + + for len(stressTest.inMN) < 1000 { + stressTest.inMN = append(stressTest.inMN, inMetricNames...) + } + + benchmarkTests := append(standardTests, stressTest) + + for _, test := range benchmarkTests { + // next stores the results of the filter metric processor + next := &ptest.CachingMetricsConsumer{} + cfg := &Config{ + ProcessorSettings: configmodels.ProcessorSettings{ + TypeVal: typeStr, + NameVal: typeStr, + }, + Metrics: MetricFilters{ + Include: test.inc, + Exclude: test.exc, + }, + } + fmp, err := newFilterMetricProcessor(next, cfg) + assert.NotNil(b, fmp) + assert.Nil(b, err) + + md := consumerdata.MetricsData{ + Metrics: make([]*metricspb.Metric, len(test.inMN)), + } + + for idx, in := range test.inMN { + md.Metrics[idx] = &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: in, + }, + } + } + + b.Run(test.name, func(b *testing.B) { + assert.NoError(b, fmp.ConsumeMetricsData(context.Background(), md)) + }) + } +} From 6dbc17cce4ea53b0f9b957056716fc629c1982bd Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Wed, 3 Jun 2020 16:21:52 -0700 Subject: [PATCH 3/6] Address PR and fix build --- processor/filterprocessor/factory_test.go | 1 + .../filterprocessor/filter_processor_test.go | 17 ++++++----------- .../testdata/config_invalid.yaml | 2 +- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/processor/filterprocessor/factory_test.go b/processor/filterprocessor/factory_test.go index a0f11720d79..d4515e0c76d 100644 --- a/processor/filterprocessor/factory_test.go +++ b/processor/filterprocessor/factory_test.go @@ -65,6 +65,7 @@ func TestCreateProcessors(t *testing.T) { factory := &Factory{} factories.Processors[typeStr] = factory config, err := config.LoadConfigFile(t, path.Join(".", "testdata", test.configName), factories) + assert.Nil(t, err) for name, cfg := range config.Processors { t.Run(fmt.Sprintf("%s/%s", test.configName, name), func(t *testing.T) { diff --git a/processor/filterprocessor/filter_processor_test.go b/processor/filterprocessor/filter_processor_test.go index c4cd98f0245..805b2618684 100644 --- a/processor/filterprocessor/filter_processor_test.go +++ b/processor/filterprocessor/filter_processor_test.go @@ -24,14 +24,13 @@ import ( "go.opentelemetry.io/collector/config/configmodels" "go.opentelemetry.io/collector/consumer/consumerdata" + etest "go.opentelemetry.io/collector/exporter/exportertest" "go.opentelemetry.io/collector/internal/processor/filtermetric" "go.opentelemetry.io/collector/internal/processor/filterset" - ptest "go.opentelemetry.io/collector/processor/processortest" ) type metricNameTest struct { name string - cfg *Config inc *filtermetric.MatchProperties exc *filtermetric.MatchProperties inMN []string // input Metric names @@ -149,16 +148,11 @@ var ( } ) -func createFilterProcessorConfig(matchType filterset.MatchType, inc []string, exc []string) *Config { - return &Config{} - -} - func TestFilterMetricProcessor(t *testing.T) { for _, test := range standardTests { t.Run(test.name, func(t *testing.T) { // next stores the results of the filter metric processor - next := &ptest.CachingMetricsConsumer{} + next := &etest.SinkMetricsExporterOld{} cfg := &Config{ ProcessorSettings: configmodels.ProcessorSettings{ TypeVal: typeStr, @@ -188,8 +182,9 @@ func TestFilterMetricProcessor(t *testing.T) { cErr := fmp.ConsumeMetricsData(context.Background(), md) assert.Nil(t, cErr) - require.Equal(t, len(test.outMN), len(next.Data.Metrics)) - for idx, out := range next.Data.Metrics { + gotMetrics := next.AllMetrics()[0].Metrics + require.Equal(t, len(test.outMN), len(gotMetrics)) + for idx, out := range gotMetrics { assert.Equal(t, test.outMN[idx], out.MetricDescriptor.Name) } }) @@ -233,7 +228,7 @@ func BenchmarkFilter_MetricNames(b *testing.B) { for _, test := range benchmarkTests { // next stores the results of the filter metric processor - next := &ptest.CachingMetricsConsumer{} + next := &etest.SinkMetricsExporterOld{} cfg := &Config{ ProcessorSettings: configmodels.ProcessorSettings{ TypeVal: typeStr, diff --git a/processor/filterprocessor/testdata/config_invalid.yaml b/processor/filterprocessor/testdata/config_invalid.yaml index d9be2ba2c82..a2ed32c8c40 100644 --- a/processor/filterprocessor/testdata/config_invalid.yaml +++ b/processor/filterprocessor/testdata/config_invalid.yaml @@ -23,4 +23,4 @@ service: metrics: receivers: [examplereceiver] processors: [filter/include] - exporters: [exampleexporter] \ No newline at end of file + exporters: [exampleexporter] From dcd85ac12def26aba51e646b27a1380d681f5b99 Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Wed, 3 Jun 2020 16:40:06 -0700 Subject: [PATCH 4/6] Remove testing caching processor --- processor/processortest/caching_processor.go | 46 -------------------- 1 file changed, 46 deletions(-) delete mode 100644 processor/processortest/caching_processor.go diff --git a/processor/processortest/caching_processor.go b/processor/processortest/caching_processor.go deleted file mode 100644 index 8d41aaf63da..00000000000 --- a/processor/processortest/caching_processor.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2020, OpenTelemetry Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package processortest - -import ( - "context" - - "go.opentelemetry.io/collector/consumer/consumerdata" -) - -// CachingTraceConsumer is a dummy consumer that simply stores the trace data. -// When testing processors, it can be specified as the next consumer of a processor -// to inspect the outputs of the processor being tested. -type CachingTraceConsumerOld struct { - Data consumerdata.TraceData -} - -// ConsumeTraceData stores the trace data and returns nil. -func (ctc *CachingTraceConsumerOld) ConsumeTraceData(ctx context.Context, td consumerdata.TraceData) error { - ctc.Data = td - return nil -} - -// CachingMetricsConsumer is a dummy consumer that simply stores the metrics data. -// When testing processors, it can be specified as the next consumer of a processor -// to inspect the outputs of the processor being tested. -type CachingMetricsConsumer struct { - Data consumerdata.MetricsData -} - -// ConsumeMetricData stores the metrics data and returns nil. -func (cmc *CachingMetricsConsumer) ConsumeMetricsData(ctx context.Context, md consumerdata.MetricsData) error { - cmc.Data = md - return nil -} From 7933e7bc8e9b1866bdaed584a255243d7805750a Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Wed, 3 Jun 2020 18:20:44 -0700 Subject: [PATCH 5/6] Address PR & increase test coverage --- processor/filterprocessor/config_test.go | 11 ---------- processor/filterprocessor/factory_test.go | 7 +++++++ processor/filterprocessor/filter_processor.go | 20 +++++++++---------- .../filterprocessor/filter_processor_test.go | 7 +++++++ .../testdata/config_regexp.yaml | 2 +- .../testdata/config_strict.yaml | 10 +--------- 6 files changed, 25 insertions(+), 32 deletions(-) diff --git a/processor/filterprocessor/config_test.go b/processor/filterprocessor/config_test.go index 27deb965a88..4329b6ab3c7 100644 --- a/processor/filterprocessor/config_test.go +++ b/processor/filterprocessor/config_test.go @@ -111,17 +111,6 @@ func TestLoadingConfigStrict(t *testing.T) { }, }, }, - }, { - filterName: "filter/strictwithconfig", - expCfg: &Config{ - ProcessorSettings: configmodels.ProcessorSettings{ - NameVal: "filter/strictwithconfig", - TypeVal: typeStr, - }, - Metrics: MetricFilters{ - Include: testDataMetricProperties, - }, - }, }, } diff --git a/processor/filterprocessor/factory_test.go b/processor/filterprocessor/factory_test.go index d4515e0c76d..c1734750ab6 100644 --- a/processor/filterprocessor/factory_test.go +++ b/processor/filterprocessor/factory_test.go @@ -29,6 +29,13 @@ import ( "go.opentelemetry.io/collector/config/configmodels" ) +func TestType(t *testing.T) { + factory := Factory{} + pType := factory.Type() + + assert.Equal(t, pType, configmodels.Type("filter")) +} + func TestCreateDefaultConfig(t *testing.T) { factory := Factory{} cfg := factory.CreateDefaultConfig() diff --git a/processor/filterprocessor/filter_processor.go b/processor/filterprocessor/filter_processor.go index 89328d2e639..b05e35b7061 100644 --- a/processor/filterprocessor/filter_processor.go +++ b/processor/filterprocessor/filter_processor.go @@ -26,11 +26,10 @@ import ( ) type filterMetricProcessor struct { - cfg *Config - capabilities component.ProcessorCapabilities - next consumer.MetricsConsumerOld - include *filtermetric.Matcher - exclude *filtermetric.Matcher + cfg *Config + next consumer.MetricsConsumerOld + include *filtermetric.Matcher + exclude *filtermetric.Matcher } var _ component.MetricsProcessorOld = (*filterMetricProcessor)(nil) @@ -47,11 +46,10 @@ func newFilterMetricProcessor(next consumer.MetricsConsumerOld, cfg *Config) (*f } return &filterMetricProcessor{ - cfg: cfg, - capabilities: component.ProcessorCapabilities{MutatesConsumedData: false}, - next: next, - include: inc, - exclude: exc, + cfg: cfg, + next: next, + include: inc, + exclude: exc, }, nil } @@ -71,7 +69,7 @@ func createMatcher(mp *filtermetric.MatchProperties) (*filtermetric.Matcher, err // GetCapabilities returns the Capabilities assocciated with the resource processor. func (fmp *filterMetricProcessor) GetCapabilities() component.ProcessorCapabilities { - return fmp.capabilities + return component.ProcessorCapabilities{MutatesConsumedData: false} } // Start is invoked during service startup. diff --git a/processor/filterprocessor/filter_processor_test.go b/processor/filterprocessor/filter_processor_test.go index 805b2618684..fae6c93ecfa 100644 --- a/processor/filterprocessor/filter_processor_test.go +++ b/processor/filterprocessor/filter_processor_test.go @@ -167,6 +167,11 @@ func TestFilterMetricProcessor(t *testing.T) { assert.NotNil(t, fmp) assert.Nil(t, err) + caps := fmp.GetCapabilities() + assert.Equal(t, false, caps.MutatesConsumedData) + ctx := context.Background() + assert.NoError(t, fmp.Start(ctx, nil)) + md := consumerdata.MetricsData{ Metrics: make([]*metricspb.Metric, len(test.inMN)), } @@ -187,6 +192,8 @@ func TestFilterMetricProcessor(t *testing.T) { for idx, out := range gotMetrics { assert.Equal(t, test.outMN[idx], out.MetricDescriptor.Name) } + + assert.NoError(t, fmp.Shutdown(ctx)) }) } } diff --git a/processor/filterprocessor/testdata/config_regexp.yaml b/processor/filterprocessor/testdata/config_regexp.yaml index db13c3f4643..6ad5a90c1f7 100644 --- a/processor/filterprocessor/testdata/config_regexp.yaml +++ b/processor/filterprocessor/testdata/config_regexp.yaml @@ -76,4 +76,4 @@ service: metrics: receivers: [examplereceiver] processors: [filter] - exporters: [exampleexporter] \ No newline at end of file + exporters: [exampleexporter] diff --git a/processor/filterprocessor/testdata/config_strict.yaml b/processor/filterprocessor/testdata/config_strict.yaml index e307f61b168..5eddd3b6cdf 100644 --- a/processor/filterprocessor/testdata/config_strict.yaml +++ b/processor/filterprocessor/testdata/config_strict.yaml @@ -35,14 +35,6 @@ processors: match_type: strict metric_names: - hello_world - filter/strictwithconfig: - metrics: - include: - match_type: strict - strict: - metric_names: - - hello_world - - hello/world exporters: exampleexporter: @@ -56,4 +48,4 @@ service: metrics: receivers: [examplereceiver] processors: [filter/empty] - exporters: [exampleexporter] \ No newline at end of file + exporters: [exampleexporter] From c698c85d4abd6757c253946c4d3480a1af4509fe Mon Sep 17 00:00:00 2001 From: Annie Fu Date: Wed, 3 Jun 2020 18:30:00 -0700 Subject: [PATCH 6/6] Pre-allocate filtered metrics slice --- processor/filterprocessor/filter_processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processor/filterprocessor/filter_processor.go b/processor/filterprocessor/filter_processor.go index b05e35b7061..c2232201664 100644 --- a/processor/filterprocessor/filter_processor.go +++ b/processor/filterprocessor/filter_processor.go @@ -93,7 +93,7 @@ func (fmp *filterMetricProcessor) ConsumeMetricsData(ctx context.Context, md con // filterMetrics filters the given spans based off the filterMetricProcessor's filters. func (fmp *filterMetricProcessor) filterMetrics(metrics []*metricspb.Metric) []*metricspb.Metric { - keep := []*metricspb.Metric{} + keep := make([]*metricspb.Metric, 0, len(metrics)) for _, m := range metrics { if fmp.shouldKeepMetric(m) { keep = append(keep, m)