diff --git a/.github/workflows/test-build-deploy.yml b/.github/workflows/test-build-deploy.yml index 024ad840f9..30bde8fead 100644 --- a/.github/workflows/test-build-deploy.yml +++ b/.github/workflows/test-build-deploy.yml @@ -144,6 +144,7 @@ jobs: - integration_querier - integration_ruler - integration_query_fuzz + - integration_remote_write_v2 steps: - name: Upgrade golang uses: actions/setup-go@41dfa10bad2bb2ae585af6ee5bb4d7d973ad74ed # v5.1.0 diff --git a/.golangci.yml b/.golangci.yml index dbfe02e837..e5336badfd 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -49,3 +49,4 @@ run: - integration_querier - integration_ruler - integration_query_fuzz + - integration_remote_write_v2 diff --git a/CHANGELOG.md b/CHANGELOG.md index 1832fc50a1..d26fa856da 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ * [FEATURE] Ruler: Minimize chances of missed rule group evaluations that can occur due to OOM kills, bad underlying nodes, or due to an unhealthy ruler that appears in the ring as healthy. This feature is enabled via `-ruler.enable-ha-evaluation` flag. #6129 * [FEATURE] Store Gateway: Add an in-memory chunk cache. #6245 * [FEATURE] Chunk Cache: Support multi level cache and add metrics. #6249 +* [FEATURE] Distributor/Ingester: Support remote write 2.0. It includes proto, samples, and (native) histograms ingestion. #6292 * [ENHANCEMENT] S3 Bucket Client: Add a list objects version configs to configure list api object version. #6280 * [ENHANCEMENT] OpenStack Swift: Add application credential configs for Openstack swift object storage backend. #6255 * [ENHANCEMENT] Query Frontend: Add new query stats metrics `cortex_query_samples_scanned_total` and `cortex_query_peak_samples` to track scannedSamples and peakSample per user. #6228 diff --git a/integration/e2e/util.go b/integration/e2e/util.go index 141d043ab5..51a3cfcaa0 100644 --- a/integration/e2e/util.go +++ b/integration/e2e/util.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/tsdbutil" @@ -149,6 +150,40 @@ func GenerateSeries(name string, ts time.Time, additionalLabels ...prompb.Label) return } +func GenerateHistogramSeriesV2(name string, ts time.Time, i uint32, floatHistogram bool, additionalLabels ...prompb.Label) (symbols []string, series []writev2.TimeSeries) { + tsMillis := TimeToMilliseconds(ts) + + st := writev2.NewSymbolTable() + + lbs := labels.Labels{labels.Label{Name: "__name__", Value: name}} + for _, lbl := range additionalLabels { + lbs = append(lbs, labels.Label{Name: lbl.Name, Value: lbl.Value}) + } + + var ( + h *histogram.Histogram + fh *histogram.FloatHistogram + ph writev2.Histogram + ) + if floatHistogram { + fh = tsdbutil.GenerateTestFloatHistogram(int(i)) + ph = writev2.FromFloatHistogram(tsMillis, fh) + } else { + h = tsdbutil.GenerateTestHistogram(int(i)) + ph = writev2.FromIntHistogram(tsMillis, h) + } + + // Generate the series + series = append(series, writev2.TimeSeries{ + LabelsRefs: st.SymbolizeLabels(lbs, nil), + Histograms: []writev2.Histogram{ph}, + }) + + symbols = st.Symbols() + + return +} + func GenerateHistogramSeries(name string, ts time.Time, i uint32, floatHistogram bool, additionalLabels ...prompb.Label) (series []prompb.TimeSeries) { tsMillis := TimeToMilliseconds(ts) @@ -188,6 +223,47 @@ func GenerateHistogramSeries(name string, ts time.Time, i uint32, floatHistogram return } +func GenerateSeriesV2(name string, ts time.Time, additionalLabels ...prompb.Label) (symbols []string, series []writev2.TimeSeries, vector model.Vector) { + tsMillis := TimeToMilliseconds(ts) + value := rand.Float64() + + st := writev2.NewSymbolTable() + lbs := labels.Labels{{Name: labels.MetricName, Value: name}} + + for _, label := range additionalLabels { + lbs = append(lbs, labels.Label{ + Name: label.Name, + Value: label.Value, + }) + } + series = append(series, writev2.TimeSeries{ + // Generate the series + LabelsRefs: st.SymbolizeLabels(lbs, nil), + Samples: []writev2.Sample{ + {Value: value, Timestamp: tsMillis}, + }, + Metadata: writev2.Metadata{ + Type: writev2.Metadata_METRIC_TYPE_GAUGE, + }, + }) + symbols = st.Symbols() + + // Generate the expected vector when querying it + metric := model.Metric{} + metric[labels.MetricName] = model.LabelValue(name) + for _, lbl := range additionalLabels { + metric[model.LabelName(lbl.Name)] = model.LabelValue(lbl.Value) + } + + vector = append(vector, &model.Sample{ + Metric: metric, + Value: model.SampleValue(value), + Timestamp: model.Time(tsMillis), + }) + + return +} + func GenerateSeriesWithSamples( name string, startTime time.Time, diff --git a/integration/e2ecortex/client.go b/integration/e2ecortex/client.go index 2de73dbdd3..286656bfa5 100644 --- a/integration/e2ecortex/client.go +++ b/integration/e2ecortex/client.go @@ -23,13 +23,13 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/rulefmt" "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage/remote" - yaml "gopkg.in/yaml.v3" - "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp" + yaml "gopkg.in/yaml.v3" "github.com/cortexproject/cortex/pkg/ruler" "github.com/cortexproject/cortex/pkg/util/backoff" @@ -113,6 +113,39 @@ func NewPromQueryClient(address string) (*Client, error) { return c, nil } +// PushV2 the input timeseries to the remote endpoint +func (c *Client) PushV2(symbols []string, timeseries []writev2.TimeSeries) (*http.Response, error) { + // Create write request + data, err := proto.Marshal(&writev2.Request{Symbols: symbols, Timeseries: timeseries}) + if err != nil { + return nil, err + } + + // Create HTTP request + compressed := snappy.Encode(nil, data) + req, err := http.NewRequest("POST", fmt.Sprintf("http://%s/api/prom/push", c.distributorAddress), bytes.NewReader(compressed)) + if err != nil { + return nil, err + } + + req.Header.Add("Content-Encoding", "snappy") + req.Header.Set("Content-Type", "application/x-protobuf;proto=io.prometheus.write.v2.Request") + req.Header.Set("X-Prometheus-Remote-Write-Version", "2.0.0") + req.Header.Set("X-Scope-OrgID", c.orgID) + + ctx, cancel := context.WithTimeout(context.Background(), c.timeout) + defer cancel() + + // Execute HTTP request + res, err := c.httpClient.Do(req.WithContext(ctx)) + if err != nil { + return nil, err + } + + defer res.Body.Close() + return res, nil +} + // Push the input timeseries to the remote endpoint func (c *Client) Push(timeseries []prompb.TimeSeries) (*http.Response, error) { // Create write request @@ -336,6 +369,11 @@ func (c *Client) Query(query string, ts time.Time) (model.Value, error) { return value, err } +func (c *Client) Metadata(name, limit string) (map[string][]promv1.Metadata, error) { + metadata, err := c.querierClient.Metadata(context.Background(), name, limit) + return metadata, err +} + // QueryExemplars runs an exemplars query func (c *Client) QueryExemplars(query string, start, end time.Time) ([]promv1.ExemplarQueryResult, error) { ctx, cancel := context.WithTimeout(context.Background(), c.timeout) diff --git a/integration/remote_write_v2_test.go b/integration/remote_write_v2_test.go new file mode 100644 index 0000000000..775ccb98bb --- /dev/null +++ b/integration/remote_write_v2_test.go @@ -0,0 +1,211 @@ +//go:build integration_remote_write_v2 +// +build integration_remote_write_v2 + +package integration + +import ( + "math/rand" + "net/http" + "path" + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/cortexproject/cortex/integration/e2e" + e2edb "github.com/cortexproject/cortex/integration/e2e/db" + "github.com/cortexproject/cortex/integration/e2ecortex" + "github.com/cortexproject/cortex/pkg/storage/tsdb" +) + +func TestIngest(t *testing.T) { + const blockRangePeriod = 5 * time.Second + + s, err := e2e.NewScenario(networkName) + require.NoError(t, err) + defer s.Close() + + // Start dependencies. + consul := e2edb.NewConsulWithName("consul") + require.NoError(t, s.StartAndWaitReady(consul)) + + flags := mergeFlags( + AlertmanagerLocalFlags(), + map[string]string{ + "-store.engine": blocksStorageEngine, + "-blocks-storage.backend": "filesystem", + "-blocks-storage.tsdb.head-compaction-interval": "4m", + "-blocks-storage.bucket-store.sync-interval": "15m", + "-blocks-storage.bucket-store.index-cache.backend": tsdb.IndexCacheBackendInMemory, + "-blocks-storage.bucket-store.bucket-index.enabled": "true", + "-querier.query-store-for-labels-enabled": "true", + "-blocks-storage.tsdb.block-ranges-period": blockRangePeriod.String(), + "-blocks-storage.tsdb.ship-interval": "1s", + "-blocks-storage.tsdb.retention-period": ((blockRangePeriod * 2) - 1).String(), + "-blocks-storage.tsdb.enable-native-histograms": "true", + // Ingester. + "-ring.store": "consul", + "-consul.hostname": consul.NetworkHTTPEndpoint(), + // Distributor. + "-distributor.replication-factor": "1", + // Store-gateway. + "-store-gateway.sharding-enabled": "false", + // alert manager + "-alertmanager.web.external-url": "http://localhost/alertmanager", + }, + ) + + // make alert manager config dir + require.NoError(t, writeFileToSharedDir(s, "alertmanager_configs", []byte{})) + + path := path.Join(s.SharedDir(), "cortex-1") + + flags = mergeFlags(flags, map[string]string{"-blocks-storage.filesystem.dir": path}) + // Start Cortex replicas. + cortex := e2ecortex.NewSingleBinary("cortex", flags, "") + require.NoError(t, s.StartAndWaitReady(cortex)) + + // Wait until Cortex replicas have updated the ring state. + require.NoError(t, cortex.WaitSumMetrics(e2e.Equals(float64(512)), "cortex_ring_tokens_total")) + + c, err := e2ecortex.NewClient(cortex.HTTPEndpoint(), cortex.HTTPEndpoint(), "", "", "user-1") + require.NoError(t, err) + + now := time.Now() + + // series push + symbols1, series, expectedVector := e2e.GenerateSeriesV2("test_series", now, prompb.Label{Name: "job", Value: "test"}, prompb.Label{Name: "foo", Value: "bar"}) + res, err := c.PushV2(symbols1, series) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + testPushHeader(t, res.Header, "1", "0", "0") + + // sample + result, err := c.Query("test_series", now) + require.NoError(t, err) + assert.Equal(t, expectedVector, result.(model.Vector)) + + // metadata + metadata, err := c.Metadata("test_series", "") + require.NoError(t, err) + require.Equal(t, 1, len(metadata["test_series"])) + + // histogram + histogramIdx := rand.Uint32() + symbols2, histogramSeries := e2e.GenerateHistogramSeriesV2("test_histogram", now, histogramIdx, false, prompb.Label{Name: "job", Value: "test"}, prompb.Label{Name: "float", Value: "false"}) + res, err = c.PushV2(symbols2, histogramSeries) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + testPushHeader(t, res.Header, "1", "1", "0") + + symbols3, histogramFloatSeries := e2e.GenerateHistogramSeriesV2("test_histogram", now, histogramIdx, false, prompb.Label{Name: "job", Value: "test"}, prompb.Label{Name: "float", Value: "true"}) + res, err = c.PushV2(symbols3, histogramFloatSeries) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + testPushHeader(t, res.Header, "1", "1", "0") + + testHistogramTimestamp := now.Add(blockRangePeriod * 2) + expectedHistogram := tsdbutil.GenerateTestHistogram(int(histogramIdx)) + result, err = c.Query(`test_histogram`, testHistogramTimestamp) + require.NoError(t, err) + require.Equal(t, model.ValVector, result.Type()) + v := result.(model.Vector) + require.Equal(t, 2, v.Len()) + for _, s := range v { + require.NotNil(t, s.Histogram) + require.Equal(t, float64(expectedHistogram.Count), float64(s.Histogram.Count)) + require.Equal(t, float64(expectedHistogram.Sum), float64(s.Histogram.Sum)) + } +} + +func TestExemplar(t *testing.T) { + s, err := e2e.NewScenario(networkName) + require.NoError(t, err) + defer s.Close() + + // Start dependencies. + consul := e2edb.NewConsulWithName("consul") + require.NoError(t, s.StartAndWaitReady(consul)) + + flags := mergeFlags( + AlertmanagerLocalFlags(), + map[string]string{ + "-store.engine": blocksStorageEngine, + "-blocks-storage.backend": "filesystem", + "-blocks-storage.tsdb.head-compaction-interval": "4m", + "-blocks-storage.bucket-store.sync-interval": "15m", + "-blocks-storage.bucket-store.index-cache.backend": tsdb.IndexCacheBackendInMemory, + "-blocks-storage.bucket-store.bucket-index.enabled": "true", + "-querier.query-store-for-labels-enabled": "true", + "-blocks-storage.tsdb.ship-interval": "1s", + "-blocks-storage.tsdb.enable-native-histograms": "true", + // Ingester. + "-ring.store": "consul", + "-consul.hostname": consul.NetworkHTTPEndpoint(), + "-ingester.max-exemplars": "100", + // Distributor. + "-distributor.replication-factor": "1", + // Store-gateway. + "-store-gateway.sharding-enabled": "false", + // alert manager + "-alertmanager.web.external-url": "http://localhost/alertmanager", + }, + ) + + // make alert manager config dir + require.NoError(t, writeFileToSharedDir(s, "alertmanager_configs", []byte{})) + + path := path.Join(s.SharedDir(), "cortex-1") + + flags = mergeFlags(flags, map[string]string{"-blocks-storage.filesystem.dir": path}) + // Start Cortex replicas. + cortex := e2ecortex.NewSingleBinary("cortex", flags, "") + require.NoError(t, s.StartAndWaitReady(cortex)) + + // Wait until Cortex replicas have updated the ring state. + require.NoError(t, cortex.WaitSumMetrics(e2e.Equals(float64(512)), "cortex_ring_tokens_total")) + + c, err := e2ecortex.NewClient(cortex.HTTPEndpoint(), cortex.HTTPEndpoint(), "", "", "user-1") + require.NoError(t, err) + + now := time.Now() + tsMillis := e2e.TimeToMilliseconds(now) + + symbols := []string{"", "__name__", "test_metric", "b", "c", "baz", "qux", "d", "e", "foo", "bar", "f", "g", "h", "i", "Test gauge for test purposes", "Maybe op/sec who knows (:", "Test counter for test purposes"} + timeseries := []writev2.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, // Symbolized writeRequestFixture.Timeseries[0].Labels + Metadata: writev2.Metadata{ + Type: writev2.Metadata_METRIC_TYPE_COUNTER, // writeV2RequestSeries1Metadata.Type. + + HelpRef: 15, // Symbolized writeV2RequestSeries1Metadata.Help. + UnitRef: 16, // Symbolized writeV2RequestSeries1Metadata.Unit. + }, + Samples: []writev2.Sample{{Value: 1, Timestamp: tsMillis}}, + Exemplars: []writev2.Exemplar{{LabelsRefs: []uint32{11, 12}, Value: 1, Timestamp: tsMillis}}, + }, + } + + res, err := c.PushV2(symbols, timeseries) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + testPushHeader(t, res.Header, "1", "0", "1") + + start := time.Now().Add(-time.Minute) + end := now.Add(time.Minute) + + exemplars, err := c.QueryExemplars("test_metric", start, end) + require.NoError(t, err) + require.Equal(t, 1, len(exemplars)) +} + +func testPushHeader(t *testing.T, header http.Header, expectedSamples, expectedHistogram, expectedExemplars string) { + require.Equal(t, expectedSamples, header.Get("X-Prometheus-Remote-Write-Samples-Written")) + require.Equal(t, expectedHistogram, header.Get("X-Prometheus-Remote-Write-Histograms-Written")) + require.Equal(t, expectedExemplars, header.Get("X-Prometheus-Remote-Write-Exemplars-Written")) +} diff --git a/pkg/cortex/modules_test.go b/pkg/cortex/modules_test.go index 7316e07274..ae5fc64582 100644 --- a/pkg/cortex/modules_test.go +++ b/pkg/cortex/modules_test.go @@ -13,6 +13,7 @@ import ( "github.com/weaveworks/common/server" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" ) func changeTargetConfig(c *Config) { @@ -160,6 +161,10 @@ func TestCortex_InitRulerStorage(t *testing.T) { type myPusher struct{} +func (p *myPusher) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + return nil, nil +} + func (p *myPusher) Push(ctx context.Context, req *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { return nil, nil } diff --git a/pkg/cortexpbv2/codecv2.go b/pkg/cortexpbv2/codecv2.go new file mode 100644 index 0000000000..9fe7b434e0 --- /dev/null +++ b/pkg/cortexpbv2/codecv2.go @@ -0,0 +1,160 @@ +package cortexpbv2 + +import ( + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + + "github.com/cortexproject/cortex/pkg/cortexpb" +) + +// ToLabels return model labels.Labels from timeseries' remote labels. +func (t TimeSeries) ToLabels(b *labels.ScratchBuilder, symbols []string) labels.Labels { + return desymbolizeLabels(b, t.GetLabelsRefs(), symbols) +} + +// ToLabels return model labels.Labels from exemplar remote labels. +func (e Exemplar) ToLabels(b *labels.ScratchBuilder, symbols []string) labels.Labels { + return desymbolizeLabels(b, e.GetLabelsRefs(), symbols) +} + +func (m Metadata) ToV1Metadata(name string, symbols []string) *cortexpb.MetricMetadata { + typ := cortexpb.UNKNOWN + + switch m.Type { + case METRIC_TYPE_COUNTER: + typ = cortexpb.COUNTER + case METRIC_TYPE_GAUGE: + typ = cortexpb.GAUGE + case METRIC_TYPE_HISTOGRAM: + typ = cortexpb.HISTOGRAM + case METRIC_TYPE_GAUGEHISTOGRAM: + typ = cortexpb.GAUGEHISTOGRAM + case METRIC_TYPE_SUMMARY: + typ = cortexpb.SUMMARY + case METRIC_TYPE_INFO: + typ = cortexpb.INFO + case METRIC_TYPE_STATESET: + typ = cortexpb.STATESET + } + + return &cortexpb.MetricMetadata{ + Type: typ, + MetricFamilyName: name, + Unit: symbols[m.UnitRef], + Help: symbols[m.HelpRef], + } +} + +// desymbolizeLabels decodes label references, with given symbols to labels. +func desymbolizeLabels(b *labels.ScratchBuilder, labelRefs []uint32, symbols []string) labels.Labels { + b.Reset() + for i := 0; i < len(labelRefs); i += 2 { + b.Add(symbols[labelRefs[i]], symbols[labelRefs[i+1]]) + } + b.Sort() + return b.Labels() +} + +// IsFloatHistogram returns true if the histogram is float. +func (h Histogram) IsFloatHistogram() bool { + _, ok := h.GetCount().(*Histogram_CountFloat) + return ok +} + +// FloatHistogramProtoToFloatHistogram extracts a float Histogram from the provided proto message. +func FloatHistogramProtoToFloatHistogram(h Histogram) *histogram.FloatHistogram { + if !h.IsFloatHistogram() { + panic("FloatHistogramProtoToFloatHistogram called with an integer histogram") + } + + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountFloat(), + Count: h.GetCountFloat(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveCounts(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeCounts(), + CustomValues: h.GetCustomValues(), + } +} + +func HistogramProtoToHistogram(h Histogram) *histogram.Histogram { + if h.IsFloatHistogram() { + panic("HistogramProtoToHistogram called with a float histogram") + } + + return &histogram.Histogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountInt(), + Count: h.GetCountInt(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveDeltas(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeDeltas(), + CustomValues: h.GetCustomValues(), + } +} + +// HistogramToHistogramProto converts a (normal integer) Histogram to its protobuf message type. +// Changed from https://github.com/prometheus/prometheus/blob/0ab95536115adfe50af249d36d73674be694ca3f/storage/remote/codec.go#L709-L723 +func HistogramToHistogramProto(timestamp int64, h *histogram.Histogram) Histogram { + return Histogram{ + Count: &Histogram_CountInt{CountInt: h.Count}, + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeBuckets, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveBuckets, + ResetHint: Histogram_ResetHint(h.CounterResetHint), + Timestamp: timestamp, + CustomValues: h.CustomValues, + } +} + +// FloatHistogramToHistogramProto converts a float Histogram to a normal +// Histogram's protobuf message type. +// Changed from https://github.com/prometheus/prometheus/blob/0ab95536115adfe50af249d36d73674be694ca3f/storage/remote/codec.go#L725-L739 +func FloatHistogramToHistogramProto(timestamp int64, fh *histogram.FloatHistogram) Histogram { + return Histogram{ + Count: &Histogram_CountFloat{CountFloat: fh.Count}, + Sum: fh.Sum, + Schema: fh.Schema, + ZeroThreshold: fh.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount}, + NegativeSpans: spansToSpansProto(fh.NegativeSpans), + NegativeCounts: fh.NegativeBuckets, + PositiveSpans: spansToSpansProto(fh.PositiveSpans), + PositiveCounts: fh.PositiveBuckets, + ResetHint: Histogram_ResetHint(fh.CounterResetHint), + Timestamp: timestamp, + CustomValues: fh.CustomValues, + } +} + +func spansProtoToSpans(s []BucketSpan) []histogram.Span { + spans := make([]histogram.Span, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +func spansToSpansProto(s []histogram.Span) []BucketSpan { + spans := make([]BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} diff --git a/pkg/cortexpbv2/compatv2.go b/pkg/cortexpbv2/compatv2.go new file mode 100644 index 0000000000..13d42d000f --- /dev/null +++ b/pkg/cortexpbv2/compatv2.go @@ -0,0 +1,88 @@ +package cortexpbv2 + +import ( + "github.com/prometheus/prometheus/model/labels" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" + + "github.com/cortexproject/cortex/pkg/cortexpb" +) + +// ToWriteRequestV2 converts matched slices of Labels, Samples, and Histograms into a WriteRequest proto. +func ToWriteRequestV2(lbls []labels.Labels, samples []Sample, histograms []Histogram, metadata []Metadata, source WriteRequest_SourceEnum, help ...string) *WriteRequest { + st := writev2.NewSymbolTable() + labelRefs := make([][]uint32, 0, len(lbls)) + for _, lbl := range lbls { + labelRefs = append(labelRefs, st.SymbolizeLabels(lbl, nil)) + } + + for _, s := range help { + st.Symbolize(s) + } + + symbols := st.Symbols() + + req := &WriteRequest{ + Timeseries: PreallocTimeseriesV2SliceFromPool(), + Symbols: symbols, + Source: source, + } + + i := 0 + for i < len(samples) || i < len(histograms) || i < len(metadata) { + ts := TimeseriesV2FromPool() + ts.LabelsRefs = labelRefs[i] + if i < len(samples) { + ts.Samples = append(ts.Samples, samples[i]) + } + if i < len(histograms) { + ts.Histograms = append(ts.Histograms, histograms[i]) + } + if i < len(metadata) { + ts.Metadata = metadata[i] + } + i++ + req.Timeseries = append(req.Timeseries, PreallocTimeseriesV2{TimeSeries: ts}) + } + + return req +} + +func GetLabelRefsFromLabelAdapters(symbols []string, las []cortexpb.LabelAdapter) []uint32 { + var ret []uint32 + + symbolMap := map[string]uint32{} + for idx, symbol := range symbols { + symbolMap[symbol] = uint32(idx) + } + + for _, lb := range las { + if idx, ok := symbolMap[lb.Name]; ok { + ret = append(ret, idx) + } + if idx, ok := symbolMap[lb.Value]; ok { + ret = append(ret, idx) + } + } + + return ret +} + +func GetLabelsRefsFromLabels(symbols []string, lbs labels.Labels) []uint32 { + var ret []uint32 + + symbolMap := map[string]uint32{} + for idx, symbol := range symbols { + symbolMap[symbol] = uint32(idx) + } + + for _, lb := range lbs { + if idx, ok := symbolMap[lb.Name]; ok { + ret = append(ret, idx) + } + if idx, ok := symbolMap[lb.Value]; ok { + ret = append(ret, idx) + } + } + + return ret +} diff --git a/pkg/cortexpbv2/compatv2_test.go b/pkg/cortexpbv2/compatv2_test.go new file mode 100644 index 0000000000..a6d809e0a4 --- /dev/null +++ b/pkg/cortexpbv2/compatv2_test.go @@ -0,0 +1,71 @@ +package cortexpbv2 + +import ( + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + "github.com/cortexproject/cortex/pkg/cortexpb" +) + +func Test_GetLabelRefsFromLabelAdapters(t *testing.T) { + tests := []struct { + symbols []string + lbs []cortexpb.LabelAdapter + expectedSeriesRefs []uint32 + }{ + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux"}, + lbs: []cortexpb.LabelAdapter{{Name: "__name__", Value: "test_metric"}, {Name: "foo", Value: "bar"}}, + expectedSeriesRefs: []uint32{1, 2, 3, 4}, + }, + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux"}, + lbs: []cortexpb.LabelAdapter{{Name: "__name__", Value: "test_metric"}, {Name: "baz", Value: "qux"}}, + expectedSeriesRefs: []uint32{1, 2, 5, 6}, + }, + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux", "1"}, + lbs: []cortexpb.LabelAdapter{{Name: "__name__", Value: "test_metric"}, {Name: "baz", Value: "qux"}, {Name: "qux", Value: "1"}}, + expectedSeriesRefs: []uint32{1, 2, 5, 6, 6, 7}, + }, + } + + for _, test := range tests { + require.Equal(t, test.expectedSeriesRefs, GetLabelRefsFromLabelAdapters(test.symbols, test.lbs)) + } +} + +func Test_GetLabelsRefsFromLabels(t *testing.T) { + tests := []struct { + symbols []string + lbs labels.Labels + expectedSeriesRefs []uint32 + }{ + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux"}, + lbs: labels.Labels{labels.Label{Name: "__name__", Value: "test_metric"}, labels.Label{Name: "foo", Value: "bar"}}, + expectedSeriesRefs: []uint32{1, 2, 3, 4}, + }, + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux"}, + lbs: labels.Labels{labels.Label{Name: "__name__", Value: "test_metric"}, labels.Label{Name: "baz", Value: "qux"}}, + expectedSeriesRefs: []uint32{1, 2, 5, 6}, + }, + { + symbols: []string{"", "__name__", "test_metric", "foo", "bar", "baz", "qux", "1"}, + lbs: labels.Labels{labels.Label{Name: "__name__", Value: "test_metric"}, labels.Label{Name: "baz", Value: "qux"}, labels.Label{Name: "qux", Value: "1"}}, + expectedSeriesRefs: []uint32{1, 2, 5, 6, 6, 7}, + }, + { + symbols: []string{"", "a help for testmetric", "a help for testmetric2"}, + lbs: labels.Labels{}, + expectedSeriesRefs: nil, + }, + } + + for _, test := range tests { + require.Equal(t, test.expectedSeriesRefs, GetLabelsRefsFromLabels(test.symbols, test.lbs)) + } +} diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 8f6b97aa5d..5cd0c13e88 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -20,6 +20,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/storage" "github.com/weaveworks/common/httpgrpc" @@ -28,6 +29,7 @@ import ( "go.uber.org/atomic" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" "github.com/cortexproject/cortex/pkg/ha" "github.com/cortexproject/cortex/pkg/ingester" ingester_client "github.com/cortexproject/cortex/pkg/ingester/client" @@ -44,7 +46,8 @@ import ( ) var ( - emptyPreallocSeries = cortexpb.PreallocTimeseries{} + emptyPreallocSeriesV2 = cortexpbv2.PreallocTimeseriesV2{} + emptyPreallocSeries = cortexpb.PreallocTimeseries{} supportedShardingStrategies = []string{util.ShardingStrategyDefault, util.ShardingStrategyShuffle} @@ -513,6 +516,17 @@ func shardByAllLabels(userID string, labels []cortexpb.LabelAdapter) uint32 { return h } +// Remove the label labelname from a slice of LabelPairs if it exists. +func removeLabelV2(labelName string, labels *labels.Labels) { + for i := 0; i < len(*labels); i++ { + pair := (*labels)[i] + if pair.Name == labelName { + *labels = append((*labels)[:i], (*labels)[i+1:]...) + return + } + } +} + // Remove the label labelname from a slice of LabelPairs if it exists. func removeLabel(labelName string, labels *[]cortexpb.LabelAdapter) { for i := 0; i < len(*labels); i++ { @@ -617,6 +631,432 @@ func (d *Distributor) validateSeries(ts cortexpb.PreallocTimeseries, userID stri nil } +func (d *Distributor) prepareSeriesKeysV2(ctx context.Context, req *cortexpbv2.WriteRequest, userID string, limits *validation.Limits, b labels.ScratchBuilder, st *writev2.SymbolsTable, removeReplica bool) ([]uint32, []cortexpbv2.PreallocTimeseriesV2, int64, int64, int64, int64, error, error) { + pSpan, _ := opentracing.StartSpanFromContext(ctx, "prepareSeriesKeysV2") + defer pSpan.Finish() + // For each timeseries or samples, we compute a hash to distribute across ingesters; + // check each sample/metadata and discard if outside limits. + validatedTimeseries := make([]cortexpbv2.PreallocTimeseriesV2, 0, len(req.Timeseries)) + seriesKeys := make([]uint32, 0, len(req.Timeseries)) + validatedFloatSamples := 0 + validatedHistogramSamples := 0 + validatedExemplars := 0 + validatedMetadata := 0 + + var firstPartialErr error + + latestSampleTimestampMs := int64(0) + defer func() { + // Update this metric even in case of errors. + if latestSampleTimestampMs > 0 { + d.latestSeenSampleTimestampPerUser.WithLabelValues(userID).Set(float64(latestSampleTimestampMs) / 1000) + } + }() + + // For each timeseries, compute a hash to distribute across ingesters; + // check each sample and discard if outside limits. + skipLabelNameValidation := d.cfg.SkipLabelNameValidation || req.GetSkipLabelNameValidation() + for _, ts := range req.Timeseries { + // Use timestamp of latest sample in the series. If samples for series are not ordered, metric for user may be wrong. + if len(ts.Samples) > 0 { + latestSampleTimestampMs = max(latestSampleTimestampMs, ts.Samples[len(ts.Samples)-1].Timestamp) + } + if len(ts.Histograms) > 0 { + latestSampleTimestampMs = max(latestSampleTimestampMs, ts.Histograms[len(ts.Histograms)-1].Timestamp) + } + + lbs := ts.ToLabels(&b, req.Symbols) + + if mrc := limits.MetricRelabelConfigs; len(mrc) > 0 { + l, _ := relabel.Process(lbs, mrc...) + if len(l) == 0 { + // all labels are gone, samples will be discarded + d.validateMetrics.DiscardedSamples.WithLabelValues( + validation.DroppedByRelabelConfiguration, + userID, + ).Add(float64(len(ts.Samples) + len(ts.Histograms))) + + // all labels are gone, exemplars will be discarded + d.validateMetrics.DiscardedExemplars.WithLabelValues( + validation.DroppedByRelabelConfiguration, + userID, + ).Add(float64(len(ts.Exemplars))) + continue + } + lbs = l + } + + // If we found both the cluster and replica labels, we only want to include the cluster label when + // storing series in Cortex. If we kept the replica label we would end up with another series for the same + // series we're trying to dedupe when HA tracking moves over to a different replica. + if removeReplica { + removeLabelV2(limits.HAReplicaLabel, &lbs) + } + + for _, labelName := range limits.DropLabels { + removeLabelV2(labelName, &lbs) + } + if len(lbs) == 0 { + d.validateMetrics.DiscardedSamples.WithLabelValues( + validation.DroppedByUserConfigurationOverride, + userID, + ).Add(float64(len(ts.Samples) + len(ts.Histograms))) + + d.validateMetrics.DiscardedExemplars.WithLabelValues( + validation.DroppedByUserConfigurationOverride, + userID, + ).Add(float64(len(ts.Exemplars))) + continue + } + + // update label refs + ts.LabelsRefs = st.SymbolizeLabels(lbs, nil) + las := cortexpb.FromLabelsToLabelAdapters(lbs) + + // We rely on sorted labels in different places: + // 1) When computing token for labels, and sharding by all labels. Here different order of labels returns + // different tokens, which is bad. + // 2) In validation code, when checking for duplicate label names. As duplicate label names are rejected + // later in the validation phase, we ignore them here. + sortLabelsIfNeeded(las) + + // Generate the sharding token based on the series labels without the HA replica + // label and dropped labels (if any) + seriesKey, err := d.tokenForLabels(userID, las) + if err != nil { + return nil, nil, 0, 0, 0, 0, nil, err + } + + validatedSeries, validationErr := d.validateSeriesV2(ts, las, req.Symbols, userID, skipLabelNameValidation, limits, b, st) + + // Errors in validation are considered non-fatal, as one series in a request may contain + // invalid data but all the remaining series could be perfectly valid. + if validationErr != nil && firstPartialErr == nil { + // The series labels may be retained by validationErr but that's not a problem for this + // use case because we format it calling Error() and then we discard it. + firstPartialErr = httpgrpc.Errorf(http.StatusBadRequest, validationErr.Error()) + } + + if ts.Metadata.Type != cortexpbv2.METRIC_TYPE_UNSPECIFIED { + // since metadata is attached, count only metadata that is not METRIC_TYPE_UNSPECIFIED. + validatedMetadata++ + } + + // validateSeriesV2 would have returned an emptyPreallocSeriesV2 if there were no valid samples. + if validatedSeries == emptyPreallocSeriesV2 { + continue + } + + seriesKeys = append(seriesKeys, seriesKey) + validatedTimeseries = append(validatedTimeseries, validatedSeries) + validatedFloatSamples += len(ts.Samples) + validatedHistogramSamples += len(ts.Histograms) + validatedExemplars += len(ts.Exemplars) + } + + return seriesKeys, validatedTimeseries, int64(validatedMetadata), int64(validatedFloatSamples), int64(validatedHistogramSamples), int64(validatedExemplars), firstPartialErr, nil +} + +func (d *Distributor) doBatchV2(ctx context.Context, req *cortexpbv2.WriteRequest, subRing ring.ReadRing, keys []uint32, validatedTimeseries []cortexpbv2.PreallocTimeseriesV2, userID string, stats *WriteStats) error { + span, _ := opentracing.StartSpanFromContext(ctx, "doBatchV2") + defer span.Finish() + + // Use a background context to make sure all ingesters get samples even if we return early + localCtx, cancel := context.WithTimeout(context.Background(), d.cfg.RemoteTimeout) + localCtx = user.InjectOrgID(localCtx, userID) + if sp := opentracing.SpanFromContext(ctx); sp != nil { + localCtx = opentracing.ContextWithSpan(localCtx, sp) + } + // Get any HTTP headers that are supposed to be added to logs and add to localCtx for later use + if headerMap := util_log.HeaderMapFromContext(ctx); headerMap != nil { + localCtx = util_log.ContextWithHeaderMap(localCtx, headerMap) + } + // Get clientIP(s) from Context and add it to localCtx + source := util.GetSourceIPsFromOutgoingCtx(ctx) + localCtx = util.AddSourceIPsToOutgoingContext(localCtx, source) + + op := ring.WriteNoExtend + if d.cfg.ExtendWrites { + op = ring.Write + } + + return ring.DoBatch(ctx, op, subRing, keys, func(ingester ring.InstanceDesc, indexes []int) error { + timeseries := make([]cortexpbv2.PreallocTimeseriesV2, 0, len(indexes)) + + for _, i := range indexes { + timeseries = append(timeseries, validatedTimeseries[i]) + } + + return d.sendV2(localCtx, req.Symbols, ingester, timeseries, req.Source, stats) + }, func() { + cortexpbv2.ReuseSlice(req.Timeseries) + cancel() + }) +} + +func (d *Distributor) sendV2(ctx context.Context, symbols []string, ingester ring.InstanceDesc, timeseries []cortexpbv2.PreallocTimeseriesV2, source cortexpbv2.WriteRequest_SourceEnum, stats *WriteStats) error { + h, err := d.ingesterPool.GetClientFor(ingester.Addr) + if err != nil { + return err + } + + id, err := d.ingestersRing.GetInstanceIdByAddr(ingester.Addr) + if err != nil { + level.Warn(d.log).Log("msg", "instance not found in the ring", "addr", ingester.Addr, "err", err) + } + + c := h.(ingester_client.HealthAndIngesterClient) + + req := cortexpbv2.PreallocWriteRequestV2FromPool() + req.Symbols = symbols + req.Timeseries = timeseries + req.Source = source + + resp, err := c.PushPreAllocV2(ctx, req) + if err == nil { + cortexpbv2.ReuseWriteRequestV2(req) + } + + if len(timeseries) > 0 { + d.ingesterAppends.WithLabelValues(id, typeSamples).Inc() + if err != nil { + d.ingesterAppendFailures.WithLabelValues(id, typeSamples, getErrorStatus(err)).Inc() + } + + metadataAppend := false + for _, ts := range timeseries { + if ts.Metadata.Type != cortexpbv2.METRIC_TYPE_UNSPECIFIED { + metadataAppend = true + break + } + } + if metadataAppend { + d.ingesterAppends.WithLabelValues(id, typeMetadata).Inc() + if err != nil { + d.ingesterAppendFailures.WithLabelValues(id, typeMetadata, getErrorStatus(err)).Inc() + } + } + } + + if resp != nil { + // track stats + stats.SetSamples(resp.Samples) + stats.SetHistograms(resp.Histograms) + stats.SetExemplars(resp.Exemplars) + } + + return err +} + +// Validates a single series from a write request. Will remove labels if +// any are configured to be dropped for the user ID. +// Returns the validated series with it's labels/samples, and any error. +// The returned error may retain the series labels. +func (d *Distributor) validateSeriesV2(ts cortexpbv2.PreallocTimeseriesV2, seriesLabels []cortexpb.LabelAdapter, symbols []string, userID string, skipLabelNameValidation bool, limits *validation.Limits, b labels.ScratchBuilder, st *writev2.SymbolsTable) (cortexpbv2.PreallocTimeseriesV2, validation.ValidationError) { + d.labelsHistogram.Observe(float64(len(ts.LabelsRefs))) + + if err := validation.ValidateLabels(d.validateMetrics, limits, userID, seriesLabels, skipLabelNameValidation); err != nil { + return emptyPreallocSeriesV2, err + } + + var samples []cortexpbv2.Sample + if len(ts.Samples) > 0 { + // Only alloc when data present + samples = make([]cortexpbv2.Sample, 0, len(ts.Samples)) + for _, s := range ts.Samples { + if err := validation.ValidateSampleTimestamp(d.validateMetrics, limits, userID, seriesLabels, s.Timestamp); err != nil { + return emptyPreallocSeriesV2, err + } + samples = append(samples, s) + } + } + + var exemplars []cortexpbv2.Exemplar + if len(ts.Exemplars) > 0 { + // Only alloc when data present + exemplars = make([]cortexpbv2.Exemplar, 0, len(ts.Exemplars)) + for _, e := range ts.Exemplars { + if err := validation.ValidateExemplarV2(d.validateMetrics, symbols, userID, seriesLabels, &e, b, st); err != nil { + // An exemplar validation error prevents ingesting samples + // in the same series object. However, because the current Prometheus + // remote write implementation only populates one or the other, + // there never will be any. + return emptyPreallocSeriesV2, err + } + exemplars = append(exemplars, e) + } + } + + var histograms []cortexpbv2.Histogram + if len(ts.Histograms) > 0 { + // Only alloc when data present + histograms = make([]cortexpbv2.Histogram, 0, len(ts.Histograms)) + for i, h := range ts.Histograms { + if err := validation.ValidateSampleTimestamp(d.validateMetrics, limits, userID, seriesLabels, h.Timestamp); err != nil { + return emptyPreallocSeriesV2, err + } + convertedHistogram, err := validation.ValidateNativeHistogramV2(d.validateMetrics, limits, userID, seriesLabels, h) + if err != nil { + return emptyPreallocSeriesV2, err + } + ts.Histograms[i] = convertedHistogram + } + histograms = append(histograms, ts.Histograms...) + } + + // validate metadata + err := validation.ValidateMetadataV2(d.validateMetrics, limits, userID, symbols, &ts.Metadata, st) + if err != nil { + return emptyPreallocSeriesV2, err + } + + return cortexpbv2.PreallocTimeseriesV2{ + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: ts.LabelsRefs, + Samples: samples, + Exemplars: exemplars, + Histograms: histograms, + Metadata: ts.Metadata, + }, + }, nil +} + +func (d *Distributor) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + span, ctx := opentracing.StartSpanFromContext(ctx, "Distributor.PushV2") + defer span.Finish() + + // We will report *this* request in the error too. + inflight := d.inflightPushRequests.Inc() + defer d.inflightPushRequests.Dec() + + now := time.Now() + d.activeUsers.UpdateUserTimestamp(userID, now) + + numFloatSamples := 0 + numHistogramSamples := 0 + numExemplars := 0 + for _, ts := range req.Timeseries { + numFloatSamples += len(ts.Samples) + numHistogramSamples += len(ts.Histograms) + numExemplars += len(ts.Exemplars) + } + + // Count the total samples, exemplars in, prior to validation or deduplication, for comparison with other metrics. + d.incomingSamples.WithLabelValues(userID, sampleMetricTypeFloat).Add(float64(numFloatSamples)) + d.incomingSamples.WithLabelValues(userID, sampleMetricTypeHistogram).Add(float64(numHistogramSamples)) + d.incomingExemplars.WithLabelValues(userID).Add(float64(numExemplars)) + // Metadata is attached to each series. + d.incomingMetadata.WithLabelValues(userID).Add(float64(len(req.Timeseries))) + + if d.cfg.InstanceLimits.MaxInflightPushRequests > 0 && inflight > int64(d.cfg.InstanceLimits.MaxInflightPushRequests) { + return nil, errTooManyInflightPushRequests + } + + if d.cfg.InstanceLimits.MaxIngestionRate > 0 { + if rate := d.ingestionRate.Rate(); rate >= d.cfg.InstanceLimits.MaxIngestionRate { + return nil, errMaxSamplesPushRateLimitReached + } + } + + b := labels.NewScratchBuilder(0) + removeReplica := false + // Cache user limit with overrides so we spend less CPU doing locking. See issue #4904 + limits := d.limits.GetOverridesForUser(userID) + + if limits.AcceptHASamples && len(req.Timeseries) > 0 { + cluster, replica := findHALabels(limits.HAReplicaLabel, limits.HAClusterLabel, cortexpb.FromLabelsToLabelAdapters(req.Timeseries[0].ToLabels(&b, req.Symbols))) + removeReplica, err = d.checkSample(ctx, userID, cluster, replica, limits) + if err != nil { + // Ensure the request slice is reused if the series get deduped. + cortexpbv2.ReuseSlice(req.Timeseries) + + if errors.Is(err, ha.ReplicasNotMatchError{}) { + // These samples have been deduped. + d.dedupedSamples.WithLabelValues(userID, cluster).Add(float64(numFloatSamples + numHistogramSamples)) + return nil, httpgrpc.Errorf(http.StatusAccepted, err.Error()) + } + + if errors.Is(err, ha.TooManyReplicaGroupsError{}) { + d.validateMetrics.DiscardedSamples.WithLabelValues(validation.TooManyHAClusters, userID).Add(float64(numFloatSamples + numHistogramSamples)) + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + + return nil, err + } + // If there wasn't an error but removeReplica is false that means we didn't find both HA labels. + if !removeReplica { // False, Nil + d.nonHASamples.WithLabelValues(userID).Add(float64(numFloatSamples + numHistogramSamples)) + } + } + + st := writev2.NewSymbolTable() + seriesKeys, validatedTimeseries, validatedMetadatas, validatedFloatSamples, validatedHistogramSamples, validatedExemplars, firstPartialErr, err := d.prepareSeriesKeysV2(ctx, req, userID, limits, b, &st, removeReplica) + if err != nil { + return nil, err + } + req.Symbols = st.Symbols() + + d.receivedSamples.WithLabelValues(userID, sampleMetricTypeFloat).Add(float64(validatedFloatSamples)) + d.receivedSamples.WithLabelValues(userID, sampleMetricTypeHistogram).Add(float64(validatedHistogramSamples)) + d.receivedExemplars.WithLabelValues(userID).Add(float64(validatedExemplars)) + // Metadata is attached to each series + d.receivedMetadata.WithLabelValues(userID).Add(float64(validatedMetadatas)) + + if len(seriesKeys) == 0 { + // Ensure the request slice is reused if there's no series or metadata passing the validation. + cortexpbv2.ReuseSlice(req.Timeseries) + + return &cortexpbv2.WriteResponse{}, firstPartialErr + } + + totalSamples := validatedFloatSamples + validatedHistogramSamples + totalN := totalSamples + validatedExemplars + validatedMetadatas + if !d.ingestionRateLimiter.AllowN(now, userID, int(totalN)) { + // Ensure the request slice is reused if the request is rate limited. + cortexpbv2.ReuseSlice(req.Timeseries) + + d.validateMetrics.DiscardedSamples.WithLabelValues(validation.RateLimited, userID).Add(float64(totalSamples)) + d.validateMetrics.DiscardedExemplars.WithLabelValues(validation.RateLimited, userID).Add(float64(validatedExemplars)) + d.validateMetrics.DiscardedMetadata.WithLabelValues(validation.RateLimited, userID).Add(float64(validatedMetadatas)) + // Return a 429 here to tell the client it is going too fast. + // Client may discard the data or slow down and re-send. + // Prometheus v2.26 added a remote-write option 'retry_on_http_429'. + return nil, httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (%v) exceeded while adding %d samples and %d metadata", d.ingestionRateLimiter.Limit(now, userID), totalSamples, validatedMetadatas) + } + + // totalN included samples and metadata. Ingester follows this pattern when computing its ingestion rate. + d.ingestionRate.Add(totalN) + + subRing := d.ingestersRing + + // Obtain a subring if required. + if d.cfg.ShardingStrategy == util.ShardingStrategyShuffle { + subRing = d.ingestersRing.ShuffleShard(userID, limits.IngestionTenantShardSize) + } + + keys := seriesKeys + + s := WriteStats{} + + err = d.doBatchV2(ctx, req, subRing, keys, validatedTimeseries, userID, &s) + if err != nil { + return nil, err + } + + resp := &cortexpbv2.WriteResponse{ + Samples: s.LoadSamples(), + Histograms: s.LoadHistogram(), + Exemplars: s.LoadExemplars(), + } + + return resp, firstPartialErr +} + // Push implements client.IngesterServer func (d *Distributor) Push(ctx context.Context, req *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { userID, err := tenant.TenantID(ctx) diff --git a/pkg/distributor/distributor_prw2_test.go b/pkg/distributor/distributor_prw2_test.go new file mode 100644 index 0000000000..80f8d92f70 --- /dev/null +++ b/pkg/distributor/distributor_prw2_test.go @@ -0,0 +1,2525 @@ +package distributor + +import ( + "context" + "fmt" + "math" + "net/http" + "strconv" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/weaveworks/common/httpgrpc" + "github.com/weaveworks/common/user" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" + "github.com/cortexproject/cortex/pkg/ingester" + "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/cortexproject/cortex/pkg/ring" + ring_client "github.com/cortexproject/cortex/pkg/ring/client" + "github.com/cortexproject/cortex/pkg/ring/kv" + "github.com/cortexproject/cortex/pkg/ring/kv/consul" + "github.com/cortexproject/cortex/pkg/tenant" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/chunkcompat" + "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/limiter" + "github.com/cortexproject/cortex/pkg/util/services" + "github.com/cortexproject/cortex/pkg/util/test" + "github.com/cortexproject/cortex/pkg/util/validation" +) + +var ( + emptyResponseV2 = &cortexpbv2.WriteResponse{} +) + +func TestDistributorPRW2_Push_LabelRemoval_RemovingNameLabelWillError(t *testing.T) { + t.Parallel() + ctx := user.InjectOrgID(context.Background(), "user") + type testcase struct { + inputSeries labels.Labels + expectedSeries labels.Labels + removeReplica bool + removeLabels []string + } + + tc := testcase{ + removeReplica: true, + removeLabels: []string{"__name__"}, + inputSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "cluster", Value: "one"}, + {Name: "__replica__", Value: "two"}, + }, + expectedSeries: labels.Labels{}, + } + + var err error + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.DropLabels = tc.removeLabels + limits.AcceptHASamples = tc.removeReplica + + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shardByAllLabels: true, + limits: &limits, + }) + + // Push the series to the distributor + req := mockWriteRequestV2([]labels.Labels{tc.inputSeries}, 1, 1, false) + _, err = ds[0].PushV2(ctx, req) + require.Error(t, err) + assert.Equal(t, "rpc error: code = Code(400) desc = sample missing metric name", err.Error()) +} + +func TestDistributorPRW2_Push_LabelRemoval(t *testing.T) { + t.Parallel() + ctx := user.InjectOrgID(context.Background(), "user") + + type testcase struct { + inputSeries labels.Labels + expectedSeries labels.Labels + removeReplica bool + removeLabels []string + exemplars []cortexpbv2.Exemplar + } + + cases := []testcase{ + // Remove both cluster and replica label. + { + removeReplica: true, + removeLabels: []string{"cluster"}, + inputSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "cluster", Value: "one"}, + {Name: "__replica__", Value: "two"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + }, + }, + // Remove multiple labels and replica. + { + removeReplica: true, + removeLabels: []string{"foo", "some"}, + inputSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "cluster", Value: "one"}, + {Name: "__replica__", Value: "two"}, + {Name: "foo", Value: "bar"}, + {Name: "some", Value: "thing"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "cluster", Value: "one"}, + }, + }, + // Don't remove any labels. + { + removeReplica: false, + inputSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "__replica__", Value: "two"}, + {Name: "cluster", Value: "one"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "some_metric"}, + {Name: "__replica__", Value: "two"}, + {Name: "cluster", Value: "one"}, + }, + }, + // No labels left. + { + removeReplica: true, + removeLabels: []string{"cluster"}, + inputSeries: labels.Labels{ + {Name: "cluster", Value: "one"}, + {Name: "__replica__", Value: "two"}, + }, + expectedSeries: labels.Labels{}, + exemplars: []cortexpbv2.Exemplar{ + {LabelsRefs: []uint32{1, 2}, Value: 1, Timestamp: 0}, + {LabelsRefs: []uint32{1, 2}, Value: 1, Timestamp: 0}, + }, + }, + } + + for _, tc := range cases { + for _, histogram := range []bool{true, false} { + var err error + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.DropLabels = tc.removeLabels + limits.AcceptHASamples = tc.removeReplica + + expectedDiscardedSamples := 0 + expectedDiscardedExemplars := 0 + if tc.expectedSeries.Len() == 0 { + expectedDiscardedSamples = 1 + expectedDiscardedExemplars = len(tc.exemplars) + // Allow series with no labels to ingest + limits.EnforceMetricName = false + } + + ds, ingesters, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shardByAllLabels: true, + limits: &limits, + }) + + // Push the series to the distributor + req := mockWriteRequestV2([]labels.Labels{tc.inputSeries}, 1, 1, histogram) + req.Timeseries[0].Exemplars = tc.exemplars + _, err = ds[0].PushV2(ctx, req) + require.NoError(t, err) + + actualDiscardedSamples := testutil.ToFloat64(ds[0].validateMetrics.DiscardedSamples.WithLabelValues(validation.DroppedByUserConfigurationOverride, "user")) + actualDiscardedExemplars := testutil.ToFloat64(ds[0].validateMetrics.DiscardedExemplars.WithLabelValues(validation.DroppedByUserConfigurationOverride, "user")) + require.Equal(t, float64(expectedDiscardedSamples), actualDiscardedSamples) + require.Equal(t, float64(expectedDiscardedExemplars), actualDiscardedExemplars) + + // Since each test pushes only 1 series, we do expect the ingester + // to have received exactly 1 series + for i := range ingesters { + timeseries := ingesters[i].series() + expectedSeries := 1 + if tc.expectedSeries.Len() == 0 { + expectedSeries = 0 + } + assert.Equal(t, expectedSeries, len(timeseries)) + for _, v := range timeseries { + assert.Equal(t, tc.expectedSeries, cortexpb.FromLabelAdaptersToLabels(v.Labels)) + } + } + } + } +} + +func TestDistributorPRW2_PushHAInstances(t *testing.T) { + t.Parallel() + ctx := user.InjectOrgID(context.Background(), "user") + + for i, tc := range []struct { + enableTracker bool + acceptedReplica string + testReplica string + cluster string + samples int + expectedResponse *cortexpbv2.WriteResponse + expectedCode int32 + }{ + { + enableTracker: true, + acceptedReplica: "instance0", + testReplica: "instance0", + cluster: "cluster0", + samples: 5, + expectedResponse: emptyResponseV2, + }, + // The 202 indicates that we didn't accept this sample. + { + enableTracker: true, + acceptedReplica: "instance2", + testReplica: "instance0", + cluster: "cluster0", + samples: 5, + expectedCode: 202, + }, + // If the HA tracker is disabled we should still accept samples that have both labels. + { + enableTracker: false, + acceptedReplica: "instance0", + testReplica: "instance0", + cluster: "cluster0", + samples: 5, + expectedResponse: emptyResponseV2, + }, + // Using very long replica label value results in validation error. + { + enableTracker: true, + acceptedReplica: "instance0", + testReplica: "instance1234567890123456789012345678901234567890", + cluster: "cluster0", + samples: 5, + expectedResponse: emptyResponseV2, + expectedCode: 400, + }, + } { + for _, shardByAllLabels := range []bool{true, false} { + tc := tc + shardByAllLabels := shardByAllLabels + for _, enableHistogram := range []bool{true, false} { + enableHistogram := enableHistogram + t.Run(fmt.Sprintf("[%d](shardByAllLabels=%v, histogram=%v)", i, shardByAllLabels, enableHistogram), func(t *testing.T) { + t.Parallel() + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.AcceptHASamples = true + limits.MaxLabelValueLength = 15 + + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + shardByAllLabels: shardByAllLabels, + limits: &limits, + enableTracker: tc.enableTracker, + }) + + d := ds[0] + + userID, err := tenant.TenantID(ctx) + assert.NoError(t, err) + err = d.HATracker.CheckReplica(ctx, userID, tc.cluster, tc.acceptedReplica, time.Now()) + assert.NoError(t, err) + + request := makeWriteRequestHAV2(tc.samples, tc.testReplica, tc.cluster, enableHistogram) + response, err := d.PushV2(ctx, request) + assert.Equal(t, tc.expectedResponse, response) + + httpResp, ok := httpgrpc.HTTPResponseFromError(err) + if ok { + assert.Equal(t, tc.expectedCode, httpResp.Code) + } else if tc.expectedCode != 0 { + assert.Fail(t, "expected HTTP status code", tc.expectedCode) + } + }) + } + } + } +} + +func BenchmarkDistributorPRW2_Push(b *testing.B) { + const ( + numSeriesPerRequest = 1000 + ) + ctx := user.InjectOrgID(context.Background(), "user") + + tests := map[string]struct { + prepareConfig func(limits *validation.Limits) + prepareSeries func() ([]labels.Labels, []cortexpbv2.Sample) + expectedErr string + }{ + "all samples successfully pushed": { + prepareConfig: func(limits *validation.Limits) {}, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "", + }, + "ingestion rate limit reached": { + prepareConfig: func(limits *validation.Limits) { + limits.IngestionRate = 1 + limits.IngestionBurstSize = 1 + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "ingestion rate limit", + }, + "too many labels limit reached": { + prepareConfig: func(limits *validation.Limits) { + limits.MaxLabelNamesPerSeries = 30 + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 1; i < 31; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "series has too many labels", + }, + "max label name length limit reached": { + prepareConfig: func(limits *validation.Limits) { + limits.MaxLabelNameLength = 1024 + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + // Add a label with a very long name. + lbls.Set(fmt.Sprintf("xxx_%0.2000d", 1), "xxx") + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "label name too long", + }, + "max label value length limit reached": { + prepareConfig: func(limits *validation.Limits) { + limits.MaxLabelValueLength = 1024 + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + // Add a label with a very long value. + lbls.Set("xxx", fmt.Sprintf("xxx_%0.2000d", 1)) + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "label value too long", + }, + "max label size bytes per series limit reached": { + prepareConfig: func(limits *validation.Limits) { + limits.MaxLabelsSizeBytes = 1024 + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + // Add a label with a very long value. + lbls.Set("xxx", fmt.Sprintf("xxx_%0.2000d", 1)) + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "labels size bytes exceeded", + }, + "timestamp too old": { + prepareConfig: func(limits *validation.Limits) { + limits.RejectOldSamples = true + limits.RejectOldSamplesMaxAge = model.Duration(time.Hour) + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().Add(-2*time.Hour).UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "timestamp too old", + }, + "timestamp too new": { + prepareConfig: func(limits *validation.Limits) { + limits.CreationGracePeriod = model.Duration(time.Minute) + }, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: "foo"}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().Add(time.Hour).UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + expectedErr: "timestamp too new", + }, + } + + tg := ring.NewRandomTokenGenerator() + + for testName, testData := range tests { + b.Run(testName, func(b *testing.B) { + + // Create an in-memory KV store for the ring with 1 ingester registered. + kvStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + b.Cleanup(func() { assert.NoError(b, closer.Close()) }) + + err := kvStore.CAS(context.Background(), ingester.RingKey, + func(_ interface{}) (interface{}, bool, error) { + d := &ring.Desc{} + d.AddIngester("ingester-1", "127.0.0.1", "", tg.GenerateTokens(d, "ingester-1", "", 128, true), ring.ACTIVE, time.Now()) + return d, true, nil + }, + ) + require.NoError(b, err) + + ingestersRing, err := ring.New(ring.Config{ + KVStore: kv.Config{Mock: kvStore}, + HeartbeatTimeout: 60 * time.Minute, + ReplicationFactor: 1, + }, ingester.RingKey, ingester.RingKey, nil, nil) + require.NoError(b, err) + require.NoError(b, services.StartAndAwaitRunning(context.Background(), ingestersRing)) + b.Cleanup(func() { + require.NoError(b, services.StopAndAwaitTerminated(context.Background(), ingestersRing)) + }) + + test.Poll(b, time.Second, 1, func() interface{} { + return ingestersRing.InstancesCount() + }) + + // Prepare the distributor configuration. + var distributorCfg Config + var clientConfig client.Config + limits := validation.Limits{} + flagext.DefaultValues(&distributorCfg, &clientConfig, &limits) + + limits.IngestionRate = 10000000 // Unlimited. + testData.prepareConfig(&limits) + + distributorCfg.ShardByAllLabels = true + distributorCfg.IngesterClientFactory = func(addr string) (ring_client.PoolClient, error) { + return &noopIngester{}, nil + } + + overrides, err := validation.NewOverrides(limits, nil) + require.NoError(b, err) + + // Start the distributor. + distributor, err := New(distributorCfg, clientConfig, overrides, ingestersRing, true, prometheus.NewRegistry(), log.NewNopLogger()) + require.NoError(b, err) + require.NoError(b, services.StartAndAwaitRunning(context.Background(), distributor)) + + b.Cleanup(func() { + require.NoError(b, services.StopAndAwaitTerminated(context.Background(), distributor)) + }) + + // Prepare the series to remote write before starting the benchmark. + metrics, samples := testData.prepareSeries() + + // Run the benchmark. + b.ReportAllocs() + b.ResetTimer() + + for n := 0; n < b.N; n++ { + _, err := distributor.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) + if testData.expectedErr == "" && err != nil { + b.Fatalf("no error expected but got %v", err) + } + if testData.expectedErr != "" && (err == nil || !strings.Contains(err.Error(), testData.expectedErr)) { + b.Fatalf("expected %v error but got %v", testData.expectedErr, err) + } + } + }) + } +} + +func TestDistributorPRW2_Push(t *testing.T) { + t.Parallel() + // Metrics to assert on. + lastSeenTimestamp := "cortex_distributor_latest_seen_sample_timestamp_seconds" + distributorAppend := "cortex_distributor_ingester_appends_total" + distributorAppendFailure := "cortex_distributor_ingester_append_failures_total" + distributorReceivedSamples := "cortex_distributor_received_samples_total" + ctx := user.InjectOrgID(context.Background(), "userDistributorPush") + + type samplesIn struct { + num int + startTimestampMs int64 + } + for name, tc := range map[string]struct { + metricNames []string + numIngesters int + happyIngesters int + samples samplesIn + histogramSamples bool + metadata int + expectedResponse *cortexpbv2.WriteResponse + expectedError error + expectedMetrics string + ingesterError error + }{ + "A push of no samples shouldn't block or return error, even if ingesters are sad": { + numIngesters: 3, + happyIngesters: 0, + expectedResponse: emptyResponseV2, + }, + "A push to 3 happy ingesters should succeed": { + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 5, startTimestampMs: 123456789000}, + metadata: 5, + expectedResponse: emptyResponseV2, + metricNames: []string{lastSeenTimestamp}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.004 + `, + }, + "A push to 2 happy ingesters should succeed": { + numIngesters: 3, + happyIngesters: 2, + samples: samplesIn{num: 5, startTimestampMs: 123456789000}, + metadata: 5, + expectedResponse: emptyResponseV2, + metricNames: []string{lastSeenTimestamp}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.004 + `, + }, + "A push to 1 happy ingesters should fail": { + numIngesters: 3, + happyIngesters: 1, + samples: samplesIn{num: 10, startTimestampMs: 123456789000}, + expectedError: errFail, + metricNames: []string{lastSeenTimestamp}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.009 + `, + }, + "A push to 0 happy ingesters should fail": { + numIngesters: 3, + happyIngesters: 0, + samples: samplesIn{num: 10, startTimestampMs: 123456789000}, + expectedError: errFail, + metricNames: []string{lastSeenTimestamp}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.009 + `, + }, + "A push exceeding burst size should fail": { + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 25, startTimestampMs: 123456789000}, + metadata: 5, + expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (20) exceeded while adding 25 samples and 5 metadata"), + metricNames: []string{lastSeenTimestamp}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.024 + `, + }, + "A push to ingesters should report the correct metrics with no metadata": { + numIngesters: 3, + happyIngesters: 2, + samples: samplesIn{num: 1, startTimestampMs: 123456789000}, + metadata: 0, + metricNames: []string{distributorAppend, distributorAppendFailure}, + expectedResponse: emptyResponseV2, + expectedMetrics: ` + # HELP cortex_distributor_ingester_append_failures_total The total number of failed batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_append_failures_total counter + cortex_distributor_ingester_append_failures_total{ingester="ingester-2",status="5xx",type="samples"} 1 + # HELP cortex_distributor_ingester_appends_total The total number of batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_appends_total counter + cortex_distributor_ingester_appends_total{ingester="ingester-0",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-1",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-2",type="samples"} 1 + `, + }, + "A push to ingesters should report samples and metadata metrics with no samples": { + numIngesters: 3, + happyIngesters: 2, + samples: samplesIn{num: 0, startTimestampMs: 123456789000}, + metadata: 1, + metricNames: []string{distributorAppend, distributorAppendFailure}, + expectedResponse: emptyResponseV2, + ingesterError: httpgrpc.Errorf(http.StatusInternalServerError, "Fail"), + expectedMetrics: ` + # HELP cortex_distributor_ingester_append_failures_total The total number of failed batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_append_failures_total counter + cortex_distributor_ingester_append_failures_total{ingester="ingester-2",status="5xx",type="metadata"} 1 + cortex_distributor_ingester_append_failures_total{ingester="ingester-2",status="5xx",type="samples"} 1 + # HELP cortex_distributor_ingester_appends_total The total number of batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_appends_total counter + cortex_distributor_ingester_appends_total{ingester="ingester-0",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-1",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-2",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-0",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-1",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-2",type="samples"} 1 + `, + }, + "A push to overloaded ingesters should report the correct metrics": { + numIngesters: 3, + happyIngesters: 2, + samples: samplesIn{num: 0, startTimestampMs: 123456789000}, + metadata: 1, + metricNames: []string{distributorAppend, distributorAppendFailure}, + expectedResponse: emptyResponseV2, + ingesterError: httpgrpc.Errorf(http.StatusTooManyRequests, "Fail"), + expectedMetrics: ` + # HELP cortex_distributor_ingester_appends_total The total number of batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_appends_total counter + cortex_distributor_ingester_appends_total{ingester="ingester-0",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-1",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-2",type="metadata"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-0",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-1",type="samples"} 1 + cortex_distributor_ingester_appends_total{ingester="ingester-2",type="samples"} 1 + # HELP cortex_distributor_ingester_append_failures_total The total number of failed batch appends sent to ingesters. + # TYPE cortex_distributor_ingester_append_failures_total counter + cortex_distributor_ingester_append_failures_total{ingester="ingester-2",status="4xx",type="metadata"} 1 + cortex_distributor_ingester_append_failures_total{ingester="ingester-2",status="4xx",type="samples"} 1 + `, + }, + "A push to 3 happy ingesters should succeed, histograms": { + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 5, startTimestampMs: 123456789000}, + histogramSamples: true, + metadata: 5, + expectedResponse: emptyResponseV2, + metricNames: []string{lastSeenTimestamp, distributorReceivedSamples}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.004 + # HELP cortex_distributor_received_samples_total The total number of received samples, excluding rejected and deduped samples. + # TYPE cortex_distributor_received_samples_total counter + cortex_distributor_received_samples_total{type="float",user="userDistributorPush"} 0 + cortex_distributor_received_samples_total{type="histogram",user="userDistributorPush"} 5 + `, + }, + "A push to 2 happy ingesters should succeed, histograms": { + numIngesters: 3, + happyIngesters: 2, + samples: samplesIn{num: 5, startTimestampMs: 123456789000}, + histogramSamples: true, + metadata: 5, + expectedResponse: emptyResponseV2, + metricNames: []string{lastSeenTimestamp, distributorReceivedSamples}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.004 + # HELP cortex_distributor_received_samples_total The total number of received samples, excluding rejected and deduped samples. + # TYPE cortex_distributor_received_samples_total counter + cortex_distributor_received_samples_total{type="float",user="userDistributorPush"} 0 + cortex_distributor_received_samples_total{type="histogram",user="userDistributorPush"} 5 + `, + }, + "A push to 1 happy ingesters should fail, histograms": { + numIngesters: 3, + happyIngesters: 1, + samples: samplesIn{num: 10, startTimestampMs: 123456789000}, + histogramSamples: true, + expectedError: errFail, + metricNames: []string{lastSeenTimestamp, distributorReceivedSamples}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.009 + # HELP cortex_distributor_received_samples_total The total number of received samples, excluding rejected and deduped samples. + # TYPE cortex_distributor_received_samples_total counter + cortex_distributor_received_samples_total{type="float",user="userDistributorPush"} 0 + cortex_distributor_received_samples_total{type="histogram",user="userDistributorPush"} 10 + `, + }, + "A push exceeding burst size should fail, histograms": { + numIngesters: 3, + happyIngesters: 3, + samples: samplesIn{num: 25, startTimestampMs: 123456789000}, + histogramSamples: true, + metadata: 5, + expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (20) exceeded while adding 25 samples and 5 metadata"), + metricNames: []string{lastSeenTimestamp, distributorReceivedSamples}, + expectedMetrics: ` + # HELP cortex_distributor_latest_seen_sample_timestamp_seconds Unix timestamp of latest received sample per user. + # TYPE cortex_distributor_latest_seen_sample_timestamp_seconds gauge + cortex_distributor_latest_seen_sample_timestamp_seconds{user="userDistributorPush"} 123456789.024 + # HELP cortex_distributor_received_samples_total The total number of received samples, excluding rejected and deduped samples. + # TYPE cortex_distributor_received_samples_total counter + cortex_distributor_received_samples_total{type="float",user="userDistributorPush"} 0 + cortex_distributor_received_samples_total{type="histogram",user="userDistributorPush"} 25 + `, + }, + } { + for _, shardByAllLabels := range []bool{true, false} { + tc := tc + name := name + shardByAllLabels := shardByAllLabels + t.Run(fmt.Sprintf("[%s](shardByAllLabels=%v)", name, shardByAllLabels), func(t *testing.T) { + t.Parallel() + limits := &validation.Limits{} + flagext.DefaultValues(limits) + limits.IngestionRate = 20 + limits.IngestionBurstSize = 20 + + ds, _, regs, _ := prepare(t, prepConfig{ + numIngesters: tc.numIngesters, + happyIngesters: tc.happyIngesters, + numDistributors: 1, + shardByAllLabels: shardByAllLabels, + limits: limits, + errFail: tc.ingesterError, + }) + + var request *cortexpbv2.WriteRequest + if !tc.histogramSamples { + request = makeWriteRequestV2WithSamples(tc.samples.startTimestampMs, tc.samples.num, tc.metadata) + } else { + request = makeWriteRequestV2WithHistogram(tc.samples.startTimestampMs, tc.samples.num, tc.metadata) + } + + response, err := ds[0].PushV2(ctx, request) + assert.Equal(t, tc.expectedResponse, response) + assert.Equal(t, status.Code(tc.expectedError), status.Code(err)) + + // Check tracked Prometheus metrics. Since the Push() response is sent as soon as the quorum + // is reached, when we reach this point the 3rd ingester may not have received series/metadata + // yet. To avoid flaky test we retry metrics assertion until we hit the desired state (no error) + // within a reasonable timeout. + if tc.expectedMetrics != "" { + test.Poll(t, time.Second, nil, func() interface{} { + return testutil.GatherAndCompare(regs[0], strings.NewReader(tc.expectedMetrics), tc.metricNames...) + }) + } + }) + } + } +} + +func TestDistributorPRW2_PushIngestionRateLimiter(t *testing.T) { + t.Parallel() + type testPush struct { + samples int + metadata int + expectedError error + } + + ctx := user.InjectOrgID(context.Background(), "user") + tests := map[string]struct { + distributors int + ingestionRateStrategy string + ingestionRate float64 + ingestionBurstSize int + pushes []testPush + }{ + "local strategy: limit should be set to each distributor": { + distributors: 2, + ingestionRateStrategy: validation.LocalIngestionRateStrategy, + ingestionRate: 10, + ingestionBurstSize: 10, + pushes: []testPush{ + {samples: 4, expectedError: nil}, + {metadata: 1, expectedError: nil}, + {samples: 6, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (10) exceeded while adding 6 samples and 0 metadata")}, + {samples: 4, metadata: 1, expectedError: nil}, + {samples: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (10) exceeded while adding 1 samples and 0 metadata")}, + {metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (10) exceeded while adding 0 samples and 1 metadata")}, + }, + }, + "global strategy: limit should be evenly shared across distributors": { + distributors: 2, + ingestionRateStrategy: validation.GlobalIngestionRateStrategy, + ingestionRate: 10, + ingestionBurstSize: 5, + pushes: []testPush{ + {samples: 2, expectedError: nil}, + {samples: 1, expectedError: nil}, + {samples: 2, metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 2 samples and 1 metadata")}, + {samples: 2, expectedError: nil}, + {samples: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 1 samples and 0 metadata")}, + {metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 0 samples and 1 metadata")}, + }, + }, + "global strategy: burst should set to each distributor": { + distributors: 2, + ingestionRateStrategy: validation.GlobalIngestionRateStrategy, + ingestionRate: 10, + ingestionBurstSize: 20, + pushes: []testPush{ + {samples: 10, expectedError: nil}, + {samples: 5, expectedError: nil}, + {samples: 5, metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 5 samples and 1 metadata")}, + {samples: 5, expectedError: nil}, + {samples: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 1 samples and 0 metadata")}, + {metadata: 1, expectedError: httpgrpc.Errorf(http.StatusTooManyRequests, "ingestion rate limit (5) exceeded while adding 0 samples and 1 metadata")}, + }, + }, + } + + for testName, testData := range tests { + testData := testData + + for _, enableHistogram := range []bool{false, true} { + enableHistogram := enableHistogram + t.Run(fmt.Sprintf("%s, histogram=%s", testName, strconv.FormatBool(enableHistogram)), func(t *testing.T) { + t.Parallel() + limits := &validation.Limits{} + flagext.DefaultValues(limits) + limits.IngestionRateStrategy = testData.ingestionRateStrategy + limits.IngestionRate = testData.ingestionRate + limits.IngestionBurstSize = testData.ingestionBurstSize + + // Start all expected distributors + distributors, _, _, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: testData.distributors, + shardByAllLabels: true, + limits: limits, + }) + + // Push samples in multiple requests to the first distributor + for _, push := range testData.pushes { + var request *cortexpbv2.WriteRequest + if !enableHistogram { + request = makeWriteRequestV2WithSamples(0, push.samples, push.metadata) + } else { + request = makeWriteRequestV2WithHistogram(0, push.samples, push.metadata) + } + response, err := distributors[0].PushV2(ctx, request) + + if push.expectedError == nil { + assert.Equal(t, emptyResponseV2, response) + assert.Nil(t, err) + } else { + assert.Nil(t, response) + assert.Equal(t, push.expectedError, err) + } + } + }) + } + } +} + +func TestPushPRW2_QuorumError(t *testing.T) { + t.Parallel() + + var limits validation.Limits + flagext.DefaultValues(&limits) + + limits.IngestionRate = math.MaxFloat64 + + dists, ingesters, _, r := prepare(t, prepConfig{ + numDistributors: 1, + numIngesters: 3, + happyIngesters: 0, + shuffleShardSize: 3, + shardByAllLabels: true, + shuffleShardEnabled: true, + limits: &limits, + }) + + ctx := user.InjectOrgID(context.Background(), "user") + + d := dists[0] + + // we should run several write request to make sure we dont have any race condition on the batchTracker#record code + numberOfWrites := 10000 + + // Using 429 just to make sure we are not hitting the &limits + // Simulating 2 4xx and 1 5xx -> Should return 4xx + ingesters[0].failResp.Store(httpgrpc.Errorf(429, "Throttling")) + ingesters[1].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + ingesters[2].failResp.Store(httpgrpc.Errorf(429, "Throttling")) + + for i := 0; i < numberOfWrites; i++ { + request := makeWriteRequestV2WithSamples(0, 30, 20) + _, err := d.PushV2(ctx, request) + status, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.Code(429), status.Code()) + } + + // Simulating 2 5xx and 1 4xx -> Should return 5xx + ingesters[0].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + ingesters[1].failResp.Store(httpgrpc.Errorf(429, "Throttling")) + ingesters[2].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + + for i := 0; i < numberOfWrites; i++ { + request := makeWriteRequestV2WithSamples(0, 300, 200) + _, err := d.PushV2(ctx, request) + status, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.Code(500), status.Code()) + } + + // Simulating 2 different errors and 1 success -> This case we may return any of the errors + ingesters[0].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + ingesters[1].failResp.Store(httpgrpc.Errorf(429, "Throttling")) + ingesters[2].happy.Store(true) + + for i := 0; i < numberOfWrites; i++ { + request := makeWriteRequestV2WithSamples(0, 30, 20) + _, err := d.PushV2(ctx, request) + status, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.Code(429), status.Code()) + } + + // Simulating 1 error -> Should return 2xx + ingesters[0].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + ingesters[1].happy.Store(true) + ingesters[2].happy.Store(true) + + for i := 0; i < 1; i++ { + request := makeWriteRequestV2WithSamples(0, 30, 20) + _, err := d.PushV2(ctx, request) + require.NoError(t, err) + } + + // Simulating an unhealthy ingester (ingester 2) + ingesters[0].failResp.Store(httpgrpc.Errorf(500, "InternalServerError")) + ingesters[1].happy.Store(true) + ingesters[2].happy.Store(true) + + err := r.KVClient.CAS(context.Background(), ingester.RingKey, func(in interface{}) (interface{}, bool, error) { + r := in.(*ring.Desc) + ingester2 := r.Ingesters["ingester-2"] + ingester2.State = ring.LEFT + ingester2.Timestamp = time.Now().Unix() + r.Ingesters["ingester-2"] = ingester2 + return in, true, nil + }) + + require.NoError(t, err) + + // Give time to the ring get updated with the KV value + test.Poll(t, 15*time.Second, true, func() interface{} { + replicationSet, _ := r.GetAllHealthy(ring.Read) + return len(replicationSet.Instances) == 2 + }) + + for i := 0; i < numberOfWrites; i++ { + request := makeWriteRequestV2WithSamples(0, 30, 20) + _, err := d.PushV2(ctx, request) + require.Error(t, err) + status, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.Code(500), status.Code()) + } +} + +func TestDistributorPRW2_PushInstanceLimits(t *testing.T) { + t.Parallel() + + type testPush struct { + samples int + metadata int + expectedError error + } + + ctx := user.InjectOrgID(context.Background(), "user") + tests := map[string]struct { + preInflight int + preRateSamples int // initial rate before first push + pushes []testPush // rate is recomputed after each push + + // limits + inflightLimit int + ingestionRateLimit float64 + + metricNames []string + expectedMetrics string + }{ + "no limits limit": { + preInflight: 100, + preRateSamples: 1000, + + pushes: []testPush{ + {samples: 100, expectedError: nil}, + }, + + metricNames: []string{instanceLimitsMetric}, + expectedMetrics: ` + # HELP cortex_distributor_instance_limits Instance limits used by this distributor. + # TYPE cortex_distributor_instance_limits gauge + cortex_distributor_instance_limits{limit="max_inflight_push_requests"} 0 + cortex_distributor_instance_limits{limit="max_ingestion_rate"} 0 + `, + }, + "below inflight limit": { + preInflight: 100, + inflightLimit: 101, + pushes: []testPush{ + {samples: 100, expectedError: nil}, + }, + + metricNames: []string{instanceLimitsMetric, "cortex_distributor_inflight_push_requests"}, + expectedMetrics: ` + # HELP cortex_distributor_inflight_push_requests Current number of inflight push requests in distributor. + # TYPE cortex_distributor_inflight_push_requests gauge + cortex_distributor_inflight_push_requests 100 + + # HELP cortex_distributor_instance_limits Instance limits used by this distributor. + # TYPE cortex_distributor_instance_limits gauge + cortex_distributor_instance_limits{limit="max_inflight_push_requests"} 101 + cortex_distributor_instance_limits{limit="max_ingestion_rate"} 0 + `, + }, + "hits inflight limit": { + preInflight: 101, + inflightLimit: 101, + pushes: []testPush{ + {samples: 100, expectedError: errTooManyInflightPushRequests}, + }, + }, + "below ingestion rate limit": { + preRateSamples: 500, + ingestionRateLimit: 1000, + + pushes: []testPush{ + {samples: 1000, expectedError: nil}, + }, + + metricNames: []string{instanceLimitsMetric, "cortex_distributor_ingestion_rate_samples_per_second"}, + expectedMetrics: ` + # HELP cortex_distributor_ingestion_rate_samples_per_second Current ingestion rate in samples/sec that distributor is using to limit access. + # TYPE cortex_distributor_ingestion_rate_samples_per_second gauge + cortex_distributor_ingestion_rate_samples_per_second 600 + + # HELP cortex_distributor_instance_limits Instance limits used by this distributor. + # TYPE cortex_distributor_instance_limits gauge + cortex_distributor_instance_limits{limit="max_inflight_push_requests"} 0 + cortex_distributor_instance_limits{limit="max_ingestion_rate"} 1000 + `, + }, + "hits rate limit on first request, but second request can proceed": { + preRateSamples: 1200, + ingestionRateLimit: 1000, + + pushes: []testPush{ + {samples: 100, expectedError: errMaxSamplesPushRateLimitReached}, + {samples: 100, expectedError: nil}, + }, + }, + "below rate limit on first request, but hits the rate limit afterwards": { + preRateSamples: 500, + ingestionRateLimit: 1000, + + pushes: []testPush{ + {samples: 5000, expectedError: nil}, // after push, rate = 500 + 0.2*(5000-500) = 1400 + {samples: 5000, expectedError: errMaxSamplesPushRateLimitReached}, // after push, rate = 1400 + 0.2*(0 - 1400) = 1120 + {samples: 5000, expectedError: errMaxSamplesPushRateLimitReached}, // after push, rate = 1120 + 0.2*(0 - 1120) = 896 + {samples: 5000, expectedError: nil}, // 896 is below 1000, so this push succeeds, new rate = 896 + 0.2*(5000-896) = 1716.8 + }, + }, + } + + for testName, testData := range tests { + testData := testData + + for _, enableHistogram := range []bool{true, false} { + enableHistogram := enableHistogram + t.Run(fmt.Sprintf("%s, histogram=%s", testName, strconv.FormatBool(enableHistogram)), func(t *testing.T) { + t.Parallel() + limits := &validation.Limits{} + flagext.DefaultValues(limits) + + // Start all expected distributors + distributors, _, regs, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + shardByAllLabels: true, + limits: limits, + maxInflightRequests: testData.inflightLimit, + maxIngestionRate: testData.ingestionRateLimit, + }) + + d := distributors[0] + d.inflightPushRequests.Add(int64(testData.preInflight)) + d.ingestionRate.Add(int64(testData.preRateSamples)) + + d.ingestionRate.Tick() + + for _, push := range testData.pushes { + var request *cortexpbv2.WriteRequest + if enableHistogram { + request = makeWriteRequestV2WithHistogram(0, push.samples, push.metadata) + } else { + request = makeWriteRequestV2WithSamples(0, push.samples, push.metadata) + } + _, err := d.PushV2(ctx, request) + + if push.expectedError == nil { + assert.Nil(t, err) + } else { + assert.Equal(t, push.expectedError, err) + } + + d.ingestionRate.Tick() + + if testData.expectedMetrics != "" { + assert.NoError(t, testutil.GatherAndCompare(regs[0], strings.NewReader(testData.expectedMetrics), testData.metricNames...)) + } + } + }) + } + } +} + +func TestDistributorPRW2_PushQuery(t *testing.T) { + t.Parallel() + const shuffleShardSize = 5 + + ctx := user.InjectOrgID(context.Background(), "user") + nameMatcher := mustEqualMatcher(model.MetricNameLabel, "foo") + barMatcher := mustEqualMatcher("bar", "baz") + + type testcase struct { + name string + numIngesters int + happyIngesters int + samples int + metadata int + matchers []*labels.Matcher + expectedIngesters int + expectedResponse model.Matrix + expectedError error + shardByAllLabels bool + shuffleShardEnabled bool + } + + // We'll programmatically build the test cases now, as we want complete + // coverage along quite a few different axis. + testcases := []testcase{} + + // Run every test in both sharding modes. + for _, shardByAllLabels := range []bool{true, false} { + + // Test with between 2 and 10 ingesters. + for numIngesters := 2; numIngesters < 10; numIngesters++ { + + // Test with between 0 and numIngesters "happy" ingesters. + for happyIngesters := 0; happyIngesters <= numIngesters; happyIngesters++ { + + // Test either with shuffle-sharding enabled or disabled. + for _, shuffleShardEnabled := range []bool{false, true} { + scenario := fmt.Sprintf("shardByAllLabels=%v, numIngester=%d, happyIngester=%d, shuffleSharding=%v)", shardByAllLabels, numIngesters, happyIngesters, shuffleShardEnabled) + + // The number of ingesters we expect to query depends whether shuffle sharding and/or + // shard by all labels are enabled. + var expectedIngesters int + if shuffleShardEnabled { + expectedIngesters = min(shuffleShardSize, numIngesters) + } else if shardByAllLabels { + expectedIngesters = numIngesters + } else { + expectedIngesters = 3 // Replication factor + } + + // When we're not sharding by metric name, queriers with more than one + // failed ingester should fail. + if shardByAllLabels && numIngesters-happyIngesters > 1 { + testcases = append(testcases, testcase{ + name: fmt.Sprintf("ExpectFail(%s)", scenario), + numIngesters: numIngesters, + happyIngesters: happyIngesters, + matchers: []*labels.Matcher{nameMatcher, barMatcher}, + expectedError: errFail, + shardByAllLabels: shardByAllLabels, + shuffleShardEnabled: shuffleShardEnabled, + }) + continue + } + + // When we have less ingesters than replication factor, any failed ingester + // will cause a failure. + if numIngesters < 3 && happyIngesters < 2 { + testcases = append(testcases, testcase{ + name: fmt.Sprintf("ExpectFail(%s)", scenario), + numIngesters: numIngesters, + happyIngesters: happyIngesters, + matchers: []*labels.Matcher{nameMatcher, barMatcher}, + expectedError: errFail, + shardByAllLabels: shardByAllLabels, + shuffleShardEnabled: shuffleShardEnabled, + }) + continue + } + + // If we're sharding by metric name and we have failed ingesters, we can't + // tell ahead of time if the query will succeed, as we don't know which + // ingesters will hold the results for the query. + if !shardByAllLabels && numIngesters-happyIngesters > 1 { + continue + } + + // Reading all the samples back should succeed. + testcases = append(testcases, testcase{ + name: fmt.Sprintf("ReadAll(%s)", scenario), + numIngesters: numIngesters, + happyIngesters: happyIngesters, + samples: 10, + matchers: []*labels.Matcher{nameMatcher, barMatcher}, + expectedResponse: expectedResponse(0, 10), + expectedIngesters: expectedIngesters, + shardByAllLabels: shardByAllLabels, + shuffleShardEnabled: shuffleShardEnabled, + }) + + // As should reading none of the samples back. + testcases = append(testcases, testcase{ + name: fmt.Sprintf("ReadNone(%s)", scenario), + numIngesters: numIngesters, + happyIngesters: happyIngesters, + samples: 10, + matchers: []*labels.Matcher{nameMatcher, mustEqualMatcher("not", "found")}, + expectedResponse: expectedResponse(0, 0), + expectedIngesters: expectedIngesters, + shardByAllLabels: shardByAllLabels, + shuffleShardEnabled: shuffleShardEnabled, + }) + + // And reading each sample individually. + for i := 0; i < 10; i++ { + testcases = append(testcases, testcase{ + name: fmt.Sprintf("ReadOne(%s, sample=%d)", scenario, i), + numIngesters: numIngesters, + happyIngesters: happyIngesters, + samples: 10, + matchers: []*labels.Matcher{nameMatcher, mustEqualMatcher("sample", strconv.Itoa(i))}, + expectedResponse: expectedResponse(i, i+1), + expectedIngesters: expectedIngesters, + shardByAllLabels: shardByAllLabels, + shuffleShardEnabled: shuffleShardEnabled, + }) + } + } + } + } + } + + for _, tc := range testcases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + ds, ingesters, _, _ := prepare(t, prepConfig{ + numIngesters: tc.numIngesters, + happyIngesters: tc.happyIngesters, + numDistributors: 1, + shardByAllLabels: tc.shardByAllLabels, + shuffleShardEnabled: tc.shuffleShardEnabled, + shuffleShardSize: shuffleShardSize, + }) + + request := makeWriteRequestV2WithSamples(0, tc.samples, tc.metadata) + writeResponse, err := ds[0].PushV2(ctx, request) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeResponse) + assert.Nil(t, err) + + var response model.Matrix + series, err := ds[0].QueryStream(ctx, 0, 10, tc.matchers...) + assert.Equal(t, tc.expectedError, err) + + if series == nil { + response, err = chunkcompat.SeriesChunksToMatrix(0, 10, nil) + } else { + response, err = chunkcompat.SeriesChunksToMatrix(0, 10, series.Chunkseries) + } + assert.NoError(t, err) + assert.Equal(t, tc.expectedResponse.String(), response.String()) + + // Check how many ingesters have been queried. + // Due to the quorum the distributor could cancel the last request towards ingesters + // if all other ones are successful, so we're good either has been queried X or X-1 + // ingesters. + if tc.expectedError == nil { + assert.Contains(t, []int{tc.expectedIngesters, tc.expectedIngesters - 1}, countMockIngestersCalls(ingesters, "QueryStream")) + } + }) + } +} + +func TestDistributorPRW2_QueryStream_ShouldReturnErrorIfMaxChunksPerQueryLimitIsReached(t *testing.T) { + t.Parallel() + const maxChunksLimit = 30 // Chunks are duplicated due to replication factor. + + for _, histogram := range []bool{true, false} { + ctx := user.InjectOrgID(context.Background(), "user") + limits := &validation.Limits{} + flagext.DefaultValues(limits) + limits.MaxChunksPerQuery = maxChunksLimit + + // Prepare distributors. + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + shardByAllLabels: true, + limits: limits, + }) + + ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, 0, maxChunksLimit, 0)) + + // Push a number of series below the max chunks limit. Each series has 1 sample, + // so expect 1 chunk per series when querying back. + initialSeries := maxChunksLimit / 3 + var writeReqV2 *cortexpbv2.WriteRequest + if histogram { + writeReqV2 = makeWriteRequestV2WithHistogram(0, initialSeries, 0) + } else { + writeReqV2 = makeWriteRequestV2WithSamples(0, initialSeries, 0) + } + + writeRes, err := ds[0].PushV2(ctx, writeReqV2) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + allSeriesMatchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, model.MetricNameLabel, ".+"), + } + + // Since the number of series (and thus chunks) is equal to the limit (but doesn't + // exceed it), we expect a query running on all series to succeed. + queryRes, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + assert.Len(t, queryRes.Chunkseries, initialSeries) + + // Push more series to exceed the limit once we'll query back all series. + + for i := 0; i < maxChunksLimit; i++ { + writeReq := &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", fmt.Sprintf("another_series_%d", i)} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: fmt.Sprintf("another_series_%d", i)}}, 0, 0, histogram, false), + ) + writeRes, err := ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + } + + // Since the number of series (and thus chunks) is exceeding to the limit, we expect + // a query running on all series to fail. + _, err = ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.Error(t, err) + assert.Contains(t, err.Error(), "the query hit the max number of chunks limit") + } +} + +func TestDistributorPRW2_QueryStream_ShouldReturnErrorIfMaxSeriesPerQueryLimitIsReached(t *testing.T) { + t.Parallel() + const maxSeriesLimit = 10 + + for _, histogram := range []bool{true, false} { + ctx := user.InjectOrgID(context.Background(), "user") + limits := &validation.Limits{} + flagext.DefaultValues(limits) + ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(maxSeriesLimit, 0, 0, 0)) + + // Prepare distributors. + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + shardByAllLabels: true, + limits: limits, + }) + + // Push a number of series below the max series limit. + initialSeries := maxSeriesLimit + var writeReqV2 *cortexpbv2.WriteRequest + if histogram { + writeReqV2 = makeWriteRequestV2WithHistogram(0, initialSeries, 0) + } else { + writeReqV2 = makeWriteRequestV2WithSamples(0, initialSeries, 0) + } + + writeRes, err := ds[0].PushV2(ctx, writeReqV2) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + allSeriesMatchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, model.MetricNameLabel, ".+"), + } + + // Since the number of series is equal to the limit (but doesn't + // exceed it), we expect a query running on all series to succeed. + queryRes, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + assert.Len(t, queryRes.Chunkseries, initialSeries) + + // Push more series to exceed the limit once we'll query back all series. + writeReq := &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", "another_series"} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: "another_series"}}, 0, 0, histogram, false), + ) + + writeRes, err = ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + // Since the number of series is exceeding the limit, we expect + // a query running on all series to fail. + _, err = ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.Error(t, err) + assert.Contains(t, err.Error(), "max number of series limit") + } +} + +func TestDistributorPRW2_QueryStream_ShouldReturnErrorIfMaxChunkBytesPerQueryLimitIsReached(t *testing.T) { + t.Parallel() + const seriesToAdd = 10 + + for _, histogram := range []bool{true, false} { + ctx := user.InjectOrgID(context.Background(), "user") + limits := &validation.Limits{} + flagext.DefaultValues(limits) + + // Prepare distributors. + // Use replication factor of 2 to always read all the chunks from both ingesters, + // this guarantees us to always read the same chunks and have a stable test. + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shardByAllLabels: true, + limits: limits, + replicationFactor: 2, + }) + + allSeriesMatchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, model.MetricNameLabel, ".+"), + } + // Push a single series to allow us to calculate the chunk size to calculate the limit for the test. + writeReq := &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", "another_series"} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: "another_series"}}, 0, 0, histogram, false), + ) + writeRes, err := ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + chunkSizeResponse, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + + // Use the resulting chunks size to calculate the limit as (series to add + our test series) * the response chunk size. + var responseChunkSize = chunkSizeResponse.ChunksSize() + var maxBytesLimit = (seriesToAdd) * responseChunkSize + + // Update the limiter with the calculated limits. + ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, maxBytesLimit, 0, 0)) + + // Push a number of series below the max chunk bytes limit. Subtract one for the series added above. + var writeReqV2 *cortexpbv2.WriteRequest + if histogram { + writeReqV2 = makeWriteRequestV2WithHistogram(0, seriesToAdd-1, 0) + } else { + writeReqV2 = makeWriteRequestV2WithSamples(0, seriesToAdd-1, 0) + } + + writeRes, err = ds[0].PushV2(ctx, writeReqV2) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + // Since the number of chunk bytes is equal to the limit (but doesn't + // exceed it), we expect a query running on all series to succeed. + queryRes, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + assert.Len(t, queryRes.Chunkseries, seriesToAdd) + + // Push another series to exceed the chunk bytes limit once we'll query back all series. + writeReq = &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", "another_series_1"} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: "another_series_1"}}, 0, 0, histogram, false), + ) + + writeRes, err = ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + // Since the aggregated chunk size is exceeding the limit, we expect + // a query running on all series to fail. + _, err = ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.Error(t, err) + assert.Equal(t, err, validation.LimitError(fmt.Sprintf(limiter.ErrMaxChunkBytesHit, maxBytesLimit))) + } +} + +func TestDistributorPRW2_QueryStream_ShouldReturnErrorIfMaxDataBytesPerQueryLimitIsReached(t *testing.T) { + t.Parallel() + const seriesToAdd = 10 + + for _, histogram := range []bool{true, false} { + ctx := user.InjectOrgID(context.Background(), "user") + limits := &validation.Limits{} + flagext.DefaultValues(limits) + + // Prepare distributors. + // Use replication factor of 2 to always read all the chunks from both ingesters, + // this guarantees us to always read the same chunks and have a stable test. + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shardByAllLabels: true, + limits: limits, + replicationFactor: 2, + }) + + allSeriesMatchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, model.MetricNameLabel, ".+"), + } + // Push a single series to allow us to calculate the label size to calculate the limit for the test. + writeReq := &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", "another_series"} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: "another_series"}}, 0, 0, histogram, false), + ) + + writeRes, err := ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + dataSizeResponse, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + + // Use the resulting chunks size to calculate the limit as (series to add + our test series) * the response chunk size. + var dataSize = dataSizeResponse.Size() + var maxBytesLimit = (seriesToAdd) * dataSize * 2 // Multiplying by RF because the limit is applied before de-duping. + + // Update the limiter with the calculated limits. + ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, 0, 0, maxBytesLimit)) + + // Push a number of series below the max chunk bytes limit. Subtract one for the series added above. + var writeReqV2 *cortexpbv2.WriteRequest + if histogram { + writeReqV2 = makeWriteRequestV2WithHistogram(0, seriesToAdd-1, 0) + } else { + writeReqV2 = makeWriteRequestV2WithSamples(0, seriesToAdd-1, 0) + } + + writeRes, err = ds[0].PushV2(ctx, writeReqV2) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + // Since the number of chunk bytes is equal to the limit (but doesn't + // exceed it), we expect a query running on all series to succeed. + queryRes, err := ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.NoError(t, err) + assert.Len(t, queryRes.Chunkseries, seriesToAdd) + + // Push another series to exceed the chunk bytes limit once we'll query back all series. + writeReq = &cortexpbv2.WriteRequest{} + writeReq.Symbols = []string{"", "__name__", "another_series_1"} + writeReq.Timeseries = append(writeReq.Timeseries, + makeWriteRequestV2Timeseries([]cortexpb.LabelAdapter{{Name: model.MetricNameLabel, Value: "another_series_1"}}, 0, 0, histogram, false), + ) + + writeRes, err = ds[0].PushV2(ctx, writeReq) + assert.Equal(t, &cortexpbv2.WriteResponse{}, writeRes) + assert.Nil(t, err) + + // Since the aggregated chunk size is exceeding the limit, we expect + // a query running on all series to fail. + _, err = ds[0].QueryStream(ctx, math.MinInt32, math.MaxInt32, allSeriesMatchers...) + require.Error(t, err) + assert.Equal(t, err, validation.LimitError(fmt.Sprintf(limiter.ErrMaxDataBytesHit, maxBytesLimit))) + } +} + +func TestDistributorPRW2_Push_ShouldGuaranteeShardingTokenConsistencyOverTheTime(t *testing.T) { + t.Parallel() + ctx := user.InjectOrgID(context.Background(), "user") + tests := map[string]struct { + inputSeries labels.Labels + expectedSeries labels.Labels + expectedToken uint32 + }{ + "metric_1 with value_1": { + inputSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + }, + expectedToken: 0xec0a2e9d, + }, + "metric_1 with value_1 and dropped label due to config": { + inputSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + {Name: "dropped", Value: "unused"}, // will be dropped, doesn't need to be in correct order + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + }, + expectedToken: 0xec0a2e9d, + }, + "metric_1 with value_1 and dropped HA replica label": { + inputSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + {Name: "__replica__", Value: "replica_1"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "cluster", Value: "cluster_1"}, + {Name: "key", Value: "value_1"}, + }, + expectedToken: 0xec0a2e9d, + }, + "metric_2 with value_1": { + inputSeries: labels.Labels{ + {Name: "__name__", Value: "metric_2"}, + {Name: "key", Value: "value_1"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "metric_2"}, + {Name: "key", Value: "value_1"}, + }, + expectedToken: 0xa60906f2, + }, + "metric_1 with value_2": { + inputSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "key", Value: "value_2"}, + }, + expectedSeries: labels.Labels{ + {Name: "__name__", Value: "metric_1"}, + {Name: "key", Value: "value_2"}, + }, + expectedToken: 0x18abc8a2, + }, + } + + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.DropLabels = []string{"dropped"} + limits.AcceptHASamples = true + + for testName, testData := range tests { + testData := testData + t.Run(testName, func(t *testing.T) { + t.Parallel() + ds, ingesters, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shardByAllLabels: true, + limits: &limits, + }) + + // Push the series to the distributor + req := mockWriteRequestV2([]labels.Labels{testData.inputSeries}, 1, 1, false) + _, err := ds[0].PushV2(ctx, req) + require.NoError(t, err) + + // Since each test pushes only 1 series, we do expect the ingester + // to have received exactly 1 series + for i := range ingesters { + timeseries := ingesters[i].series() + assert.Equal(t, 1, len(timeseries)) + + series, ok := timeseries[testData.expectedToken] + require.True(t, ok) + assert.Equal(t, testData.expectedSeries, cortexpb.FromLabelAdaptersToLabels(series.Labels)) + } + }) + } +} + +func makeWriteRequestV2WithSamples(startTimestampMs int64, samples int, metadata int) *cortexpbv2.WriteRequest { + request := &cortexpbv2.WriteRequest{} + st := writev2.NewSymbolTable() + st.SymbolizeLabels(labels.Labels{{Name: "__name__", Value: "foo"}, {Name: "bar", Value: "baz"}}, nil) + + for i := 0; i < samples; i++ { + st.SymbolizeLabels(labels.Labels{{Name: "sample", Value: fmt.Sprintf("%d", i)}, {Name: "bar", Value: "baz"}}, nil) + request.Timeseries = append(request.Timeseries, makeTimeseriesV2FromST( + []cortexpb.LabelAdapter{ + {Name: model.MetricNameLabel, Value: "foo"}, + {Name: "bar", Value: "baz"}, + {Name: "sample", Value: fmt.Sprintf("%d", i)}, + }, &st, startTimestampMs+int64(i), i, false, i < metadata)) + } + + for i := 0; i < metadata-samples; i++ { + request.Timeseries = append(request.Timeseries, makeMetadataV2FromST(i, &st)) + } + + request.Symbols = st.Symbols() + + return request +} + +func TestDistributorPRW2_Push_LabelNameValidation(t *testing.T) { + t.Parallel() + inputLabels := labels.Labels{ + {Name: model.MetricNameLabel, Value: "foo"}, + {Name: "999.illegal", Value: "baz"}, + } + ctx := user.InjectOrgID(context.Background(), "user") + + tests := map[string]struct { + inputLabels labels.Labels + skipLabelNameValidationCfg bool + skipLabelNameValidationReq bool + errExpected bool + errMessage string + }{ + "label name validation is on by default": { + inputLabels: inputLabels, + errExpected: true, + errMessage: `sample invalid label: "999.illegal" metric "foo{999.illegal=\"baz\"}"`, + }, + "label name validation can be skipped via config": { + inputLabels: inputLabels, + skipLabelNameValidationCfg: true, + errExpected: false, + }, + "label name validation can be skipped via WriteRequest parameter": { + inputLabels: inputLabels, + skipLabelNameValidationReq: true, + errExpected: false, + }, + } + + for testName, tc := range tests { + tc := tc + for _, histogram := range []bool{true, false} { + histogram := histogram + t.Run(fmt.Sprintf("%s, histogram=%s", testName, strconv.FormatBool(histogram)), func(t *testing.T) { + t.Parallel() + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shuffleShardSize: 1, + skipLabelNameValidation: tc.skipLabelNameValidationCfg, + }) + req := mockWriteRequestV2([]labels.Labels{tc.inputLabels}, 42, 100000, histogram) + req.SkipLabelNameValidation = tc.skipLabelNameValidationReq + _, err := ds[0].PushV2(ctx, req) + if tc.errExpected { + fromError, _ := status.FromError(err) + assert.Equal(t, tc.errMessage, fromError.Message()) + } else { + assert.Nil(t, err) + } + }) + } + } +} + +func TestDistributorPRW2_Push_ExemplarValidation(t *testing.T) { + t.Parallel() + ctx := user.InjectOrgID(context.Background(), "user") + manyLabels := []string{model.MetricNameLabel, "test"} + for i := 1; i < 31; i++ { + manyLabels = append(manyLabels, fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + tests := map[string]struct { + req *cortexpbv2.WriteRequest + errMsg string + }{ + "valid exemplar": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test"}, 1000, []string{"foo", "bar"}), + }, + "rejects exemplar with no labels": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test"}, 1000, []string{}), + errMsg: `exemplar missing labels, timestamp: 1000 series: {__name__="test"} labels: {}`, + }, + "rejects exemplar with no timestamp": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test"}, 0, []string{"foo", "bar"}), + errMsg: `exemplar missing timestamp, timestamp: 0 series: {__name__="test"} labels: {foo="bar"}`, + }, + "rejects exemplar with too long labelset": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test"}, 1000, []string{"foo", strings.Repeat("0", 126)}), + errMsg: fmt.Sprintf(`exemplar combined labelset exceeds 128 characters, timestamp: 1000 series: {__name__="test"} labels: {foo="%s"}`, strings.Repeat("0", 126)), + }, + "rejects exemplar with too many series labels": { + req: makeWriteRequestV2Exemplar(manyLabels, 0, nil), + errMsg: "series has too many labels", + }, + "rejects exemplar with duplicate series labels": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test", "foo", "bar", "foo", "bar"}, 0, nil), + errMsg: "duplicate label name", + }, + "rejects exemplar with empty series label name": { + req: makeWriteRequestV2Exemplar([]string{model.MetricNameLabel, "test", "", "bar"}, 0, nil), + errMsg: "invalid label", + }, + } + + for testName, tc := range tests { + tc := tc + t.Run(testName, func(t *testing.T) { + t.Parallel() + ds, _, _, _ := prepare(t, prepConfig{ + numIngesters: 2, + happyIngesters: 2, + numDistributors: 1, + shuffleShardSize: 1, + }) + _, err := ds[0].PushV2(ctx, tc.req) + if tc.errMsg != "" { + fromError, _ := status.FromError(err) + assert.Contains(t, fromError.Message(), tc.errMsg) + } else { + assert.Nil(t, err) + } + }) + } +} + +func TestDistributorPRW2_MetricsForLabelMatchers_SingleSlowIngester(t *testing.T) { + t.Parallel() + for _, histogram := range []bool{true, false} { + // Create distributor + ds, ing, _, _ := prepare(t, prepConfig{ + numIngesters: 3, + happyIngesters: 3, + numDistributors: 1, + shardByAllLabels: true, + shuffleShardEnabled: true, + shuffleShardSize: 3, + replicationFactor: 3, + }) + + ing[2].queryDelay = 50 * time.Millisecond + + ctx := user.InjectOrgID(context.Background(), "test") + + now := model.Now() + + for i := 0; i < 100; i++ { + req := mockWriteRequestV2([]labels.Labels{{{Name: labels.MetricName, Value: "test"}, {Name: "app", Value: "m"}, {Name: "uniq8", Value: strconv.Itoa(i)}}}, 1, now.Unix(), histogram) + _, err := ds[0].PushV2(ctx, req) + require.NoError(t, err) + } + + for i := 0; i < 50; i++ { + _, err := ds[0].MetricsForLabelMatchers(ctx, now, now, nil, mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test")) + require.NoError(t, err) + } + } +} + +func TestDistributorPRW2_MetricsForLabelMatchers(t *testing.T) { + t.Parallel() + const numIngesters = 5 + + fixtures := []struct { + lbls labels.Labels + value int64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "500"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + // The two following series have the same FastFingerprint=e002a3a451262627 + {labels.Labels{{Name: labels.MetricName, Value: "fast_fingerprint_collision"}, {Name: "app", Value: "l"}, {Name: "uniq0", Value: "0"}, {Name: "uniq1", Value: "1"}}, 1, 300000}, + {labels.Labels{{Name: labels.MetricName, Value: "fast_fingerprint_collision"}, {Name: "app", Value: "m"}, {Name: "uniq0", Value: "1"}, {Name: "uniq1", Value: "1"}}, 1, 300000}, + } + + tests := map[string]struct { + shuffleShardEnabled bool + shuffleShardSize int + matchers []*labels.Matcher + expectedResult []model.Metric + expectedIngesters int + queryLimiter *limiter.QueryLimiter + expectedErr error + }{ + "should return an empty response if no metric match": { + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "unknown"), + }, + expectedResult: []model.Metric{}, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should filter metrics by single matcher": { + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[0].lbls), + util.LabelsToMetric(fixtures[1].lbls), + }, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should filter metrics by multiple matchers": { + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "status", "200"), + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[0].lbls), + }, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should return all matching metrics even if their FastFingerprint collide": { + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "fast_fingerprint_collision"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[3].lbls), + util.LabelsToMetric(fixtures[4].lbls), + }, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should query only ingesters belonging to tenant's subring if shuffle sharding is enabled": { + shuffleShardEnabled: true, + shuffleShardSize: 3, + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[0].lbls), + util.LabelsToMetric(fixtures[1].lbls), + }, + expectedIngesters: 3, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should query all ingesters if shuffle sharding is enabled but shard size is 0": { + shuffleShardEnabled: true, + shuffleShardSize: 0, + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[0].lbls), + util.LabelsToMetric(fixtures[1].lbls), + }, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 0), + expectedErr: nil, + }, + "should return err if series limit is exhausted": { + shuffleShardEnabled: true, + shuffleShardSize: 0, + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: nil, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(1, 0, 0, 0), + expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxSeriesHit, 1)), + }, + "should return err if data bytes limit is exhausted": { + shuffleShardEnabled: true, + shuffleShardSize: 0, + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_1"), + }, + expectedResult: nil, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(0, 0, 0, 1), + expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxDataBytesHit, 1)), + }, + "should not exhaust series limit when only one series is fetched": { + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_2"), + }, + expectedResult: []model.Metric{ + util.LabelsToMetric(fixtures[2].lbls), + }, + expectedIngesters: numIngesters, + queryLimiter: limiter.NewQueryLimiter(1, 0, 0, 0), + expectedErr: nil, + }, + } + + for testName, testData := range tests { + testData := testData + for _, histogram := range []bool{true, false} { + histogram := histogram + t.Run(fmt.Sprintf("%s, histogram=%s", testName, strconv.FormatBool(histogram)), func(t *testing.T) { + t.Parallel() + now := model.Now() + + // Create distributor + ds, ingesters, _, _ := prepare(t, prepConfig{ + numIngesters: numIngesters, + happyIngesters: numIngesters, + numDistributors: 1, + shardByAllLabels: true, + shuffleShardEnabled: testData.shuffleShardEnabled, + shuffleShardSize: testData.shuffleShardSize, + }) + + // Push fixtures + ctx := user.InjectOrgID(context.Background(), "test") + ctx = limiter.AddQueryLimiterToContext(ctx, testData.queryLimiter) + + for _, series := range fixtures { + req := mockWriteRequestV2([]labels.Labels{series.lbls}, series.value, series.timestamp, histogram) + _, err := ds[0].PushV2(ctx, req) + require.NoError(t, err) + } + + { + metrics, err := ds[0].MetricsForLabelMatchers(ctx, now, now, nil, testData.matchers...) + + if testData.expectedErr != nil { + assert.ErrorIs(t, err, testData.expectedErr) + return + } + + require.NoError(t, err) + assert.ElementsMatch(t, testData.expectedResult, metrics) + + // Check how many ingesters have been queried. + // Due to the quorum the distributor could cancel the last request towards ingesters + // if all other ones are successful, so we're good either has been queried X or X-1 + // ingesters. + assert.Contains(t, []int{testData.expectedIngesters, testData.expectedIngesters - 1}, countMockIngestersCalls(ingesters, "MetricsForLabelMatchers")) + } + + { + metrics, err := ds[0].MetricsForLabelMatchersStream(ctx, now, now, nil, testData.matchers...) + if testData.expectedErr != nil { + assert.ErrorIs(t, err, testData.expectedErr) + return + } + + require.NoError(t, err) + assert.ElementsMatch(t, testData.expectedResult, metrics) + + assert.Contains(t, []int{testData.expectedIngesters, testData.expectedIngesters - 1}, countMockIngestersCalls(ingesters, "MetricsForLabelMatchersStream")) + } + }) + } + } +} + +func BenchmarkDistributorPRW2_MetricsForLabelMatchers(b *testing.B) { + const ( + numIngesters = 100 + numSeriesPerRequest = 100 + ) + + tests := map[string]struct { + prepareConfig func(limits *validation.Limits) + prepareSeries func() ([]labels.Labels, []cortexpbv2.Sample) + matchers []*labels.Matcher + queryLimiter *limiter.QueryLimiter + expectedErr error + }{ + "get series within limits": { + prepareConfig: func(limits *validation.Limits) {}, + prepareSeries: func() ([]labels.Labels, []cortexpbv2.Sample) { + metrics := make([]labels.Labels, numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, numSeriesPerRequest) + + for i := 0; i < numSeriesPerRequest; i++ { + lbls := labels.NewBuilder(labels.Labels{{Name: model.MetricNameLabel, Value: fmt.Sprintf("foo_%d", i)}}) + for i := 0; i < 10; i++ { + lbls.Set(fmt.Sprintf("name_%d", i), fmt.Sprintf("value_%d", i)) + } + + metrics[i] = lbls.Labels() + samples[i] = cortexpbv2.Sample{ + Value: float64(i), + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + } + } + + return metrics, samples + }, + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchRegexp, model.MetricNameLabel, "foo.+"), + }, + queryLimiter: limiter.NewQueryLimiter(100, 0, 0, 0), + expectedErr: nil, + }, + } + + for testName, testData := range tests { + b.Run(testName, func(b *testing.B) { + // Create distributor + ds, ingesters, _, _ := prepare(b, prepConfig{ + numIngesters: numIngesters, + happyIngesters: numIngesters, + numDistributors: 1, + shardByAllLabels: true, + shuffleShardEnabled: false, + shuffleShardSize: 0, + }) + + // Push fixtures + ctx := user.InjectOrgID(context.Background(), "test") + ctx = limiter.AddQueryLimiterToContext(ctx, testData.queryLimiter) + + // Prepare the series to remote write before starting the benchmark. + metrics, samples := testData.prepareSeries() + + if _, err := ds[0].PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)); err != nil { + b.Fatalf("error pushing to distributor %v", err) + } + + // Run the benchmark. + b.ReportAllocs() + b.ResetTimer() + + for n := 0; n < b.N; n++ { + now := model.Now() + metrics, err := ds[0].MetricsForLabelMatchers(ctx, now, now, nil, testData.matchers...) + + if testData.expectedErr != nil { + assert.EqualError(b, err, testData.expectedErr.Error()) + return + } + + require.NoError(b, err) + + // Check how many ingesters have been queried. + // Due to the quorum the distributor could cancel the last request towards ingesters + // if all other ones are successful, so we're good either has been queried X or X-1 + // ingesters. + assert.Contains(b, []int{numIngesters, numIngesters - 1}, countMockIngestersCalls(ingesters, "MetricsForLabelMatchers")) + assert.Equal(b, numSeriesPerRequest, len(metrics)) + } + }) + } +} + +func TestDistributorPRW2_MetricsMetadata(t *testing.T) { + t.Parallel() + const numIngesters = 5 + + tests := map[string]struct { + shuffleShardEnabled bool + shuffleShardSize int + expectedIngesters int + }{ + "should query all ingesters if shuffle sharding is disabled": { + shuffleShardEnabled: false, + expectedIngesters: numIngesters, + }, + "should query all ingesters if shuffle sharding is enabled but shard size is 0": { + shuffleShardEnabled: true, + shuffleShardSize: 0, + expectedIngesters: numIngesters, + }, + "should query only ingesters belonging to tenant's subring if shuffle sharding is enabled": { + shuffleShardEnabled: true, + shuffleShardSize: 3, + expectedIngesters: 3, + }, + } + + for testName, testData := range tests { + testData := testData + t.Run(testName, func(t *testing.T) { + t.Parallel() + // Create distributor + ds, ingesters, _, _ := prepare(t, prepConfig{ + numIngesters: numIngesters, + happyIngesters: numIngesters, + numDistributors: 1, + shardByAllLabels: true, + shuffleShardEnabled: testData.shuffleShardEnabled, + shuffleShardSize: testData.shuffleShardSize, + limits: nil, + }) + + // Push metadata + ctx := user.InjectOrgID(context.Background(), "test") + + req := makeWriteRequestV2WithSamples(0, 0, 10) + _, err := ds[0].PushV2(ctx, req) + require.NoError(t, err) + + // Assert on metric metadata + metadata, err := ds[0].MetricsMetadata(ctx) + require.NoError(t, err) + assert.Equal(t, 10, len(metadata)) + + // Check how many ingesters have been queried. + // Due to the quorum the distributor could cancel the last request towards ingesters + // if all other ones are successful, so we're good either has been queried X or X-1 + // ingesters. + assert.Contains(t, []int{testData.expectedIngesters, testData.expectedIngesters - 1}, countMockIngestersCalls(ingesters, "MetricsMetadata")) + }) + } +} + +func makeWriteRequestV2WithHistogram(startTimestampMs int64, histogram int, metadata int) *cortexpbv2.WriteRequest { + request := &cortexpbv2.WriteRequest{} + st := writev2.NewSymbolTable() + st.SymbolizeLabels(labels.Labels{{Name: "__name__", Value: "foo"}, {Name: "bar", Value: "baz"}}, nil) + + for i := 0; i < histogram; i++ { + st.SymbolizeLabels(labels.Labels{{Name: "histogram", Value: fmt.Sprintf("%d", i)}}, nil) + request.Timeseries = append(request.Timeseries, makeTimeseriesV2FromST( + []cortexpb.LabelAdapter{ + {Name: model.MetricNameLabel, Value: "foo"}, + {Name: "bar", Value: "baz"}, + {Name: "histogram", Value: fmt.Sprintf("%d", i)}, + }, &st, startTimestampMs+int64(i), i, true, i < metadata)) + } + + for i := 0; i < metadata-histogram; i++ { + request.Timeseries = append(request.Timeseries, makeMetadataV2FromST(i, &st)) + } + + request.Symbols = st.Symbols() + + return request +} + +func makeMetadataV2FromST(value int, st *writev2.SymbolsTable) cortexpbv2.PreallocTimeseriesV2 { + t := cortexpbv2.PreallocTimeseriesV2{ + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: []uint32{1, 2}, + }, + } + helpRef := st.Symbolize(fmt.Sprintf("a help for metric_%d", value)) + t.Metadata.Type = cortexpbv2.METRIC_TYPE_COUNTER + t.Metadata.HelpRef = helpRef + + return t +} + +func makeTimeseriesV2FromST(labels []cortexpb.LabelAdapter, st *writev2.SymbolsTable, ts int64, value int, histogram bool, metadata bool) cortexpbv2.PreallocTimeseriesV2 { + var helpRef uint32 + if metadata { + helpRef = st.Symbolize(fmt.Sprintf("a help for metric_%d", value)) + } + + t := cortexpbv2.PreallocTimeseriesV2{ + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: st.SymbolizeLabels(cortexpb.FromLabelAdaptersToLabels(labels), nil), + }, + } + if metadata { + t.Metadata.Type = cortexpbv2.METRIC_TYPE_COUNTER + t.Metadata.HelpRef = helpRef + } + + if histogram { + t.Histograms = append(t.Histograms, cortexpbv2.HistogramToHistogramProto(ts, tsdbutil.GenerateTestHistogram(value))) + } else { + t.Samples = append(t.Samples, cortexpbv2.Sample{ + Timestamp: ts, + Value: float64(value), + }) + } + + return t +} + +func makeWriteRequestV2Timeseries(labels []cortexpb.LabelAdapter, ts int64, value int, histogram bool, metadata bool) cortexpbv2.PreallocTimeseriesV2 { + st := writev2.NewSymbolTable() + st.SymbolizeLabels(cortexpb.FromLabelAdaptersToLabels(labels), nil) + + var helpRef uint32 + if metadata { + helpRef = st.Symbolize(fmt.Sprintf("a help for metric_%d", value)) + } + + t := cortexpbv2.PreallocTimeseriesV2{ + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: st.SymbolizeLabels(cortexpb.FromLabelAdaptersToLabels(labels), nil), + }, + } + if metadata { + t.Metadata.Type = cortexpbv2.METRIC_TYPE_COUNTER + t.Metadata.HelpRef = helpRef + } + + if histogram { + t.Histograms = append(t.Histograms, cortexpbv2.HistogramToHistogramProto(ts, tsdbutil.GenerateTestHistogram(value))) + } else { + t.Samples = append(t.Samples, cortexpbv2.Sample{ + Timestamp: ts, + Value: float64(value), + }) + } + + return t +} + +func makeWriteRequestV2Exemplar(seriesLabels []string, timestamp int64, exemplarLabels []string) *cortexpbv2.WriteRequest { + st := writev2.NewSymbolTable() + for _, l := range seriesLabels { + st.Symbolize(l) + } + for _, l := range exemplarLabels { + st.Symbolize(l) + } + + return &cortexpbv2.WriteRequest{ + Symbols: st.Symbols(), + Timeseries: []cortexpbv2.PreallocTimeseriesV2{ + { + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: cortexpbv2.GetLabelRefsFromLabelAdapters(st.Symbols(), cortexpb.FromLabelsToLabelAdapters(labels.FromStrings(seriesLabels...))), + Exemplars: []cortexpbv2.Exemplar{ + { + LabelsRefs: cortexpbv2.GetLabelRefsFromLabelAdapters(st.Symbols(), cortexpb.FromLabelsToLabelAdapters(labels.FromStrings(exemplarLabels...))), + Timestamp: timestamp, + }, + }, + }, + }, + }, + } +} + +func mockWriteRequestV2(lbls []labels.Labels, value int64, timestamp int64, histogram bool) *cortexpbv2.WriteRequest { + var ( + samples []cortexpbv2.Sample + histograms []cortexpbv2.Histogram + ) + if histogram { + histograms = make([]cortexpbv2.Histogram, len(lbls)) + for i := range lbls { + histograms[i] = cortexpbv2.HistogramToHistogramProto(timestamp, tsdbutil.GenerateTestHistogram(int(value))) + } + } else { + samples = make([]cortexpbv2.Sample, len(lbls)) + for i := range lbls { + samples[i] = cortexpbv2.Sample{ + Timestamp: timestamp, + Value: float64(value), + } + } + } + + return cortexpbv2.ToWriteRequestV2(lbls, samples, histograms, nil, cortexpbv2.API) +} + +func makeWriteRequestHAV2(samples int, replica, cluster string, histogram bool) *cortexpbv2.WriteRequest { + request := &cortexpbv2.WriteRequest{} + st := writev2.NewSymbolTable() + for i := 0; i < samples; i++ { + ts := cortexpbv2.PreallocTimeseriesV2{ + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: st.SymbolizeLabels(labels.Labels{{Name: "__name__", Value: "foo"}, {Name: "__replica__", Value: replica}, {Name: "bar", Value: "baz"}, {Name: "cluster", Value: cluster}, {Name: "sample", Value: fmt.Sprintf("%d", i)}}, nil), + }, + } + if histogram { + ts.Histograms = []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(int64(i), tsdbutil.GenerateTestHistogram(i)), + } + } else { + ts.Samples = []cortexpbv2.Sample{ + { + Value: float64(i), + Timestamp: int64(i), + }, + } + } + request.Timeseries = append(request.Timeseries, ts) + } + request.Symbols = st.Symbols() + return request +} diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index dce3d05c91..68d3353cf8 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -35,6 +35,7 @@ import ( promchunk "github.com/cortexproject/cortex/pkg/chunk/encoding" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" "github.com/cortexproject/cortex/pkg/ha" "github.com/cortexproject/cortex/pkg/ingester" "github.com/cortexproject/cortex/pkg/ingester/client" @@ -3069,6 +3070,77 @@ func (i *mockIngester) PushPreAlloc(ctx context.Context, in *cortexpb.PreallocWr return i.Push(ctx, &in.WriteRequest, opts...) } +func (i *mockIngester) PushPreAllocV2(ctx context.Context, in *cortexpbv2.PreallocWriteRequestV2, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + return i.PushV2(ctx, &in.WriteRequest, opts...) +} + +func (i *mockIngester) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + i.Lock() + defer i.Unlock() + + i.trackCall("PushV2") + + if !i.happy.Load() { + return nil, i.failResp.Load() + } + + if i.timeseries == nil { + i.timeseries = map[uint32]*cortexpb.PreallocTimeseries{} + } + + if i.metadata == nil { + i.metadata = map[uint32]map[cortexpb.MetricMetadata]struct{}{} + } + + orgid, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + b := labels.NewScratchBuilder(0) + + for _, series := range req.Timeseries { + tsLabels := series.ToLabels(&b, req.Symbols) + labels := cortexpb.FromLabelsToLabelAdapters(tsLabels) + hash := shardByAllLabels(orgid, labels) + existing, ok := i.timeseries[hash] + var v1Sample []cortexpb.Sample + for _, s := range series.Samples { + v1Sample = append(v1Sample, cortexpb.Sample{ + Value: s.Value, + TimestampMs: s.Timestamp, + }) + } + if !ok { + // Make a copy because the request Timeseries are reused + item := cortexpb.TimeSeries{ + Labels: make([]cortexpb.LabelAdapter, len(labels)), + Samples: make([]cortexpb.Sample, len(v1Sample)), + } + + copy(item.Labels, labels) + copy(item.Samples, v1Sample) + + i.timeseries[hash] = &cortexpb.PreallocTimeseries{TimeSeries: &item} + } else { + existing.Samples = append(existing.Samples, v1Sample...) + } + + if series.Metadata.Type != cortexpbv2.METRIC_TYPE_UNSPECIFIED { + m := series.Metadata.ToV1Metadata(tsLabels.Get(model.MetricNameLabel), req.Symbols) + hash = shardByMetricName(orgid, m.MetricFamilyName) + set, ok := i.metadata[hash] + if !ok { + set = map[cortexpb.MetricMetadata]struct{}{} + i.metadata[hash] = set + } + set[*m] = struct{}{} + } + } + + return &cortexpbv2.WriteResponse{}, nil +} + func (i *mockIngester) Push(ctx context.Context, req *cortexpb.WriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) { i.Lock() defer i.Unlock() @@ -3319,6 +3391,10 @@ func (i *noopIngester) PushPreAlloc(ctx context.Context, in *cortexpb.PreallocWr return nil, nil } +func (i *noopIngester) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + return nil, nil +} + func (i *noopIngester) Push(ctx context.Context, req *cortexpb.WriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) { return nil, nil } diff --git a/pkg/distributor/distributorpb/distributor.pb.go b/pkg/distributor/distributorpb/distributor.pb.go index 9711c9efe6..5946061f32 100644 --- a/pkg/distributor/distributorpb/distributor.pb.go +++ b/pkg/distributor/distributorpb/distributor.pb.go @@ -7,6 +7,7 @@ import ( context "context" fmt "fmt" cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" + cortexpbv2 "github.com/cortexproject/cortex/pkg/cortexpbv2" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" grpc "google.golang.org/grpc" @@ -29,21 +30,23 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package func init() { proto.RegisterFile("distributor.proto", fileDescriptor_c518e33639ca565d) } var fileDescriptor_c518e33639ca565d = []byte{ - // 212 bytes of a gzipped FileDescriptorProto + // 245 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x4c, 0xc9, 0x2c, 0x2e, 0x29, 0xca, 0x4c, 0x2a, 0x2d, 0xc9, 0x2f, 0xd2, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x46, 0x12, 0x92, 0x12, 0x49, 0xcf, 0x4f, 0xcf, 0x07, 0x8b, 0xeb, 0x83, 0x58, 0x10, 0x25, 0x52, 0x96, 0xe9, 0x99, 0x25, 0x19, 0xa5, 0x49, 0x7a, 0xc9, 0xf9, 0xb9, 0xfa, 0xc9, 0xf9, 0x45, 0x25, 0xa9, 0x15, 0x05, 0x45, 0xf9, 0x59, 0xa9, 0xc9, 0x25, 0x50, 0x9e, 0x7e, 0x41, 0x76, 0x3a, 0x4c, 0x22, - 0x09, 0xca, 0x80, 0x68, 0x35, 0xf2, 0xe0, 0xe2, 0x76, 0x41, 0x98, 0x2f, 0x64, 0xc9, 0xc5, 0x12, - 0x50, 0x5a, 0x9c, 0x21, 0x24, 0xa6, 0x07, 0x53, 0xae, 0x17, 0x5e, 0x94, 0x59, 0x92, 0x1a, 0x94, - 0x5a, 0x58, 0x9a, 0x5a, 0x5c, 0x22, 0x25, 0x8e, 0x21, 0x5e, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0xaa, - 0xc4, 0xe0, 0xe4, 0x7c, 0xe1, 0xa1, 0x1c, 0xc3, 0x8d, 0x87, 0x72, 0x0c, 0x1f, 0x1e, 0xca, 0x31, - 0x36, 0x3c, 0x92, 0x63, 0x5c, 0xf1, 0x48, 0x8e, 0xf1, 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, - 0x18, 0x1f, 0x3c, 0x92, 0x63, 0x7c, 0xf1, 0x48, 0x8e, 0xe1, 0xc3, 0x23, 0x39, 0xc6, 0x09, 0x8f, - 0xe5, 0x18, 0x2e, 0x3c, 0x96, 0x63, 0xb8, 0xf1, 0x58, 0x8e, 0x21, 0x8a, 0x17, 0xc9, 0x77, 0x05, - 0x49, 0x49, 0x6c, 0x60, 0x57, 0x19, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x0d, 0x5b, 0x37, 0x6f, - 0x08, 0x01, 0x00, 0x00, + 0x09, 0xca, 0x80, 0x6a, 0xb5, 0x25, 0x45, 0x6b, 0x99, 0x11, 0x94, 0x59, 0x66, 0x04, 0xd1, 0x6e, + 0xd4, 0xc9, 0xc8, 0xc5, 0xed, 0x82, 0x70, 0x9f, 0x90, 0x25, 0x17, 0x4b, 0x40, 0x69, 0x71, 0x86, + 0x90, 0x98, 0x1e, 0x4c, 0x8f, 0x5e, 0x78, 0x51, 0x66, 0x49, 0x6a, 0x50, 0x6a, 0x61, 0x69, 0x6a, + 0x71, 0x89, 0x94, 0x38, 0x86, 0x78, 0x71, 0x41, 0x7e, 0x5e, 0x71, 0xaa, 0x12, 0x83, 0x90, 0x3d, + 0x17, 0x1b, 0x48, 0x6b, 0x98, 0x91, 0x90, 0x84, 0x1e, 0xc2, 0x42, 0x54, 0xed, 0x92, 0x58, 0x64, + 0x60, 0x06, 0x38, 0x39, 0x5f, 0x78, 0x28, 0xc7, 0x70, 0xe3, 0xa1, 0x1c, 0xc3, 0x87, 0x87, 0x72, + 0x8c, 0x0d, 0x8f, 0xe4, 0x18, 0x57, 0x3c, 0x92, 0x63, 0x3c, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, + 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x5f, 0x3c, 0x92, 0x63, 0xf8, 0xf0, 0x48, 0x8e, 0x71, 0xc2, + 0x63, 0x39, 0x86, 0x0b, 0x8f, 0xe5, 0x18, 0x6e, 0x3c, 0x96, 0x63, 0x88, 0xe2, 0x45, 0x0a, 0xde, + 0x82, 0xa4, 0x24, 0x36, 0xb0, 0xbf, 0x8c, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x56, 0x3b, + 0xcc, 0x89, 0x01, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -59,6 +62,7 @@ const _ = grpc.SupportPackageIsVersion4 // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type DistributorClient interface { Push(ctx context.Context, in *cortexpb.WriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) + PushV2(ctx context.Context, in *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) } type distributorClient struct { @@ -78,9 +82,19 @@ func (c *distributorClient) Push(ctx context.Context, in *cortexpb.WriteRequest, return out, nil } +func (c *distributorClient) PushV2(ctx context.Context, in *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + out := new(cortexpbv2.WriteResponse) + err := c.cc.Invoke(ctx, "/distributor.Distributor/PushV2", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DistributorServer is the server API for Distributor service. type DistributorServer interface { Push(context.Context, *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) + PushV2(context.Context, *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) } // UnimplementedDistributorServer can be embedded to have forward compatible implementations. @@ -90,6 +104,9 @@ type UnimplementedDistributorServer struct { func (*UnimplementedDistributorServer) Push(ctx context.Context, req *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Push not implemented") } +func (*UnimplementedDistributorServer) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PushV2 not implemented") +} func RegisterDistributorServer(s *grpc.Server, srv DistributorServer) { s.RegisterService(&_Distributor_serviceDesc, srv) @@ -113,6 +130,24 @@ func _Distributor_Push_Handler(srv interface{}, ctx context.Context, dec func(in return interceptor(ctx, in, info, handler) } +func _Distributor_PushV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(cortexpbv2.WriteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DistributorServer).PushV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/distributor.Distributor/PushV2", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DistributorServer).PushV2(ctx, req.(*cortexpbv2.WriteRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Distributor_serviceDesc = grpc.ServiceDesc{ ServiceName: "distributor.Distributor", HandlerType: (*DistributorServer)(nil), @@ -121,6 +156,10 @@ var _Distributor_serviceDesc = grpc.ServiceDesc{ MethodName: "Push", Handler: _Distributor_Push_Handler, }, + { + MethodName: "PushV2", + Handler: _Distributor_PushV2_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "distributor.proto", diff --git a/pkg/distributor/distributorpb/distributor.proto b/pkg/distributor/distributorpb/distributor.proto index 5319ba44a1..93234f1abb 100644 --- a/pkg/distributor/distributorpb/distributor.proto +++ b/pkg/distributor/distributorpb/distributor.proto @@ -6,10 +6,12 @@ option go_package = "distributorpb"; import "gogoproto/gogo.proto"; import "github.com/cortexproject/cortex/pkg/cortexpb/cortex.proto"; +import "github.com/cortexproject/cortex/pkg/cortexpbv2/cortexv2.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.unmarshaler_all) = true; service Distributor { rpc Push(cortexpb.WriteRequest) returns (cortexpb.WriteResponse) {}; + rpc PushV2(cortexpbv2.WriteRequest) returns (cortexpbv2.WriteResponse) {}; } diff --git a/pkg/distributor/stats.go b/pkg/distributor/stats.go new file mode 100644 index 0000000000..0f7fbc332d --- /dev/null +++ b/pkg/distributor/stats.go @@ -0,0 +1,62 @@ +package distributor + +import ( + "go.uber.org/atomic" +) + +type WriteStats struct { + // Samples represents X-Prometheus-Remote-Write-Written-Samples + Samples atomic.Int64 + // Histograms represents X-Prometheus-Remote-Write-Written-Histograms + Histograms atomic.Int64 + // Exemplars represents X-Prometheus-Remote-Write-Written-Exemplars + Exemplars atomic.Int64 +} + +func (w *WriteStats) SetSamples(samples int64) { + if w == nil { + return + } + + w.Samples.Store(samples) +} + +func (w *WriteStats) SetHistograms(histograms int64) { + if w == nil { + return + } + + w.Histograms.Store(histograms) +} + +func (w *WriteStats) SetExemplars(exemplars int64) { + if w == nil { + return + } + + w.Exemplars.Store(exemplars) +} + +func (w *WriteStats) LoadSamples() int64 { + if w == nil { + return 0 + } + + return w.Samples.Load() +} + +func (w *WriteStats) LoadHistogram() int64 { + if w == nil { + return 0 + } + + return w.Histograms.Load() +} + +func (w *WriteStats) LoadExemplars() int64 { + if w == nil { + return 0 + } + + return w.Exemplars.Load() +} diff --git a/pkg/distributor/stats_test.go b/pkg/distributor/stats_test.go new file mode 100644 index 0000000000..10f0bf87b2 --- /dev/null +++ b/pkg/distributor/stats_test.go @@ -0,0 +1,41 @@ +package distributor + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func Test_SetAndLoad(t *testing.T) { + s := &WriteStats{} + + t.Run("Samples", func(t *testing.T) { + s.SetSamples(3) + assert.Equal(t, int64(3), s.LoadSamples()) + }) + t.Run("Histograms", func(t *testing.T) { + s.SetHistograms(10) + assert.Equal(t, int64(10), s.LoadHistogram()) + }) + t.Run("Exemplars", func(t *testing.T) { + s.SetExemplars(2) + assert.Equal(t, int64(2), s.LoadExemplars()) + }) +} + +func Test_NilReceiver(t *testing.T) { + var s *WriteStats + + t.Run("Samples", func(t *testing.T) { + s.SetSamples(3) + assert.Equal(t, int64(0), s.LoadSamples()) + }) + t.Run("Histograms", func(t *testing.T) { + s.SetHistograms(10) + assert.Equal(t, int64(0), s.LoadHistogram()) + }) + t.Run("Exemplars", func(t *testing.T) { + s.SetExemplars(2) + assert.Equal(t, int64(0), s.LoadExemplars()) + }) +} diff --git a/pkg/ingester/client/client.go b/pkg/ingester/client/client.go index b1c5a8b28a..b8b3ddefe9 100644 --- a/pkg/ingester/client/client.go +++ b/pkg/ingester/client/client.go @@ -5,6 +5,7 @@ import ( "flag" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" "github.com/cortexproject/cortex/pkg/util/grpcclient" "github.com/cortexproject/cortex/pkg/util/grpcencoding/snappyblock" @@ -43,6 +44,7 @@ type HealthAndIngesterClient interface { grpc_health_v1.HealthClient Close() error PushPreAlloc(ctx context.Context, in *cortexpb.PreallocWriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) + PushPreAllocV2(ctx context.Context, in *cortexpbv2.PreallocWriteRequestV2, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) } type closableHealthAndIngesterClient struct { @@ -55,6 +57,17 @@ type closableHealthAndIngesterClient struct { inflightPushRequests *prometheus.GaugeVec } +func (c *closableHealthAndIngesterClient) PushPreAllocV2(ctx context.Context, in *cortexpbv2.PreallocWriteRequestV2, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + return c.handlePushRequestV2(func() (*cortexpbv2.WriteResponse, error) { + out := new(cortexpbv2.WriteResponse) + err := c.conn.Invoke(ctx, "/cortex.Ingester/PushV2", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil + }) +} + func (c *closableHealthAndIngesterClient) PushPreAlloc(ctx context.Context, in *cortexpb.PreallocWriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) { return c.handlePushRequest(func() (*cortexpb.WriteResponse, error) { out := new(cortexpb.WriteResponse) @@ -72,6 +85,24 @@ func (c *closableHealthAndIngesterClient) Push(ctx context.Context, in *cortexpb }) } +func (c *closableHealthAndIngesterClient) PushV2(ctx context.Context, in *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + return c.handlePushRequestV2(func() (*cortexpbv2.WriteResponse, error) { + return c.IngesterClient.PushV2(ctx, in, opts...) + }) +} + +func (c *closableHealthAndIngesterClient) handlePushRequestV2(mainFunc func() (*cortexpbv2.WriteResponse, error)) (*cortexpbv2.WriteResponse, error) { + currentInflight := c.inflightRequests.Inc() + c.inflightPushRequests.WithLabelValues(c.addr).Set(float64(currentInflight)) + defer func() { + c.inflightPushRequests.WithLabelValues(c.addr).Set(float64(c.inflightRequests.Dec())) + }() + if c.maxInflightPushRequests > 0 && currentInflight > c.maxInflightPushRequests { + return nil, errTooManyInflightPushRequests + } + return mainFunc() +} + func (c *closableHealthAndIngesterClient) handlePushRequest(mainFunc func() (*cortexpb.WriteResponse, error)) (*cortexpb.WriteResponse, error) { currentInflight := c.inflightRequests.Inc() c.inflightPushRequests.WithLabelValues(c.addr).Set(float64(currentInflight)) diff --git a/pkg/ingester/client/cortex_mock_test.go b/pkg/ingester/client/cortex_mock_test.go index fd98c77082..5aff8bf302 100644 --- a/pkg/ingester/client/cortex_mock_test.go +++ b/pkg/ingester/client/cortex_mock_test.go @@ -6,12 +6,18 @@ import ( "github.com/stretchr/testify/mock" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" ) type IngesterServerMock struct { mock.Mock } +func (m *IngesterServerMock) PushV2(ctx context.Context, r *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + args := m.Called(ctx, r) + return args.Get(0).(*cortexpbv2.WriteResponse), args.Error(1) +} + func (m *IngesterServerMock) Push(ctx context.Context, r *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { args := m.Called(ctx, r) return args.Get(0).(*cortexpb.WriteResponse), args.Error(1) diff --git a/pkg/ingester/client/ingester.pb.go b/pkg/ingester/client/ingester.pb.go index 374348afae..e587962642 100644 --- a/pkg/ingester/client/ingester.pb.go +++ b/pkg/ingester/client/ingester.pb.go @@ -12,6 +12,7 @@ import ( fmt "fmt" cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" + cortexpbv2 "github.com/cortexproject/cortex/pkg/cortexpbv2" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" grpc "google.golang.org/grpc" @@ -1484,91 +1485,93 @@ func init() { func init() { proto.RegisterFile("ingester.proto", fileDescriptor_60f6df4f3586b478) } var fileDescriptor_60f6df4f3586b478 = []byte{ - // 1339 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x57, 0x4b, 0x6f, 0x14, 0xc7, - 0x13, 0xdf, 0xf1, 0x3e, 0xec, 0xad, 0x7d, 0xb0, 0x6e, 0x1b, 0xbc, 0x0c, 0x7f, 0xc6, 0x30, 0x88, - 0x7f, 0xac, 0x24, 0xd8, 0xe0, 0x24, 0x12, 0xe4, 0x85, 0x6c, 0x30, 0x60, 0xc0, 0x18, 0xc6, 0x86, - 0x44, 0x51, 0xa2, 0xd1, 0x78, 0xb7, 0xb1, 0x27, 0xcc, 0x63, 0x99, 0xee, 0x41, 0x90, 0x53, 0xa2, - 0x7c, 0x80, 0xe4, 0x98, 0x6b, 0x6e, 0xf9, 0x00, 0xf9, 0x10, 0x1c, 0x39, 0xe4, 0x80, 0x72, 0x40, - 0x61, 0x91, 0xa2, 0x1c, 0xc9, 0x37, 0x88, 0xa6, 0x1f, 0xf3, 0xf2, 0xf8, 0x41, 0x04, 0xb9, 0xed, - 0x54, 0xfd, 0xaa, 0xba, 0xea, 0xd7, 0x55, 0x5d, 0xb5, 0xd0, 0xb6, 0xbd, 0x4d, 0x4c, 0x28, 0x0e, - 0x66, 0x07, 0x81, 0x4f, 0x7d, 0x54, 0xeb, 0xf9, 0x01, 0xc5, 0x0f, 0xd5, 0xc9, 0x4d, 0x7f, 0xd3, - 0x67, 0xa2, 0xb9, 0xe8, 0x17, 0xd7, 0xaa, 0xe7, 0x36, 0x6d, 0xba, 0x15, 0x6e, 0xcc, 0xf6, 0x7c, - 0x77, 0x8e, 0x03, 0x07, 0x81, 0xff, 0x35, 0xee, 0x51, 0xf1, 0x35, 0x37, 0xb8, 0xb7, 0x29, 0x15, - 0x1b, 0xe2, 0x07, 0x37, 0xd5, 0x3f, 0x81, 0x86, 0x81, 0xad, 0xbe, 0x81, 0xef, 0x87, 0x98, 0x50, - 0x34, 0x0b, 0xa3, 0xf7, 0x43, 0x1c, 0xd8, 0x98, 0x74, 0x95, 0x63, 0xe5, 0x99, 0xc6, 0xfc, 0xe4, - 0xac, 0x80, 0xdf, 0x0a, 0x71, 0xf0, 0x48, 0xc0, 0x0c, 0x09, 0xd2, 0xcf, 0x43, 0x93, 0x9b, 0x93, - 0x81, 0xef, 0x11, 0x8c, 0xe6, 0x60, 0x34, 0xc0, 0x24, 0x74, 0xa8, 0xb4, 0x3f, 0x98, 0xb3, 0xe7, - 0x38, 0x43, 0xa2, 0xf4, 0x6b, 0xd0, 0xca, 0x68, 0xd0, 0x87, 0x00, 0xd4, 0x76, 0x31, 0x29, 0x0a, - 0x62, 0xb0, 0x31, 0xbb, 0x6e, 0xbb, 0x78, 0x8d, 0xe9, 0x16, 0x2b, 0x8f, 0x9f, 0x4d, 0x97, 0x8c, - 0x14, 0x5a, 0xff, 0x49, 0x81, 0x66, 0x3a, 0x4e, 0xf4, 0x2e, 0x20, 0x42, 0xad, 0x80, 0x9a, 0x0c, - 0x44, 0x2d, 0x77, 0x60, 0xba, 0x91, 0x53, 0x65, 0xa6, 0x6c, 0x74, 0x98, 0x66, 0x5d, 0x2a, 0x56, - 0x08, 0x9a, 0x81, 0x0e, 0xf6, 0xfa, 0x59, 0xec, 0x08, 0xc3, 0xb6, 0xb1, 0xd7, 0x4f, 0x23, 0x4f, - 0xc3, 0x98, 0x6b, 0xd1, 0xde, 0x16, 0x0e, 0x48, 0xb7, 0x9c, 0xe5, 0xe9, 0xba, 0xb5, 0x81, 0x9d, - 0x15, 0xae, 0x34, 0x62, 0x94, 0xfe, 0xb3, 0x02, 0x93, 0x4b, 0x0f, 0xb1, 0x3b, 0x70, 0xac, 0xe0, - 0x3f, 0x09, 0xf1, 0xcc, 0xb6, 0x10, 0x0f, 0x16, 0x85, 0x48, 0x52, 0x31, 0x7e, 0x09, 0x13, 0x2c, - 0xb4, 0x35, 0x1a, 0x60, 0xcb, 0x8d, 0x6f, 0xe4, 0x3c, 0x34, 0x7a, 0x5b, 0xa1, 0x77, 0x2f, 0x73, - 0x25, 0x53, 0xd2, 0x59, 0x72, 0x21, 0x17, 0x22, 0x90, 0xb8, 0x95, 0xb4, 0xc5, 0xd5, 0xca, 0xd8, - 0x48, 0xa7, 0xac, 0xaf, 0xc1, 0xc1, 0x1c, 0x01, 0xaf, 0xe1, 0xc6, 0x7f, 0x53, 0x00, 0xb1, 0x74, - 0xee, 0x58, 0x4e, 0x88, 0x89, 0x24, 0xf5, 0x28, 0x80, 0x13, 0x49, 0x4d, 0xcf, 0x72, 0x31, 0x23, - 0xb3, 0x6e, 0xd4, 0x99, 0xe4, 0x86, 0xe5, 0xe2, 0x1d, 0x38, 0x1f, 0x79, 0x05, 0xce, 0xcb, 0x7b, - 0x72, 0x5e, 0x39, 0xa6, 0xec, 0x83, 0x73, 0x34, 0x09, 0x55, 0xc7, 0x76, 0x6d, 0xda, 0xad, 0x32, - 0x8f, 0xfc, 0x43, 0x3f, 0x0b, 0x13, 0x99, 0xac, 0x04, 0x53, 0xc7, 0xa1, 0xc9, 0xd3, 0x7a, 0xc0, - 0xe4, 0x8c, 0xab, 0xba, 0xd1, 0x70, 0x12, 0xa8, 0xfe, 0x29, 0x1c, 0x4e, 0x59, 0xe6, 0x6e, 0x72, - 0x1f, 0xf6, 0xbf, 0x2a, 0x30, 0x7e, 0x5d, 0x12, 0x45, 0xde, 0x74, 0x91, 0xc6, 0xd9, 0x97, 0x53, - 0xd9, 0xff, 0x0b, 0x1a, 0xf5, 0x0f, 0x44, 0x19, 0x88, 0xa8, 0x45, 0xbe, 0xd3, 0xd0, 0x48, 0xca, - 0x40, 0xa6, 0x0b, 0x71, 0x1d, 0x10, 0xfd, 0x23, 0xe8, 0x26, 0x66, 0x39, 0xb2, 0xf6, 0x34, 0x46, - 0xd0, 0xb9, 0x4d, 0x70, 0xb0, 0x46, 0x2d, 0x2a, 0x89, 0xd2, 0xbf, 0x1b, 0x81, 0xf1, 0x94, 0x50, - 0xb8, 0x3a, 0x29, 0xdf, 0x73, 0xdb, 0xf7, 0xcc, 0xc0, 0xa2, 0xbc, 0x24, 0x15, 0xa3, 0x15, 0x4b, - 0x0d, 0x8b, 0xe2, 0xa8, 0x6a, 0xbd, 0xd0, 0x35, 0x45, 0x23, 0x44, 0x8c, 0x55, 0x8c, 0xba, 0x17, - 0xba, 0xbc, 0xfa, 0xa3, 0x4b, 0xb0, 0x06, 0xb6, 0x99, 0xf3, 0x54, 0x66, 0x9e, 0x3a, 0xd6, 0xc0, - 0x5e, 0xce, 0x38, 0x9b, 0x85, 0x89, 0x20, 0x74, 0x70, 0x1e, 0x5e, 0x61, 0xf0, 0xf1, 0x48, 0x95, - 0xc5, 0x9f, 0x80, 0x96, 0xd5, 0xa3, 0xf6, 0x03, 0x2c, 0xcf, 0xaf, 0xb2, 0xf3, 0x9b, 0x5c, 0x28, - 0x42, 0x38, 0x01, 0x2d, 0xc7, 0xb7, 0xfa, 0xb8, 0x6f, 0x6e, 0x38, 0x7e, 0xef, 0x1e, 0xe9, 0xd6, - 0x38, 0x88, 0x0b, 0x17, 0x99, 0x4c, 0xff, 0x0a, 0x26, 0x22, 0x0a, 0x96, 0x2f, 0x66, 0x49, 0x98, - 0x82, 0xd1, 0x90, 0xe0, 0xc0, 0xb4, 0xfb, 0xa2, 0x21, 0x6b, 0xd1, 0xe7, 0x72, 0x1f, 0x9d, 0x82, - 0x4a, 0xdf, 0xa2, 0x16, 0x4b, 0xb8, 0x31, 0x7f, 0x58, 0x5e, 0xf5, 0x36, 0x1a, 0x0d, 0x06, 0xd3, - 0x2f, 0x03, 0x8a, 0x54, 0x24, 0xeb, 0xfd, 0x0c, 0x54, 0x49, 0x24, 0x10, 0xef, 0xc7, 0x91, 0xb4, - 0x97, 0x5c, 0x24, 0x06, 0x47, 0xea, 0x8f, 0x15, 0xd0, 0x56, 0x30, 0x0d, 0xec, 0x1e, 0xb9, 0xe4, - 0x07, 0xd9, 0xca, 0x7a, 0xc3, 0x75, 0x7f, 0x16, 0x9a, 0xb2, 0x74, 0x4d, 0x82, 0xe9, 0xee, 0x0f, - 0x74, 0x43, 0x42, 0xd7, 0x30, 0x4d, 0x3a, 0xa6, 0x92, 0x7e, 0x2f, 0xae, 0xc1, 0xf4, 0x8e, 0x99, - 0x08, 0x82, 0x66, 0xa0, 0xe6, 0x32, 0x88, 0x60, 0xa8, 0x93, 0xbc, 0xb0, 0xdc, 0xd4, 0x10, 0x7a, - 0xfd, 0x16, 0x9c, 0xdc, 0xc1, 0x59, 0xae, 0x43, 0xf6, 0xef, 0xb2, 0x0b, 0x87, 0x84, 0xcb, 0x15, - 0x4c, 0xad, 0xe8, 0x1a, 0x65, 0xc3, 0xac, 0xc2, 0xd4, 0x36, 0x8d, 0x70, 0xff, 0x3e, 0x8c, 0xb9, - 0x42, 0x26, 0x0e, 0xe8, 0xe6, 0x0f, 0x88, 0x6d, 0x62, 0xa4, 0xfe, 0xb7, 0x02, 0x07, 0x72, 0x33, - 0x29, 0xba, 0x98, 0xbb, 0x81, 0xef, 0x9a, 0x72, 0xa9, 0x4a, 0x6a, 0xb0, 0x1d, 0xc9, 0x97, 0x85, - 0x78, 0xb9, 0x9f, 0x2e, 0xd2, 0x91, 0x4c, 0x91, 0x7a, 0x50, 0x63, 0xad, 0x2f, 0x87, 0xe9, 0x44, - 0x12, 0x0a, 0xa3, 0xe8, 0xa6, 0x65, 0x07, 0x8b, 0x0b, 0xd1, 0x7c, 0xfa, 0xfd, 0xd9, 0xf4, 0x2b, - 0xed, 0x63, 0xdc, 0x7e, 0xa1, 0x6f, 0x0d, 0x28, 0x0e, 0x0c, 0x71, 0x0a, 0x7a, 0x07, 0x6a, 0x7c, - 0x84, 0x76, 0x2b, 0xec, 0xbc, 0x96, 0xac, 0x8d, 0xf4, 0x94, 0x15, 0x10, 0xfd, 0x07, 0x05, 0xaa, - 0x3c, 0xd3, 0x37, 0x55, 0xb0, 0x2a, 0x8c, 0x61, 0xaf, 0xe7, 0xf7, 0x6d, 0x6f, 0x93, 0xbd, 0x38, - 0x55, 0x23, 0xfe, 0x46, 0x48, 0xf4, 0x6f, 0x54, 0x91, 0x4d, 0xd1, 0xa4, 0x0b, 0xd0, 0xca, 0x54, - 0x4e, 0x66, 0x63, 0x52, 0xf6, 0xb5, 0x31, 0x99, 0xd0, 0x4c, 0x6b, 0xd0, 0x49, 0xa8, 0xd0, 0x47, - 0x03, 0xfe, 0x74, 0xb6, 0xe7, 0xc7, 0xa5, 0x35, 0x53, 0xaf, 0x3f, 0x1a, 0x60, 0x83, 0xa9, 0xa3, - 0x68, 0xd8, 0xd0, 0xe7, 0xd7, 0xc7, 0x7e, 0x47, 0x4d, 0xc3, 0x26, 0x1e, 0x0b, 0xbd, 0x6e, 0xf0, - 0x0f, 0xfd, 0x7b, 0x05, 0xda, 0x49, 0xa5, 0x5c, 0xb2, 0x1d, 0xfc, 0x3a, 0x0a, 0x45, 0x85, 0xb1, - 0xbb, 0xb6, 0x83, 0x59, 0x0c, 0xfc, 0xb8, 0xf8, 0xbb, 0x88, 0xa9, 0xb7, 0xaf, 0x42, 0x3d, 0x4e, - 0x01, 0xd5, 0xa1, 0xba, 0x74, 0xeb, 0xf6, 0xc2, 0xf5, 0x4e, 0x09, 0xb5, 0xa0, 0x7e, 0x63, 0x75, - 0xdd, 0xe4, 0x9f, 0x0a, 0x3a, 0x00, 0x0d, 0x63, 0xe9, 0xf2, 0xd2, 0xe7, 0xe6, 0xca, 0xc2, 0xfa, - 0x85, 0x2b, 0x9d, 0x11, 0x84, 0xa0, 0xcd, 0x05, 0x37, 0x56, 0x85, 0xac, 0x3c, 0xff, 0xe7, 0x28, - 0x8c, 0xc9, 0x18, 0xd1, 0x39, 0xa8, 0xdc, 0x0c, 0xc9, 0x16, 0x3a, 0x94, 0x54, 0xea, 0x67, 0x81, - 0x4d, 0xb1, 0xe8, 0x3c, 0x75, 0x6a, 0x9b, 0x9c, 0xf7, 0x9d, 0x5e, 0x42, 0x17, 0xa1, 0x91, 0x5a, - 0x04, 0x51, 0xe1, 0x7f, 0x00, 0xf5, 0x48, 0x46, 0x9a, 0x7d, 0x1a, 0xf4, 0xd2, 0x69, 0x05, 0xad, - 0x42, 0x9b, 0xa9, 0xe4, 0xd6, 0x47, 0xd0, 0xff, 0xa4, 0x49, 0xd1, 0x26, 0xac, 0x1e, 0xdd, 0x41, - 0x1b, 0x87, 0x75, 0x05, 0x1a, 0xa9, 0xdd, 0x06, 0xa9, 0x99, 0x02, 0xca, 0x2c, 0x80, 0x49, 0x70, - 0x05, 0x6b, 0x94, 0x5e, 0x42, 0x77, 0xc4, 0x92, 0x93, 0xde, 0x92, 0x76, 0xf5, 0x77, 0xbc, 0x40, - 0x57, 0x90, 0xf2, 0x12, 0x40, 0xb2, 0x4f, 0xa0, 0xc3, 0x19, 0xa3, 0xf4, 0x42, 0xa5, 0xaa, 0x45, - 0xaa, 0x38, 0xbc, 0x35, 0xe8, 0xe4, 0xd7, 0x92, 0xdd, 0x9c, 0x1d, 0xdb, 0xae, 0x2a, 0x88, 0x6d, - 0x11, 0xea, 0xf1, 0x48, 0x45, 0xdd, 0x82, 0x29, 0xcb, 0x9d, 0xed, 0x3c, 0x7f, 0xf5, 0x12, 0xba, - 0x04, 0xcd, 0x05, 0xc7, 0xd9, 0x8f, 0x1b, 0x35, 0xad, 0x21, 0x79, 0x3f, 0x4e, 0xfc, 0xea, 0xe7, - 0x47, 0x0c, 0xfa, 0x7f, 0xdc, 0xd8, 0xbb, 0x8e, 0x66, 0xf5, 0xad, 0x3d, 0x71, 0xf1, 0x69, 0xdf, - 0xc0, 0xd1, 0x5d, 0x07, 0xda, 0xbe, 0xcf, 0x3c, 0xb5, 0x07, 0xae, 0x80, 0xf5, 0x75, 0x38, 0x90, - 0x9b, 0x6f, 0x48, 0xcb, 0x79, 0xc9, 0x8d, 0x44, 0x75, 0x7a, 0x47, 0xbd, 0xf4, 0xbb, 0xf8, 0xf1, - 0x93, 0xe7, 0x5a, 0xe9, 0xe9, 0x73, 0xad, 0xf4, 0xf2, 0xb9, 0xa6, 0x7c, 0x3b, 0xd4, 0x94, 0x5f, - 0x86, 0x9a, 0xf2, 0x78, 0xa8, 0x29, 0x4f, 0x86, 0x9a, 0xf2, 0xc7, 0x50, 0x53, 0xfe, 0x1a, 0x6a, - 0xa5, 0x97, 0x43, 0x4d, 0xf9, 0xf1, 0x85, 0x56, 0x7a, 0xf2, 0x42, 0x2b, 0x3d, 0x7d, 0xa1, 0x95, - 0xbe, 0xa8, 0xf5, 0x1c, 0x1b, 0x7b, 0x74, 0xa3, 0xc6, 0xfe, 0xfa, 0xbf, 0xf7, 0x4f, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x84, 0xf7, 0x8d, 0x61, 0x65, 0x10, 0x00, 0x00, + // 1368 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4b, 0x73, 0x14, 0x55, + 0x14, 0x9e, 0xce, 0x3c, 0x98, 0x39, 0xf3, 0x60, 0x72, 0x13, 0xc8, 0xa4, 0x91, 0x0e, 0x34, 0x85, + 0xa6, 0x54, 0x26, 0x30, 0x6a, 0x15, 0xa8, 0x48, 0x25, 0x10, 0x20, 0x40, 0x08, 0x74, 0x02, 0x5a, + 0x96, 0x56, 0x57, 0x67, 0xe6, 0x92, 0xb4, 0xf4, 0x63, 0xe8, 0x7b, 0x27, 0x05, 0xae, 0xb4, 0xfc, + 0x01, 0xba, 0x74, 0xeb, 0x4a, 0x7f, 0x80, 0x3f, 0x82, 0x25, 0x0b, 0x17, 0x94, 0x0b, 0x4a, 0x86, + 0x8d, 0x4b, 0xfc, 0x07, 0x56, 0xdf, 0x47, 0xbf, 0x32, 0x79, 0x60, 0x81, 0xbb, 0xbe, 0xe7, 0x7c, + 0xe7, 0xdc, 0x73, 0xbf, 0x7b, 0xce, 0x3d, 0x67, 0x06, 0x1a, 0xb6, 0xb7, 0x81, 0x09, 0xc5, 0x41, + 0xbb, 0x1f, 0xf8, 0xd4, 0x47, 0xa5, 0xae, 0x1f, 0x50, 0xfc, 0x50, 0x9d, 0xdc, 0xf0, 0x37, 0x7c, + 0x26, 0x9a, 0x0b, 0xbf, 0xb8, 0x56, 0x3d, 0xb7, 0x61, 0xd3, 0xcd, 0xc1, 0x7a, 0xbb, 0xeb, 0xbb, + 0x73, 0x1c, 0xd8, 0x0f, 0xfc, 0x6f, 0x70, 0x97, 0x8a, 0xd5, 0x5c, 0xff, 0xfe, 0x86, 0x54, 0xac, + 0x8b, 0x0f, 0x61, 0x7a, 0xfe, 0x55, 0x4c, 0xb7, 0x3a, 0xe2, 0x73, 0xab, 0xc3, 0xcd, 0xf5, 0xf3, + 0x50, 0x35, 0xb0, 0xd5, 0x33, 0xf0, 0x83, 0x01, 0x26, 0x14, 0xb5, 0xe1, 0xc0, 0x83, 0x01, 0x0e, + 0x6c, 0x4c, 0x5a, 0xca, 0xb1, 0xfc, 0x6c, 0xb5, 0x33, 0xd9, 0x16, 0xbb, 0xdd, 0x1e, 0xe0, 0xe0, + 0x91, 0x80, 0x19, 0x12, 0xa4, 0x5f, 0x80, 0x1a, 0x37, 0x27, 0x7d, 0xdf, 0x23, 0x18, 0xcd, 0xc1, + 0x81, 0x00, 0x93, 0x81, 0x43, 0xa5, 0xfd, 0xa1, 0x8c, 0x3d, 0xc7, 0x19, 0x12, 0xa5, 0x5f, 0x87, + 0x7a, 0x4a, 0x83, 0x3e, 0x06, 0xa0, 0xb6, 0x8b, 0xc9, 0xa8, 0x20, 0xfa, 0xeb, 0xed, 0x35, 0xdb, + 0xc5, 0xab, 0x4c, 0xb7, 0x50, 0x78, 0xfc, 0x6c, 0x26, 0x67, 0x24, 0xd0, 0xfa, 0xcf, 0x0a, 0xd4, + 0x92, 0x71, 0xa2, 0xf7, 0x01, 0x11, 0x6a, 0x05, 0xd4, 0x64, 0x20, 0x6a, 0xb9, 0x7d, 0xd3, 0x0d, + 0x9d, 0x2a, 0xb3, 0x79, 0xa3, 0xc9, 0x34, 0x6b, 0x52, 0xb1, 0x4c, 0xd0, 0x2c, 0x34, 0xb1, 0xd7, + 0x4b, 0x63, 0xc7, 0x18, 0xb6, 0x81, 0xbd, 0x5e, 0x12, 0x79, 0x1a, 0xca, 0xae, 0x45, 0xbb, 0x9b, + 0x38, 0x20, 0xad, 0x7c, 0x9a, 0xa7, 0x1b, 0xd6, 0x3a, 0x76, 0x96, 0xb9, 0xd2, 0x88, 0x50, 0xfa, + 0x2f, 0x0a, 0x4c, 0x2e, 0x3e, 0xc4, 0x6e, 0xdf, 0xb1, 0x82, 0xff, 0x25, 0xc4, 0x33, 0xdb, 0x42, + 0x3c, 0x34, 0x2a, 0x44, 0x92, 0x88, 0xf1, 0x2b, 0x98, 0x60, 0xa1, 0xad, 0xd2, 0x00, 0x5b, 0x6e, + 0x74, 0x23, 0x17, 0xa0, 0xda, 0xdd, 0x1c, 0x78, 0xf7, 0x53, 0x57, 0x32, 0x25, 0x9d, 0xc5, 0x17, + 0x72, 0x31, 0x04, 0x89, 0x5b, 0x49, 0x5a, 0x5c, 0x2b, 0x94, 0xc7, 0x9a, 0x79, 0x7d, 0x15, 0x0e, + 0x65, 0x08, 0x78, 0x0d, 0x37, 0xfe, 0x87, 0x02, 0x88, 0x1d, 0xe7, 0xae, 0xe5, 0x0c, 0x30, 0x91, + 0xa4, 0x1e, 0x05, 0x70, 0x42, 0xa9, 0xe9, 0x59, 0x2e, 0x66, 0x64, 0x56, 0x8c, 0x0a, 0x93, 0xdc, + 0xb4, 0x5c, 0xbc, 0x03, 0xe7, 0x63, 0xaf, 0xc0, 0x79, 0x7e, 0x4f, 0xce, 0x0b, 0xc7, 0x94, 0x7d, + 0x70, 0x8e, 0x26, 0xa1, 0xe8, 0xd8, 0xae, 0x4d, 0x5b, 0x45, 0xe6, 0x91, 0x2f, 0xf4, 0xb3, 0x30, + 0x91, 0x3a, 0x95, 0x60, 0xea, 0x38, 0xd4, 0xf8, 0xb1, 0xb6, 0x98, 0x9c, 0x71, 0x55, 0x31, 0xaa, + 0x4e, 0x0c, 0xd5, 0x3f, 0x83, 0xe9, 0x84, 0x65, 0xe6, 0x26, 0xf7, 0x61, 0xff, 0xbb, 0x02, 0xe3, + 0x37, 0x24, 0x51, 0xe4, 0x4d, 0x27, 0x69, 0x74, 0xfa, 0x7c, 0xe2, 0xf4, 0xff, 0x81, 0x46, 0xfd, + 0x23, 0x91, 0x06, 0x22, 0x6a, 0x71, 0xde, 0x19, 0xa8, 0xc6, 0x69, 0x20, 0x8f, 0x0b, 0x51, 0x1e, + 0x10, 0xfd, 0x13, 0x68, 0xc5, 0x66, 0x19, 0xb2, 0xf6, 0x34, 0x46, 0xd0, 0xbc, 0x43, 0x70, 0xb0, + 0x4a, 0x2d, 0x2a, 0x89, 0xd2, 0xbf, 0x1f, 0x83, 0xf1, 0x84, 0x50, 0xb8, 0x3a, 0x29, 0xdb, 0x81, + 0xed, 0x7b, 0x66, 0x60, 0x51, 0x9e, 0x92, 0x8a, 0x51, 0x8f, 0xa4, 0x86, 0x45, 0x71, 0x98, 0xb5, + 0xde, 0xc0, 0x35, 0x45, 0x21, 0x84, 0x8c, 0x15, 0x8c, 0x8a, 0x37, 0x70, 0x79, 0xf6, 0x87, 0x97, + 0x60, 0xf5, 0x6d, 0x33, 0xe3, 0x29, 0xcf, 0x3c, 0x35, 0xad, 0xbe, 0xbd, 0x94, 0x72, 0xd6, 0x86, + 0x89, 0x60, 0xe0, 0xe0, 0x2c, 0xbc, 0xc0, 0xe0, 0xe3, 0xa1, 0x2a, 0x8d, 0x3f, 0x01, 0x75, 0xab, + 0x4b, 0xed, 0x2d, 0x2c, 0xf7, 0x2f, 0xb2, 0xfd, 0x6b, 0x5c, 0x28, 0x42, 0x38, 0x01, 0x75, 0xc7, + 0xb7, 0x7a, 0xb8, 0x67, 0xae, 0x3b, 0x7e, 0xf7, 0x3e, 0x69, 0x95, 0x38, 0x88, 0x0b, 0x17, 0x98, + 0x4c, 0xff, 0x1a, 0x26, 0x42, 0x0a, 0x96, 0x2e, 0xa5, 0x49, 0x98, 0x82, 0x03, 0x03, 0x82, 0x03, + 0xd3, 0xee, 0x89, 0x82, 0x2c, 0x85, 0xcb, 0xa5, 0x1e, 0x3a, 0x05, 0x85, 0x9e, 0x45, 0x2d, 0x76, + 0xe0, 0x6a, 0x67, 0x5a, 0x5e, 0xf5, 0x36, 0x1a, 0x0d, 0x06, 0xd3, 0xaf, 0x00, 0x0a, 0x55, 0x24, + 0xed, 0xfd, 0x0c, 0x14, 0x49, 0x28, 0x10, 0xef, 0xc7, 0x91, 0xa4, 0x97, 0x4c, 0x24, 0x06, 0x47, + 0xea, 0x8f, 0x15, 0xd0, 0x96, 0x31, 0x0d, 0xec, 0x2e, 0xb9, 0xec, 0x07, 0xe9, 0xcc, 0x7a, 0xc3, + 0x79, 0x7f, 0x16, 0x6a, 0x32, 0x75, 0x4d, 0x82, 0xe9, 0xee, 0x0f, 0x74, 0x55, 0x42, 0x57, 0x31, + 0x8d, 0x2b, 0xa6, 0x90, 0x7c, 0x2f, 0xae, 0xc3, 0xcc, 0x8e, 0x27, 0x11, 0x04, 0xcd, 0x42, 0xc9, + 0x65, 0x10, 0xc1, 0x50, 0x33, 0x7e, 0x61, 0xb9, 0xa9, 0x21, 0xf4, 0xfa, 0x6d, 0x38, 0xb9, 0x83, + 0xb3, 0x4c, 0x85, 0xec, 0xdf, 0x65, 0x0b, 0x0e, 0x0b, 0x97, 0xcb, 0x98, 0x5a, 0xe1, 0x35, 0xca, + 0x82, 0x59, 0x81, 0xa9, 0x6d, 0x1a, 0xe1, 0xfe, 0x43, 0x28, 0xbb, 0x42, 0x26, 0x36, 0x68, 0x65, + 0x37, 0x88, 0x6c, 0x22, 0xa4, 0xfe, 0x8f, 0x02, 0x07, 0x33, 0x3d, 0x29, 0xbc, 0x98, 0x7b, 0x81, + 0xef, 0x9a, 0x72, 0x26, 0x8b, 0x73, 0xb0, 0x11, 0xca, 0x97, 0x84, 0x78, 0xa9, 0x97, 0x4c, 0xd2, + 0xb1, 0x54, 0x92, 0x7a, 0x50, 0x62, 0xa5, 0x2f, 0x9b, 0xe9, 0x44, 0x1c, 0x0a, 0xa3, 0xe8, 0x96, + 0x65, 0x07, 0x0b, 0xf3, 0x61, 0x7f, 0xfa, 0xf3, 0xd9, 0xcc, 0x2b, 0x8d, 0x73, 0xdc, 0x7e, 0xbe, + 0x67, 0xf5, 0x29, 0x0e, 0x0c, 0xb1, 0x0b, 0x7a, 0x0f, 0x4a, 0xbc, 0x85, 0xb6, 0x0a, 0x6c, 0xbf, + 0xba, 0xcc, 0x8d, 0x64, 0x97, 0x15, 0x10, 0xfd, 0x47, 0x05, 0x8a, 0xfc, 0xa4, 0x6f, 0x2a, 0x61, + 0x55, 0x28, 0x63, 0xaf, 0xeb, 0xf7, 0x6c, 0x6f, 0x83, 0xbd, 0x38, 0x45, 0x23, 0x5a, 0x23, 0x24, + 0xea, 0x37, 0xcc, 0xc8, 0x9a, 0x28, 0xd2, 0x79, 0xa8, 0xa7, 0x32, 0x27, 0x35, 0x31, 0x29, 0xfb, + 0x9a, 0x98, 0x4c, 0xa8, 0x25, 0x35, 0xe8, 0x24, 0x14, 0xe8, 0xa3, 0x3e, 0x7f, 0x3a, 0x1b, 0x9d, + 0x71, 0x69, 0xcd, 0xd4, 0x6b, 0x8f, 0xfa, 0xd8, 0x60, 0xea, 0x30, 0x1a, 0xd6, 0xf4, 0xf9, 0xf5, + 0xb1, 0xef, 0xb0, 0x68, 0x58, 0xc7, 0x63, 0xa1, 0x57, 0x0c, 0xbe, 0xd0, 0x7f, 0x50, 0xa0, 0x11, + 0x67, 0xca, 0x65, 0xdb, 0xc1, 0xaf, 0x23, 0x51, 0x54, 0x28, 0xdf, 0xb3, 0x1d, 0xcc, 0x62, 0xe0, + 0xdb, 0x45, 0xeb, 0x51, 0x4c, 0xbd, 0x7b, 0x0d, 0x2a, 0xd1, 0x11, 0x50, 0x05, 0x8a, 0x8b, 0xb7, + 0xef, 0xcc, 0xdf, 0x68, 0xe6, 0x50, 0x1d, 0x2a, 0x37, 0x57, 0xd6, 0x4c, 0xbe, 0x54, 0xd0, 0x41, + 0xa8, 0x1a, 0x8b, 0x57, 0x16, 0xbf, 0x30, 0x97, 0xe7, 0xd7, 0x2e, 0x5e, 0x6d, 0x8e, 0x21, 0x04, + 0x0d, 0x2e, 0xb8, 0xb9, 0x22, 0x64, 0xf9, 0xce, 0xaf, 0x65, 0x28, 0xcb, 0x18, 0xd1, 0x39, 0x28, + 0xdc, 0x1a, 0x90, 0x4d, 0x74, 0x38, 0xce, 0xd4, 0xcf, 0x03, 0x9b, 0x62, 0x51, 0x79, 0xea, 0xd4, + 0x36, 0x39, 0xaf, 0x3b, 0x3d, 0x87, 0x2e, 0x40, 0x29, 0x34, 0xbd, 0xdb, 0x41, 0x71, 0xc5, 0x6d, + 0x75, 0xd2, 0xe6, 0xd3, 0x23, 0x34, 0x91, 0x83, 0x4b, 0x50, 0x4d, 0x4c, 0x92, 0x68, 0xe4, 0x8f, + 0x08, 0xf5, 0x48, 0x4a, 0x9a, 0x7e, 0x5b, 0xf4, 0xdc, 0x69, 0x05, 0xad, 0x40, 0x83, 0xa9, 0xe4, + 0xd8, 0x48, 0xd0, 0x5b, 0xd2, 0x64, 0xd4, 0x28, 0xad, 0x1e, 0xdd, 0x41, 0x1b, 0x85, 0x75, 0x15, + 0xaa, 0x89, 0xe1, 0x08, 0xa9, 0xa9, 0x0c, 0x4c, 0x4d, 0x90, 0x71, 0x70, 0x23, 0xe6, 0x30, 0x3d, + 0x87, 0xee, 0x8a, 0x29, 0x29, 0x39, 0x66, 0xed, 0xea, 0xef, 0xf8, 0x08, 0xdd, 0x88, 0x23, 0x2f, + 0x02, 0xc4, 0x03, 0x09, 0x9a, 0x4e, 0x19, 0x25, 0x27, 0x32, 0x55, 0x1d, 0xa5, 0x8a, 0xc2, 0x5b, + 0x85, 0x66, 0x76, 0xae, 0xd9, 0xcd, 0xd9, 0xb1, 0xed, 0xaa, 0x11, 0xb1, 0x2d, 0x40, 0x25, 0xea, + 0xc9, 0x51, 0x62, 0xb4, 0xb3, 0x23, 0x90, 0xba, 0x73, 0x03, 0xd7, 0x73, 0xe8, 0x32, 0xd4, 0xe6, + 0x1d, 0x67, 0x3f, 0x6e, 0xd4, 0xa4, 0x86, 0x64, 0xfd, 0x38, 0x51, 0xdb, 0xc8, 0xf6, 0x28, 0xf4, + 0x76, 0xf4, 0x32, 0xec, 0xda, 0xdb, 0xd5, 0x77, 0xf6, 0xc4, 0x45, 0xbb, 0x7d, 0x0b, 0x47, 0x77, + 0xed, 0x88, 0xfb, 0xde, 0xf3, 0xd4, 0x1e, 0xb8, 0x11, 0xac, 0xaf, 0xc1, 0xc1, 0x4c, 0x83, 0x44, + 0x5a, 0xc6, 0x4b, 0xa6, 0xa7, 0xaa, 0x33, 0x3b, 0xea, 0xa5, 0xdf, 0x85, 0x4f, 0x9f, 0x3c, 0xd7, + 0x72, 0x4f, 0x9f, 0x6b, 0xb9, 0x97, 0xcf, 0x35, 0xe5, 0xbb, 0xa1, 0xa6, 0xfc, 0x36, 0xd4, 0x94, + 0xc7, 0x43, 0x4d, 0x79, 0x32, 0xd4, 0x94, 0xbf, 0x86, 0x9a, 0xf2, 0xf7, 0x50, 0xcb, 0xbd, 0x1c, + 0x6a, 0xca, 0x4f, 0x2f, 0xb4, 0xdc, 0x93, 0x17, 0x5a, 0xee, 0xe9, 0x0b, 0x2d, 0xf7, 0x65, 0xa9, + 0xeb, 0xd8, 0xd8, 0xa3, 0xeb, 0x25, 0xf6, 0xdf, 0xc1, 0x07, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, + 0x63, 0x80, 0x72, 0xdc, 0xe5, 0x10, 0x00, 0x00, } func (x MatchType) String() string { @@ -2779,6 +2782,7 @@ const _ = grpc.SupportPackageIsVersion4 // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type IngesterClient interface { Push(ctx context.Context, in *cortexpb.WriteRequest, opts ...grpc.CallOption) (*cortexpb.WriteResponse, error) + PushV2(ctx context.Context, in *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) QueryStream(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (Ingester_QueryStreamClient, error) QueryExemplars(ctx context.Context, in *ExemplarQueryRequest, opts ...grpc.CallOption) (*ExemplarQueryResponse, error) LabelValues(ctx context.Context, in *LabelValuesRequest, opts ...grpc.CallOption) (*LabelValuesResponse, error) @@ -2809,6 +2813,15 @@ func (c *ingesterClient) Push(ctx context.Context, in *cortexpb.WriteRequest, op return out, nil } +func (c *ingesterClient) PushV2(ctx context.Context, in *cortexpbv2.WriteRequest, opts ...grpc.CallOption) (*cortexpbv2.WriteResponse, error) { + out := new(cortexpbv2.WriteResponse) + err := c.cc.Invoke(ctx, "/cortex.Ingester/PushV2", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *ingesterClient) QueryStream(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (Ingester_QueryStreamClient, error) { stream, err := c.cc.NewStream(ctx, &_Ingester_serviceDesc.Streams[0], "/cortex.Ingester/QueryStream", opts...) if err != nil { @@ -3003,6 +3016,7 @@ func (c *ingesterClient) MetricsMetadata(ctx context.Context, in *MetricsMetadat // IngesterServer is the server API for Ingester service. type IngesterServer interface { Push(context.Context, *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) + PushV2(context.Context, *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) QueryStream(*QueryRequest, Ingester_QueryStreamServer) error QueryExemplars(context.Context, *ExemplarQueryRequest) (*ExemplarQueryResponse, error) LabelValues(context.Context, *LabelValuesRequest) (*LabelValuesResponse, error) @@ -3023,6 +3037,9 @@ type UnimplementedIngesterServer struct { func (*UnimplementedIngesterServer) Push(ctx context.Context, req *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Push not implemented") } +func (*UnimplementedIngesterServer) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PushV2 not implemented") +} func (*UnimplementedIngesterServer) QueryStream(req *QueryRequest, srv Ingester_QueryStreamServer) error { return status.Errorf(codes.Unimplemented, "method QueryStream not implemented") } @@ -3079,6 +3096,24 @@ func _Ingester_Push_Handler(srv interface{}, ctx context.Context, dec func(inter return interceptor(ctx, in, info, handler) } +func _Ingester_PushV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(cortexpbv2.WriteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IngesterServer).PushV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cortex.Ingester/PushV2", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IngesterServer).PushV2(ctx, req.(*cortexpbv2.WriteRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Ingester_QueryStream_Handler(srv interface{}, stream grpc.ServerStream) error { m := new(QueryRequest) if err := stream.RecvMsg(m); err != nil { @@ -3297,6 +3332,10 @@ var _Ingester_serviceDesc = grpc.ServiceDesc{ MethodName: "Push", Handler: _Ingester_Push_Handler, }, + { + MethodName: "PushV2", + Handler: _Ingester_PushV2_Handler, + }, { MethodName: "QueryExemplars", Handler: _Ingester_QueryExemplars_Handler, diff --git a/pkg/ingester/client/ingester.proto b/pkg/ingester/client/ingester.proto index 68f343693e..01f7be6f83 100644 --- a/pkg/ingester/client/ingester.proto +++ b/pkg/ingester/client/ingester.proto @@ -7,12 +7,15 @@ option go_package = "client"; import "gogoproto/gogo.proto"; import "github.com/cortexproject/cortex/pkg/cortexpb/cortex.proto"; +import "github.com/cortexproject/cortex/pkg/cortexpbv2/cortexv2.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.unmarshaler_all) = true; service Ingester { rpc Push(cortexpb.WriteRequest) returns (cortexpb.WriteResponse) {}; + rpc PushV2(cortexpbv2.WriteRequest) returns (cortexpbv2.WriteResponse) {}; + rpc QueryStream(QueryRequest) returns (stream QueryStreamResponse) {}; rpc QueryExemplars(ExemplarQueryRequest) returns (ExemplarQueryResponse) {}; diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 00dd1337ce..e8891c6a59 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -44,6 +44,7 @@ import ( "github.com/cortexproject/cortex/pkg/chunk/encoding" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/querysharding" "github.com/cortexproject/cortex/pkg/ring" @@ -1056,6 +1057,375 @@ type extendedAppender interface { storage.GetRef } +func (i *Ingester) PushV2(ctx context.Context, req *cortexpbv2.WriteRequest) (*cortexpbv2.WriteResponse, error) { + if err := i.checkRunning(); err != nil { + return nil, err + } + + span, ctx := opentracing.StartSpanFromContext(ctx, "Ingester.PushV2") + defer span.Finish() + + // We will report *this* request in the error too. + inflight := i.inflightPushRequests.Inc() + defer i.inflightPushRequests.Dec() + + gl := i.getInstanceLimits() + if gl != nil && gl.MaxInflightPushRequests > 0 { + if inflight > gl.MaxInflightPushRequests { + return nil, errTooManyInflightPushRequests + } + } + + var firstPartialErr error + + // NOTE: because we use `unsafe` in deserialisation, we must not + // retain anything from `req` past the call to ReuseSlice + defer cortexpbv2.ReuseSlice(req.Timeseries) + + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + il := i.getInstanceLimits() + if il != nil && il.MaxIngestionRate > 0 { + if rate := i.ingestionRate.Rate(); rate >= il.MaxIngestionRate { + return nil, errMaxSamplesPushRateLimitReached + } + } + + db, err := i.getOrCreateTSDB(userID, false) + if err != nil { + return nil, wrapWithUser(err, userID) + } + + // Ensure the ingester shutdown procedure hasn't started + i.stoppedMtx.RLock() + if i.stopped { + i.stoppedMtx.RUnlock() + return nil, errIngesterStopping + } + i.stoppedMtx.RUnlock() + + if err := db.acquireAppendLock(); err != nil { + return &cortexpbv2.WriteResponse{}, httpgrpc.Errorf(http.StatusServiceUnavailable, wrapWithUser(err, userID).Error()) + } + defer db.releaseAppendLock() + + // Keep track of some stats which are tracked only if the samples will be + // successfully committed + var ( + succeededSamplesCount = 0 + failedSamplesCount = 0 + succeededExemplarsCount = 0 + succeededMetadataCount = 0 + failedMetadataCount = 0 + failedExemplarsCount = 0 + startAppend = time.Now() + sampleOutOfBoundsCount = 0 + sampleOutOfOrderCount = 0 + sampleTooOldCount = 0 + newValueForTimestampCount = 0 + perUserSeriesLimitCount = 0 + perLabelSetSeriesLimitCount = 0 + perMetricSeriesLimitCount = 0 + nativeHistogramCount = 0 + succeededHistogramCount = 0 + + updateFirstPartial = func(errFn func() error) { + if firstPartialErr == nil { + firstPartialErr = errFn() + } + } + + handleAppendFailure = func(err error, timestampMs int64, lbls []cortexpb.LabelAdapter, copiedLabels labels.Labels) (rollback bool) { + // Check if the error is a soft error we can proceed on. If so, we keep track + // of it, so that we can return it back to the distributor, which will return a + // 400 error to the client. The client (Prometheus) will not retry on 400, and + // we actually ingested all samples which haven't failed. + switch cause := errors.Cause(err); { + case errors.Is(cause, storage.ErrOutOfBounds): + sampleOutOfBoundsCount++ + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, storage.ErrOutOfOrderSample): + sampleOutOfOrderCount++ + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, storage.ErrDuplicateSampleForTimestamp): + newValueForTimestampCount++ + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, storage.ErrTooOldSample): + sampleTooOldCount++ + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, errMaxSeriesPerUserLimitExceeded): + perUserSeriesLimitCount++ + updateFirstPartial(func() error { return makeLimitError(perUserSeriesLimit, i.limiter.FormatError(userID, cause)) }) + + case errors.Is(cause, errMaxSeriesPerMetricLimitExceeded): + perMetricSeriesLimitCount++ + updateFirstPartial(func() error { + return makeMetricLimitError(perMetricSeriesLimit, copiedLabels, i.limiter.FormatError(userID, cause)) + }) + + case errors.As(cause, &errMaxSeriesPerLabelSetLimitExceeded{}): + perLabelSetSeriesLimitCount++ + updateFirstPartial(func() error { + return makeMetricLimitError(perLabelsetSeriesLimit, copiedLabels, i.limiter.FormatError(userID, cause)) + }) + + case errors.Is(cause, histogram.ErrHistogramSpanNegativeOffset): + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, histogram.ErrHistogramSpansBucketsMismatch): + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, histogram.ErrHistogramNegativeBucketCount): + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, histogram.ErrHistogramCountNotBigEnough): + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + case errors.Is(cause, histogram.ErrHistogramCountMismatch): + updateFirstPartial(func() error { return wrappedTSDBIngestErr(err, model.Time(timestampMs), lbls) }) + + default: + rollback = true + } + return + } + ) + + // Walk the samples, appending them to the users database + app := db.Appender(ctx).(extendedAppender) + b := labels.NewScratchBuilder(0) + for _, ts := range req.Timeseries { + tsLabels := ts.ToLabels(&b, req.Symbols) + seriesLabels := cortexpb.FromLabelsToLabelAdapters(tsLabels) + + // The labels must be sorted (in our case, it's guaranteed a write request + // has sorted labels once hit the ingester). + + // Look up a reference for this series. + tsLabelsHash := tsLabels.Hash() + ref, copiedLabels := app.GetRef(tsLabels, tsLabelsHash) + + // To find out if any sample was added to this series, we keep old value. + oldSucceededSamplesCount := succeededSamplesCount + + for _, s := range ts.Samples { + var err error + + // If the cached reference exists, we try to use it. + if ref != 0 { + if _, err = app.Append(ref, copiedLabels, s.Timestamp, s.Value); err == nil { + succeededSamplesCount++ + continue + } + + } else { + // Copy the label set because both TSDB and the active series tracker may retain it. + copiedLabels = cortexpb.FromLabelAdaptersToLabelsWithCopy(seriesLabels) + + // Retain the reference in case there are multiple samples for the series. + if ref, err = app.Append(0, copiedLabels, s.Timestamp, s.Value); err == nil { + succeededSamplesCount++ + continue + } + } + + failedSamplesCount++ + + if rollback := handleAppendFailure(err, s.Timestamp, seriesLabels, copiedLabels); !rollback { + continue + } + // The error looks an issue on our side, so we should rollback + if rollbackErr := app.Rollback(); rollbackErr != nil { + level.Warn(logutil.WithContext(ctx, i.logger)).Log("msg", "failed to rollback on error", "user", userID, "err", rollbackErr) + } + + return nil, wrapWithUser(err, userID) + } + + if i.cfg.BlocksStorageConfig.TSDB.EnableNativeHistograms { + for _, hp := range ts.Histograms { + var ( + err error + h *histogram.Histogram + fh *histogram.FloatHistogram + ) + + if hp.GetCountFloat() > 0 { + fh = cortexpbv2.FloatHistogramProtoToFloatHistogram(hp) + } else { + h = cortexpbv2.HistogramProtoToHistogram(hp) + } + + if ref != 0 { + if _, err = app.AppendHistogram(ref, copiedLabels, hp.Timestamp, h, fh); err == nil { + succeededHistogramCount++ + succeededSamplesCount++ + continue + } + } else { + // Copy the label set because both TSDB and the active series tracker may retain it. + copiedLabels = cortexpb.FromLabelAdaptersToLabelsWithCopy(seriesLabels) + if ref, err = app.AppendHistogram(0, copiedLabels, hp.Timestamp, h, fh); err == nil { + succeededHistogramCount++ + succeededSamplesCount++ + continue + } + } + + failedSamplesCount++ + + if rollback := handleAppendFailure(err, hp.Timestamp, seriesLabels, copiedLabels); !rollback { + continue + } + // The error looks an issue on our side, so we should rollback + if rollbackErr := app.Rollback(); rollbackErr != nil { + level.Warn(logutil.WithContext(ctx, i.logger)).Log("msg", "failed to rollback on error", "user", userID, "err", rollbackErr) + } + return nil, wrapWithUser(err, userID) + } + } else { + nativeHistogramCount += len(ts.Histograms) + } + + if i.cfg.ActiveSeriesMetricsEnabled && succeededSamplesCount > oldSucceededSamplesCount { + db.activeSeries.UpdateSeries(tsLabels, tsLabelsHash, startAppend, func(l labels.Labels) labels.Labels { + // we must already have copied the labels if succeededSamplesCount has been incremented. + return copiedLabels + }) + } + + maxExemplarsForUser := i.getMaxExemplars(userID) + if maxExemplarsForUser > 0 { + // app.AppendExemplar currently doesn't create the series, it must + // already exist. If it does not then drop. + if ref == 0 && len(ts.Exemplars) > 0 { + updateFirstPartial(func() error { + return wrappedTSDBIngestExemplarErr(errExemplarRef, + model.Time(ts.Exemplars[0].Timestamp), seriesLabels, cortexpb.FromLabelsToLabelAdapters(ts.Exemplars[0].ToLabels(&b, req.Symbols))) + }) + failedExemplarsCount += len(ts.Exemplars) + } else { // Note that else is explicit, rather than a continue in the above if, in case of additional logic post exemplar processing. + for _, ex := range ts.Exemplars { + exLabels := ex.ToLabels(&b, req.Symbols) + e := exemplar.Exemplar{ + Value: ex.Value, + Ts: ex.Timestamp, + HasTs: true, + Labels: exLabels, + } + + if _, err = app.AppendExemplar(ref, nil, e); err == nil { + succeededExemplarsCount++ + continue + } + + // Error adding exemplar + updateFirstPartial(func() error { + return wrappedTSDBIngestExemplarErr(err, model.Time(ex.Timestamp), seriesLabels, cortexpb.FromLabelsToLabelAdapters(exLabels)) + }) + failedExemplarsCount++ + } + } + } + + if ts.Metadata.Type != cortexpbv2.METRIC_TYPE_UNSPECIFIED { + metaData := ts.Metadata.ToV1Metadata(tsLabels.Get(model.MetricNameLabel), req.Symbols) + if err := i.appendMetadata(userID, metaData); err == nil { + succeededMetadataCount++ + } else { + level.Warn(i.logger).Log("msg", "failed to ingest metadata", "err", err) + failedMetadataCount++ + } + } + } + // At this point all samples have been added to the appender, so we can track the time it took. + i.TSDBState.appenderAddDuration.Observe(time.Since(startAppend).Seconds()) + + startCommit := time.Now() + if err := app.Commit(); err != nil { + return nil, wrapWithUser(err, userID) + } + i.TSDBState.appenderCommitDuration.Observe(time.Since(startCommit).Seconds()) + + // If only invalid samples are pushed, don't change "last update", as TSDB was not modified. + if succeededSamplesCount > 0 { + db.setLastUpdate(time.Now()) + } + + // Increment metrics only if the samples have been successfully committed. + // If the code didn't reach this point, it means that we returned an error + // which will be converted into an HTTP 5xx and the client should/will retry. + i.metrics.ingestedMetadata.Add(float64(succeededMetadataCount)) + i.metrics.ingestedMetadataFail.Add(float64(failedMetadataCount)) + i.metrics.ingestedSamples.Add(float64(succeededSamplesCount)) + i.metrics.ingestedSamplesFail.Add(float64(failedSamplesCount)) + i.metrics.ingestedExemplars.Add(float64(succeededExemplarsCount)) + i.metrics.ingestedExemplarsFail.Add(float64(failedExemplarsCount)) + + if sampleOutOfBoundsCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(sampleOutOfBounds, userID).Add(float64(sampleOutOfBoundsCount)) + } + if sampleOutOfOrderCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(sampleOutOfOrder, userID).Add(float64(sampleOutOfOrderCount)) + } + if sampleTooOldCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(sampleTooOld, userID).Add(float64(sampleTooOldCount)) + } + if newValueForTimestampCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(newValueForTimestamp, userID).Add(float64(newValueForTimestampCount)) + } + if perUserSeriesLimitCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(perUserSeriesLimit, userID).Add(float64(perUserSeriesLimitCount)) + } + if perMetricSeriesLimitCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(perMetricSeriesLimit, userID).Add(float64(perMetricSeriesLimitCount)) + } + if perLabelSetSeriesLimitCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(perLabelsetSeriesLimit, userID).Add(float64(perLabelSetSeriesLimitCount)) + } + + if !i.cfg.BlocksStorageConfig.TSDB.EnableNativeHistograms && nativeHistogramCount > 0 { + i.validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramSample, userID).Add(float64(nativeHistogramCount)) + } + + // Distributor counts both samples, metadata and histograms, so for consistency ingester does the same. + i.ingestionRate.Add(int64(succeededSamplesCount + succeededMetadataCount)) + + switch req.Source { + case cortexpbv2.RULE: + db.ingestedRuleSamples.Add(int64(succeededSamplesCount)) + case cortexpbv2.API: + fallthrough + default: + db.ingestedAPISamples.Add(int64(succeededSamplesCount)) + } + + if firstPartialErr != nil { + code := http.StatusBadRequest + var ve *validationError + if errors.As(firstPartialErr, &ve) { + code = ve.code + } + level.Debug(logutil.WithContext(ctx, i.logger)).Log("msg", "partial failures to push", "totalSamples", succeededSamplesCount+failedSamplesCount, "failedSamples", failedSamplesCount, "firstPartialErr", firstPartialErr) + return &cortexpbv2.WriteResponse{}, httpgrpc.Errorf(code, wrapWithUser(firstPartialErr, userID).Error()) + } + + writeResponse := &cortexpbv2.WriteResponse{ + Samples: int64(succeededSamplesCount), + Histograms: int64(succeededHistogramCount), + Exemplars: int64(succeededExemplarsCount), + } + + return writeResponse, nil +} + // Push adds metrics to a block func (i *Ingester) Push(ctx context.Context, req *cortexpb.WriteRequest) (*cortexpb.WriteResponse, error) { if err := i.checkRunning(); err != nil { diff --git a/pkg/ingester/ingester_prw2_test.go b/pkg/ingester/ingester_prw2_test.go new file mode 100644 index 0000000000..ac132527fb --- /dev/null +++ b/pkg/ingester/ingester_prw2_test.go @@ -0,0 +1,4468 @@ +package ingester + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "math" + "net" + "net/http" + "net/http/httptest" + "net/url" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/shipper" + "github.com/weaveworks/common/httpgrpc" + "github.com/weaveworks/common/middleware" + "github.com/weaveworks/common/user" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/encoding" + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" + "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/cortexproject/cortex/pkg/ring" + cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/services" + "github.com/cortexproject/cortex/pkg/util/test" + "github.com/cortexproject/cortex/pkg/util/validation" +) + +func TestIngesterPRW2_Push(t *testing.T) { + metricLabelAdapters := []cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}} + metricLabels := cortexpb.FromLabelAdaptersToLabels(metricLabelAdapters) + metricNames := []string{ + "cortex_ingester_ingested_samples_total", + "cortex_ingester_ingested_samples_failures_total", + "cortex_ingester_memory_series", + "cortex_ingester_memory_users", + "cortex_ingester_memory_series_created_total", + "cortex_ingester_memory_series_removed_total", + } + userID := "test" + + testHistogramV2 := cortexpbv2.HistogramToHistogramProto(10, tsdbutil.GenerateTestHistogram(1)) + testHistogram := cortexpb.HistogramToHistogramProto(10, tsdbutil.GenerateTestHistogram(1)) + testFloatHistogramV2 := cortexpbv2.FloatHistogramToHistogramProto(11, tsdbutil.GenerateTestFloatHistogram(1)) + testFloatHistogram := cortexpb.FloatHistogramToHistogramProto(11, tsdbutil.GenerateTestFloatHistogram(1)) + tests := map[string]struct { + reqs []*cortexpbv2.WriteRequest + expectedErr error + expectedIngested []cortexpb.TimeSeries + expectedMetadataIngested []*cortexpb.MetricMetadata + expectedExemplarsIngested []cortexpb.TimeSeries + expectedMetrics string + additionalMetrics []string + disableActiveSeries bool + maxExemplars int + oooTimeWindow time.Duration + disableNativeHistogram bool + }{ + "should record native histogram discarded": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + []cortexpbv2.Histogram{{Timestamp: 10}}, + []cortexpbv2.Metadata{{Type: cortexpbv2.METRIC_TYPE_GAUGE, HelpRef: 3}}, + cortexpbv2.API, + "a help for metric_name_2"), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 10}}}, + }, + expectedMetadataIngested: []*cortexpb.MetricMetadata{ + {MetricFamilyName: "test", Help: "a help for metric_name_2", Unit: "", Type: cortexpb.GAUGE}, + }, + additionalMetrics: []string{"cortex_discarded_samples_total", "cortex_ingester_active_series"}, + disableNativeHistogram: true, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="native-histogram-sample",user="test"} 1 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should succeed on valid series and metadata": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + []cortexpbv2.Metadata{{HelpRef: 3, Type: cortexpbv2.METRIC_TYPE_COUNTER}}, + cortexpbv2.API, + "a help for metric_name_1"), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + nil, + []cortexpbv2.Metadata{{HelpRef: 3, Type: cortexpbv2.METRIC_TYPE_GAUGE}}, + cortexpbv2.API, + "a help for metric_name_2"), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 9}, {Value: 2, TimestampMs: 10}}}, + }, + expectedMetadataIngested: []*cortexpb.MetricMetadata{ + {MetricFamilyName: "test", Help: "a help for metric_name_2", Unit: "", Type: cortexpb.GAUGE}, + {MetricFamilyName: "test", Help: "a help for metric_name_1", Unit: "", Type: cortexpb.COUNTER}, + }, + additionalMetrics: []string{ + // Metadata. + "cortex_ingester_memory_metadata", + "cortex_ingester_memory_metadata_created_total", + "cortex_ingester_ingested_metadata_total", + "cortex_ingester_ingested_metadata_failures_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_metadata_failures_total The total number of metadata that errored on ingestion. + # TYPE cortex_ingester_ingested_metadata_failures_total counter + cortex_ingester_ingested_metadata_failures_total 0 + # HELP cortex_ingester_ingested_metadata_total The total number of metadata ingested. + # TYPE cortex_ingester_ingested_metadata_total counter + cortex_ingester_ingested_metadata_total 2 + # HELP cortex_ingester_memory_metadata The current number of metadata in memory. + # TYPE cortex_ingester_memory_metadata gauge + cortex_ingester_memory_metadata 2 + # HELP cortex_ingester_memory_metadata_created_total The total number of metadata that were created per user + # TYPE cortex_ingester_memory_metadata_created_total counter + cortex_ingester_memory_metadata_created_total{user="test"} 2 + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 2 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should succeed on valid series with exemplars": { + maxExemplars: 2, + reqs: []*cortexpbv2.WriteRequest{ + // Ingesting an exemplar requires a sample to create the series first + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + { + Symbols: []string{"", "__name__", "test", "traceID", "123", "456"}, + Timeseries: []cortexpbv2.PreallocTimeseriesV2{ + { + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: []uint32{1, 2}, + Exemplars: []cortexpbv2.Exemplar{ + { + LabelsRefs: []uint32{3, 4}, + Timestamp: 1000, + Value: 1000, + }, + { + LabelsRefs: []uint32{3, 5}, + Timestamp: 1001, + Value: 1001, + }, + }, + }, + }, + }, + }, + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 9}}}, + }, + expectedExemplarsIngested: []cortexpb.TimeSeries{ + { + Labels: metricLabelAdapters, + Exemplars: []cortexpb.Exemplar{ + { + Labels: []cortexpb.LabelAdapter{{Name: "traceID", Value: "123"}}, + TimestampMs: 1000, + Value: 1000, + }, + { + Labels: []cortexpb.LabelAdapter{{Name: "traceID", Value: "456"}}, + TimestampMs: 1001, + Value: 1001, + }, + }, + }, + }, + expectedMetadataIngested: nil, + additionalMetrics: []string{ + "cortex_ingester_tsdb_exemplar_exemplars_appended_total", + "cortex_ingester_tsdb_exemplar_exemplars_in_storage", + "cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage", + "cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds", + "cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + + # HELP cortex_ingester_tsdb_exemplar_exemplars_appended_total Total number of TSDB exemplars appended. + # TYPE cortex_ingester_tsdb_exemplar_exemplars_appended_total counter + cortex_ingester_tsdb_exemplar_exemplars_appended_total 2 + + # HELP cortex_ingester_tsdb_exemplar_exemplars_in_storage Number of TSDB exemplars currently in storage. + # TYPE cortex_ingester_tsdb_exemplar_exemplars_in_storage gauge + cortex_ingester_tsdb_exemplar_exemplars_in_storage 2 + + # HELP cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage Number of TSDB series with exemplars currently in storage. + # TYPE cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage gauge + cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{user="test"} 1 + + # HELP cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds The timestamp of the oldest exemplar stored in circular storage. Useful to check for what time range the current exemplar buffer limit allows. This usually means the last timestamp for all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series. + # TYPE cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds gauge + cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds{user="test"} 1 + + # HELP cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total Total number of out of order exemplar ingestion failed attempts. + # TYPE cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total counter + cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total 0 + `, + }, + "successful push, active series disabled": { + disableActiveSeries: true, + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 9}, {Value: 2, TimestampMs: 10}}}, + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 2 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + `, + }, + "ooo disabled, should soft fail on sample out of order": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(9, tsdbutil.GenerateTestHistogram(1)), + }, + nil, + cortexpbv2.API), + }, + expectedErr: httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestErr(storage.ErrOutOfOrderSample, model.Time(9), cortexpb.FromLabelsToLabelAdapters(metricLabels)), userID).Error()), + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 10}}}, + }, + additionalMetrics: []string{ + "cortex_ingester_tsdb_out_of_order_samples_total", + "cortex_ingester_tsdb_head_out_of_order_samples_appended_total", + "cortex_discarded_samples_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 2 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_tsdb_head_out_of_order_samples_appended_total Total number of appended out of order samples. + # TYPE cortex_ingester_tsdb_head_out_of_order_samples_appended_total counter + cortex_ingester_tsdb_head_out_of_order_samples_appended_total{type="float",user="test"} 0 + # HELP cortex_ingester_tsdb_out_of_order_samples_total Total number of out of order samples ingestion failed attempts due to out of order being disabled. + # TYPE cortex_ingester_tsdb_out_of_order_samples_total counter + cortex_ingester_tsdb_out_of_order_samples_total{type="float",user="test"} 1 + cortex_ingester_tsdb_out_of_order_samples_total{type="histogram",user="test"} 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="sample-out-of-order",user="test"} 2 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "ooo disabled, should soft fail on sample out of bound": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 1575043969}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 1575043969 - (86400 * 1000)}}, + []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(1575043969-(86400*1000), tsdbutil.GenerateTestHistogram(1)), + }, + nil, + cortexpbv2.API), + }, + expectedErr: httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestErr(storage.ErrOutOfBounds, model.Time(1575043969-(86400*1000)), cortexpb.FromLabelsToLabelAdapters(metricLabels)), userID).Error()), + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 1575043969}}}, + }, + additionalMetrics: []string{"cortex_ingester_active_series"}, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 2 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="sample-out-of-bounds",user="test"} 2 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "ooo enabled, should soft fail on sample too old": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 1575043969}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 1575043969 - (600 * 1000)}}, + nil, + nil, + cortexpbv2.API), + }, + oooTimeWindow: 5 * time.Minute, + expectedErr: httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestErr(storage.ErrTooOldSample, model.Time(1575043969-(600*1000)), cortexpb.FromLabelsToLabelAdapters(metricLabels)), userID).Error()), + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 1575043969}}}, + }, + additionalMetrics: []string{ + "cortex_discarded_samples_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 1 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="sample-too-old",user="test"} 1 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "ooo enabled, should succeed": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 1575043969}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 1575043969 - (60 * 1000)}}, + nil, + nil, + cortexpbv2.API), + }, + oooTimeWindow: 5 * time.Minute, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 1575043969 - (60 * 1000)}, {Value: 2, TimestampMs: 1575043969}}}, + }, + additionalMetrics: []string{"cortex_ingester_active_series"}, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 2 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should soft fail on two different sample values at the same timestamp": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 1575043969}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 1575043969}}, + nil, + nil, + cortexpbv2.API), + }, + expectedErr: httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestErr(storage.NewDuplicateFloatErr(1575043969, 2, 1), model.Time(1575043969), cortexpb.FromLabelsToLabelAdapters(metricLabels)), userID).Error()), + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 1575043969}}}, + }, + additionalMetrics: []string{"cortex_discarded_samples_total", "cortex_ingester_active_series"}, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 1 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="new-value-for-timestamp",user="test"} 1 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should soft fail on exemplar with unknown series": { + maxExemplars: 1, + reqs: []*cortexpbv2.WriteRequest{ + // Ingesting an exemplar requires a sample to create the series first + // This is not done here. + { + Symbols: []string{"", "__name__", "test", "traceID", "123"}, + Timeseries: []cortexpbv2.PreallocTimeseriesV2{ + { + TimeSeries: &cortexpbv2.TimeSeries{ + LabelsRefs: []uint32{1, 2}, + Exemplars: []cortexpbv2.Exemplar{ + { + LabelsRefs: []uint32{3, 4}, + Timestamp: 1000, + Value: 1000, + }, + }, + }, + }, + }, + }, + }, + expectedErr: httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestExemplarErr(errExemplarRef, model.Time(1000), cortexpb.FromLabelsToLabelAdapters(metricLabels), []cortexpb.LabelAdapter{{Name: "traceID", Value: "123"}}), userID).Error()), + expectedIngested: nil, + expectedMetadataIngested: nil, + additionalMetrics: []string{ + "cortex_ingester_tsdb_exemplar_exemplars_appended_total", + "cortex_ingester_tsdb_exemplar_exemplars_in_storage", + "cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage", + "cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds", + "cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 0 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 0 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 0 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 0 + + # HELP cortex_ingester_tsdb_exemplar_exemplars_appended_total Total number of TSDB exemplars appended. + # TYPE cortex_ingester_tsdb_exemplar_exemplars_appended_total counter + cortex_ingester_tsdb_exemplar_exemplars_appended_total 0 + + # HELP cortex_ingester_tsdb_exemplar_exemplars_in_storage Number of TSDB exemplars currently in storage. + # TYPE cortex_ingester_tsdb_exemplar_exemplars_in_storage gauge + cortex_ingester_tsdb_exemplar_exemplars_in_storage 0 + + # HELP cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage Number of TSDB series with exemplars currently in storage. + # TYPE cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage gauge + cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{user="test"} 0 + + # HELP cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds The timestamp of the oldest exemplar stored in circular storage. Useful to check for what time range the current exemplar buffer limit allows. This usually means the last timestamp for all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series. + # TYPE cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds gauge + cortex_ingester_tsdb_exemplar_last_exemplars_timestamp_seconds{user="test"} 0 + + # HELP cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total Total number of out of order exemplar ingestion failed attempts. + # TYPE cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total counter + cortex_ingester_tsdb_exemplar_out_of_order_exemplars_total 0 + `, + }, + "should succeed when only native histogram present if enabled": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + nil, + []cortexpbv2.Histogram{testHistogramV2}, + nil, + cortexpbv2.API), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Histograms: []cortexpb.Histogram{testHistogram}}, + }, + additionalMetrics: []string{ + "cortex_ingester_tsdb_head_samples_appended_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_tsdb_head_out_of_order_samples_appended_total Total number of appended out of order samples. + # TYPE cortex_ingester_tsdb_head_out_of_order_samples_appended_total counter + cortex_ingester_tsdb_head_out_of_order_samples_appended_total{type="float",user="test"} 0 + # HELP cortex_ingester_tsdb_head_samples_appended_total Total number of appended samples. + # TYPE cortex_ingester_tsdb_head_samples_appended_total counter + cortex_ingester_tsdb_head_samples_appended_total{type="float",user="test"} 0 + cortex_ingester_tsdb_head_samples_appended_total{type="histogram",user="test"} 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should succeed when only float native histogram present if enabled": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + nil, + []cortexpbv2.Histogram{testFloatHistogramV2}, + nil, + cortexpbv2.API), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Histograms: []cortexpb.Histogram{testFloatHistogram}}, + }, + additionalMetrics: []string{ + "cortex_ingester_tsdb_head_samples_appended_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 1 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_tsdb_head_out_of_order_samples_appended_total Total number of appended out of order samples. + # TYPE cortex_ingester_tsdb_head_out_of_order_samples_appended_total counter + cortex_ingester_tsdb_head_out_of_order_samples_appended_total{type="float",user="test"} 0 + # HELP cortex_ingester_tsdb_head_samples_appended_total Total number of appended samples. + # TYPE cortex_ingester_tsdb_head_samples_appended_total counter + cortex_ingester_tsdb_head_samples_appended_total{type="float",user="test"} 0 + cortex_ingester_tsdb_head_samples_appended_total{type="histogram",user="test"} 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + "should fail to ingest histogram due to OOO native histogram. Sample and histogram has same timestamp but sample got ingested first": { + reqs: []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + []cortexpbv2.Histogram{testHistogramV2}, + nil, + cortexpbv2.API), + }, + expectedErr: nil, + expectedIngested: []cortexpb.TimeSeries{ + {Labels: metricLabelAdapters, Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 10}}}, + }, + additionalMetrics: []string{ + "cortex_ingester_tsdb_head_samples_appended_total", + "cortex_ingester_tsdb_out_of_order_samples_total", + "cortex_ingester_active_series", + }, + expectedMetrics: ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 2 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + # HELP cortex_ingester_tsdb_head_samples_appended_total Total number of appended samples. + # TYPE cortex_ingester_tsdb_head_samples_appended_total counter + cortex_ingester_tsdb_head_samples_appended_total{type="float",user="test"} 1 + cortex_ingester_tsdb_head_samples_appended_total{type="histogram",user="test"} 0 + # HELP cortex_ingester_tsdb_out_of_order_samples_total Total number of out of order samples ingestion failed attempts due to out of order being disabled. + # TYPE cortex_ingester_tsdb_out_of_order_samples_total counter + cortex_ingester_tsdb_out_of_order_samples_total{type="float",user="test"} 0 + cortex_ingester_tsdb_out_of_order_samples_total{type="histogram",user="test"} 1 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 1 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test"} 0 + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test"} 1 + `, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + registry := prometheus.NewRegistry() + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.ActiveSeriesMetricsEnabled = !testData.disableActiveSeries + + limits := defaultLimitsTestConfig() + limits.MaxExemplars = testData.maxExemplars + limits.OutOfOrderTimeWindow = model.Duration(testData.oooTimeWindow) + i, err := prepareIngesterWithBlocksStorageAndLimits(t, cfg, limits, nil, "", registry, !testData.disableNativeHistogram) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + ctx := user.InjectOrgID(context.Background(), userID) + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push timeseries + for idx, req := range testData.reqs { + _, err := i.PushV2(ctx, req) + + // We expect no error on any request except the last one + // which may error (and in that case we assert on it) + if idx < len(testData.reqs)-1 { + assert.NoError(t, err) + } else { + assert.Equal(t, testData.expectedErr, err) + } + } + + // Read back samples to see what has been really ingested + s := &mockQueryStreamServer{ctx: ctx} + err = i.QueryStream(&client.QueryRequest{ + StartTimestampMs: math.MinInt64, + EndTimestampMs: math.MaxInt64, + Matchers: []*client.LabelMatcher{{Type: client.REGEX_MATCH, Name: labels.MetricName, Value: ".*"}}, + }, s) + require.NoError(t, err) + set, err := seriesSetFromResponseStream(s) + require.NoError(t, err) + + require.NotNil(t, set) + r, err := client.SeriesSetToQueryResponse(set) + require.NoError(t, err) + assert.Equal(t, testData.expectedIngested, r.Timeseries) + + // Read back samples to see what has been really ingested + exemplarRes, err := i.QueryExemplars(ctx, &client.ExemplarQueryRequest{ + StartTimestampMs: math.MinInt64, + EndTimestampMs: math.MaxInt64, + Matchers: []*client.LabelMatchers{ + {Matchers: []*client.LabelMatcher{{Type: client.REGEX_MATCH, Name: labels.MetricName, Value: ".*"}}}, + }, + }) + + require.NoError(t, err) + require.NotNil(t, exemplarRes) + assert.Equal(t, testData.expectedExemplarsIngested, exemplarRes.Timeseries) + + // Read back metadata to see what has been really ingested. + mres, err := i.MetricsMetadata(ctx, &client.MetricsMetadataRequest{}) + + require.NoError(t, err) + require.NotNil(t, mres) + + // Order is never guaranteed. + assert.ElementsMatch(t, testData.expectedMetadataIngested, mres.Metadata) + + // Update active series for metrics check. + if !testData.disableActiveSeries { + i.updateActiveSeries(ctx) + } + + // Append additional metrics to assert on. + mn := append(metricNames, testData.additionalMetrics...) + + // Check tracked Prometheus metrics + err = testutil.GatherAndCompare(registry, strings.NewReader(testData.expectedMetrics), mn...) + assert.NoError(t, err) + }) + } +} + +func TestIngesterPRW2_MetricLimitExceeded(t *testing.T) { + limits := defaultLimitsTestConfig() + limits.MaxLocalSeriesPerMetric = 1 + limits.MaxLocalMetadataPerMetric = 1 + + dir := t.TempDir() + + chunksDir := filepath.Join(dir, "chunks") + blocksDir := filepath.Join(dir, "blocks") + require.NoError(t, os.Mkdir(chunksDir, os.ModePerm)) + require.NoError(t, os.Mkdir(blocksDir, os.ModePerm)) + + blocksIngesterGenerator := func() *Ingester { + ing, err := prepareIngesterWithBlocksStorageAndLimits(t, defaultIngesterTestConfig(t), limits, nil, blocksDir, prometheus.NewRegistry(), true) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), ing)) + // Wait until it's ACTIVE + test.Poll(t, time.Second, ring.ACTIVE, func() interface{} { + return ing.lifecycler.GetState() + }) + + return ing + } + + tests := []string{"chunks", "blocks"} + for i, ingGenerator := range []func() *Ingester{blocksIngesterGenerator} { + t.Run(tests[i], func(t *testing.T) { + ing := ingGenerator() + + userID := "1" + labels1 := labels.Labels{{Name: labels.MetricName, Value: "testmetric"}, {Name: "foo", Value: "bar"}} + sample1 := cortexpbv2.Sample{ + Timestamp: 0, + Value: 1, + } + sample2 := cortexpbv2.Sample{ + Timestamp: 1, + Value: 2, + } + labels3 := labels.Labels{{Name: labels.MetricName, Value: "testmetric"}, {Name: "foo", Value: "biz"}} + sample3 := cortexpbv2.Sample{ + Timestamp: 1, + Value: 3, + } + + // Append only one series and one metadata first, expect no error. + ctx := user.InjectOrgID(context.Background(), userID) + _, err := ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1}, []cortexpbv2.Sample{sample1}, nil, []cortexpbv2.Metadata{{HelpRef: 5, Type: cortexpbv2.METRIC_TYPE_COUNTER}}, cortexpbv2.API, "a help for testmetric")) + require.NoError(t, err) + + testLimits := func() { + // Append two series, expect series-exceeded error. + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1, labels3}, []cortexpbv2.Sample{sample2, sample3}, nil, nil, cortexpbv2.API)) + httpResp, ok := httpgrpc.HTTPResponseFromError(err) + require.True(t, ok, "returned error is not an httpgrpc response") + assert.Equal(t, http.StatusBadRequest, int(httpResp.Code)) + assert.Equal(t, wrapWithUser(makeMetricLimitError(perMetricSeriesLimit, labels3, ing.limiter.FormatError(userID, errMaxSeriesPerMetricLimitExceeded)), userID).Error(), string(httpResp.Body)) + + // Append two metadata for the same metric. Drop the second one, and expect no error since metadata is a best effort approach. + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1, labels3}, nil, nil, []cortexpbv2.Metadata{{HelpRef: 6, Type: cortexpbv2.METRIC_TYPE_COUNTER}, {HelpRef: 7, Type: cortexpbv2.METRIC_TYPE_COUNTER}}, cortexpbv2.API, "a help for testmetric", "a help for testmetric2")) + require.NoError(t, err) + + // Read samples back via ingester queries. + res, _, err := runTestQuery(ctx, t, ing, labels.MatchEqual, model.MetricNameLabel, "testmetric") + require.NoError(t, err) + + // Verify Series + expected := model.Matrix{ + { + Metric: cortexpb.FromLabelAdaptersToMetric(cortexpb.FromLabelsToLabelAdapters(labels1)), + Values: []model.SamplePair{ + { + Timestamp: model.Time(sample1.Timestamp), + Value: model.SampleValue(sample1.Value), + }, + { + Timestamp: model.Time(sample2.Timestamp), + Value: model.SampleValue(sample2.Value), + }, + }, + }, + } + + assert.Equal(t, expected, res) + + // Verify metadata + m, err := ing.MetricsMetadata(ctx, nil) + require.NoError(t, err) + resultMetadata := &cortexpb.MetricMetadata{MetricFamilyName: "testmetric", Help: "a help for testmetric", Type: cortexpb.COUNTER} + assert.Equal(t, []*cortexpb.MetricMetadata{resultMetadata}, m.Metadata) + } + + testLimits() + + // Limits should hold after restart. + services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + ing = ingGenerator() + defer services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + + testLimits() + }) + } +} + +func TestIngesterPRW2_UserLimitExceeded(t *testing.T) { + limits := defaultLimitsTestConfig() + limits.MaxLocalSeriesPerUser = 1 + limits.MaxLocalMetricsWithMetadataPerUser = 1 + + dir := t.TempDir() + + chunksDir := filepath.Join(dir, "chunks") + blocksDir := filepath.Join(dir, "blocks") + require.NoError(t, os.Mkdir(chunksDir, os.ModePerm)) + require.NoError(t, os.Mkdir(blocksDir, os.ModePerm)) + + blocksIngesterGenerator := func() *Ingester { + ing, err := prepareIngesterWithBlocksStorageAndLimits(t, defaultIngesterTestConfig(t), limits, nil, blocksDir, prometheus.NewRegistry(), true) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), ing)) + // Wait until it's ACTIVE + test.Poll(t, time.Second, ring.ACTIVE, func() interface{} { + return ing.lifecycler.GetState() + }) + + return ing + } + + tests := []string{"blocks"} + for i, ingGenerator := range []func() *Ingester{blocksIngesterGenerator} { + t.Run(tests[i], func(t *testing.T) { + ing := ingGenerator() + + userID := "1" + // Series + labels1 := labels.Labels{{Name: labels.MetricName, Value: "testmetric"}, {Name: "foo", Value: "bar"}} + sample1 := cortexpbv2.Sample{ + Timestamp: 0, + Value: 1, + } + sample2 := cortexpbv2.Sample{ + Timestamp: 1, + Value: 2, + } + labels3 := labels.Labels{{Name: labels.MetricName, Value: "testmetric2"}, {Name: "foo", Value: "biz"}} + sample3 := cortexpbv2.Sample{ + Timestamp: 1, + Value: 3, + } + + // Append only one series and one metadata first, expect no error. + ctx := user.InjectOrgID(context.Background(), userID) + _, err := ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1}, []cortexpbv2.Sample{sample1}, nil, []cortexpbv2.Metadata{{HelpRef: 5, Type: cortexpbv2.METRIC_TYPE_COUNTER}}, cortexpbv2.API, "a help for testmetric")) + require.NoError(t, err) + + testLimits := func() { + // Append to two series, expect series-exceeded error. + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1, labels3}, []cortexpbv2.Sample{sample2, sample3}, nil, nil, cortexpbv2.API)) + httpResp, ok := httpgrpc.HTTPResponseFromError(err) + require.True(t, ok, "returned error is not an httpgrpc response") + assert.Equal(t, http.StatusBadRequest, int(httpResp.Code)) + assert.Equal(t, wrapWithUser(makeLimitError(perUserSeriesLimit, ing.limiter.FormatError(userID, errMaxSeriesPerUserLimitExceeded)), userID).Error(), string(httpResp.Body)) + + // Append two metadata, expect no error since metadata is a best effort approach. + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2([]labels.Labels{labels1, labels3}, nil, nil, []cortexpbv2.Metadata{{HelpRef: 7, Type: cortexpbv2.METRIC_TYPE_COUNTER}, {HelpRef: 8, Type: cortexpbv2.METRIC_TYPE_COUNTER}}, cortexpbv2.API, "a help for testmetric", "a help for testmetric2")) + require.NoError(t, err) + + // Read samples back via ingester queries. + res, _, err := runTestQuery(ctx, t, ing, labels.MatchEqual, model.MetricNameLabel, "testmetric") + require.NoError(t, err) + + expected := model.Matrix{ + { + Metric: cortexpb.FromLabelAdaptersToMetric(cortexpb.FromLabelsToLabelAdapters(labels1)), + Values: []model.SamplePair{ + { + Timestamp: model.Time(sample1.Timestamp), + Value: model.SampleValue(sample1.Value), + }, + { + Timestamp: model.Time(sample2.Timestamp), + Value: model.SampleValue(sample2.Value), + }, + }, + }, + } + + // Verify samples + require.Equal(t, expected, res) + + // Verify metadata + m, err := ing.MetricsMetadata(ctx, nil) + require.NoError(t, err) + resultMetadata := &cortexpb.MetricMetadata{MetricFamilyName: "testmetric", Help: "a help for testmetric", Type: cortexpb.COUNTER} + assert.Equal(t, []*cortexpb.MetricMetadata{resultMetadata}, m.Metadata) + } + + testLimits() + + // Limits should hold after restart. + services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + ing = ingGenerator() + defer services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + + testLimits() + }) + } + +} + +func TestIngesterPRW2_PerLabelsetLimitExceeded(t *testing.T) { + limits := defaultLimitsTestConfig() + userID := "1" + registry := prometheus.NewRegistry() + + limits.LimitsPerLabelSet = []validation.LimitsPerLabelSet{ + { + LabelSet: labels.FromMap(map[string]string{ + "label1": "value1", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 3, + }, + }, + { + LabelSet: labels.FromMap(map[string]string{ + "label2": "value2", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 2, + }, + }, + } + tenantLimits := newMockTenantLimits(map[string]*validation.Limits{userID: &limits}) + + b, err := json.Marshal(limits) + require.NoError(t, err) + require.NoError(t, limits.UnmarshalJSON(b)) + + dir := t.TempDir() + chunksDir := filepath.Join(dir, "chunks") + blocksDir := filepath.Join(dir, "blocks") + require.NoError(t, os.Mkdir(chunksDir, os.ModePerm)) + require.NoError(t, os.Mkdir(blocksDir, os.ModePerm)) + + ing, err := prepareIngesterWithBlocksStorageAndLimits(t, defaultIngesterTestConfig(t), limits, tenantLimits, blocksDir, registry, true) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), ing)) + // Wait until it's ACTIVE + test.Poll(t, time.Second, ring.ACTIVE, func() interface{} { + return ing.lifecycler.GetState() + }) + + ctx := user.InjectOrgID(context.Background(), userID) + samples := []cortexpbv2.Sample{{Value: 2, Timestamp: 10}} + + // Create first series within the limits + for _, set := range limits.LimitsPerLabelSet { + lbls := []string{labels.MetricName, "metric_name"} + for _, lbl := range set.LabelSet { + lbls = append(lbls, lbl.Name, lbl.Value) + } + for i := 0; i < set.Limits.MaxSeries; i++ { + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "extraLabel", fmt.Sprintf("extraValue%v", i))...)}, samples, nil, nil, cortexpbv2.API)) + require.NoError(t, err) + } + } + + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset")) + + // Should impose limits + for _, set := range limits.LimitsPerLabelSet { + lbls := []string{labels.MetricName, "metric_name"} + for _, lbl := range set.LabelSet { + lbls = append(lbls, lbl.Name, lbl.Value) + } + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "newLabel", "newValue")...)}, samples, nil, nil, cortexpbv2.API)) + httpResp, ok := httpgrpc.HTTPResponseFromError(err) + require.True(t, ok, "returned error is not an httpgrpc response") + assert.Equal(t, http.StatusBadRequest, int(httpResp.Code)) + require.ErrorContains(t, err, set.Id) + } + + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="per_labelset_series_limit",user="1"} 2 + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset", "cortex_discarded_samples_total")) + + // Should apply composite limits + limits.LimitsPerLabelSet = append(limits.LimitsPerLabelSet, + validation.LimitsPerLabelSet{LabelSet: labels.FromMap(map[string]string{ + "comp1": "compValue1", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 10, + }, + }, + validation.LimitsPerLabelSet{LabelSet: labels.FromMap(map[string]string{ + "comp2": "compValue2", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 10, + }, + }, + validation.LimitsPerLabelSet{LabelSet: labels.FromMap(map[string]string{ + "comp1": "compValue1", + "comp2": "compValue2", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 2, + }, + }, + ) + + b, err = json.Marshal(limits) + require.NoError(t, err) + require.NoError(t, limits.UnmarshalJSON(b)) + tenantLimits.setLimits(userID, &limits) + + // Should backfill + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="per_labelset_series_limit",user="1"} 2 + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 0 + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 0 + cortex_ingester_usage_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 0 + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset", "cortex_discarded_samples_total")) + + // Adding 5 metrics with only 1 label + for i := 0; i < 5; i++ { + lbls := []string{labels.MetricName, "metric_name", "comp1", "compValue1"} + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "extraLabel", fmt.Sprintf("extraValue%v", i))...)}, samples, nil, nil, cortexpbv2.API)) + require.NoError(t, err) + } + + // Adding 2 metrics with both labels (still below the limit) + lbls := []string{labels.MetricName, "metric_name", "comp1", "compValue1", "comp2", "compValue2"} + for i := 0; i < 2; i++ { + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "extraLabel", fmt.Sprintf("extraValue%v", i))...)}, samples, nil, nil, cortexpbv2.API)) + require.NoError(t, err) + } + + // Now we should hit the limit as we already have 2 metrics with comp1=compValue1, comp2=compValue2 + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "newLabel", "newValue")...)}, samples, nil, nil, cortexpbv2.API)) + httpResp, ok := httpgrpc.HTTPResponseFromError(err) + require.True(t, ok, "returned error is not an httpgrpc response") + assert.Equal(t, http.StatusBadRequest, int(httpResp.Code)) + require.ErrorContains(t, err, labels.FromStrings("comp1", "compValue1", "comp2", "compValue2").String()) + + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_discarded_samples_total The total number of samples that were discarded. + # TYPE cortex_discarded_samples_total counter + cortex_discarded_samples_total{reason="per_labelset_series_limit",user="1"} 3 + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 7 + cortex_ingester_usage_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset", "cortex_discarded_samples_total")) + + // Should bootstrap and apply limits when configuration change + limits.LimitsPerLabelSet = append(limits.LimitsPerLabelSet, + validation.LimitsPerLabelSet{LabelSet: labels.FromMap(map[string]string{ + labels.MetricName: "metric_name", + "comp2": "compValue2", + }), + Limits: validation.LimitsPerLabelSetEntry{ + MaxSeries: 3, // we already have 2 so we need to allow 1 more + }, + }, + ) + + b, err = json.Marshal(limits) + require.NoError(t, err) + require.NoError(t, limits.UnmarshalJSON(b)) + tenantLimits.setLimits(userID, &limits) + + lbls = []string{labels.MetricName, "metric_name", "comp2", "compValue2"} + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "extraLabel", "extraValueUpdate")...)}, samples, nil, nil, cortexpbv2.API)) + require.NoError(t, err) + + _, err = ing.PushV2(ctx, cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(append(lbls, "extraLabel", "extraValueUpdate2")...)}, samples, nil, nil, cortexpbv2.API)) + httpResp, ok = httpgrpc.HTTPResponseFromError(err) + require.True(t, ok, "returned error is not an httpgrpc response") + assert.Equal(t, http.StatusBadRequest, int(httpResp.Code)) + require.ErrorContains(t, err, labels.FromStrings(lbls...).String()) + + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{__name__=\"metric_name\", comp2=\"compValue2\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + cortex_ingester_limits_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 10 + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{__name__=\"metric_name\", comp2=\"compValue2\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\", comp2=\"compValue2\"}",limit="max_series",user="1"} 2 + cortex_ingester_usage_per_labelset{labelset="{comp1=\"compValue1\"}",limit="max_series",user="1"} 7 + cortex_ingester_usage_per_labelset{labelset="{comp2=\"compValue2\"}",limit="max_series",user="1"} 3 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset")) + + // Should remove metrics when the limits is removed + limits.LimitsPerLabelSet = limits.LimitsPerLabelSet[:2] + b, err = json.Marshal(limits) + require.NoError(t, err) + require.NoError(t, limits.UnmarshalJSON(b)) + tenantLimits.setLimits(userID, &limits) + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset")) + + // Should persist between restarts + services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + registry = prometheus.NewRegistry() + ing, err = prepareIngesterWithBlocksStorageAndLimits(t, defaultIngesterTestConfig(t), limits, tenantLimits, blocksDir, registry, true) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), ing)) + ing.updateActiveSeries(ctx) + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP cortex_ingester_limits_per_labelset Limits per user and labelset. + # TYPE cortex_ingester_limits_per_labelset gauge + cortex_ingester_limits_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_limits_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + # HELP cortex_ingester_usage_per_labelset Current usage per user and labelset. + # TYPE cortex_ingester_usage_per_labelset gauge + cortex_ingester_usage_per_labelset{labelset="{label1=\"value1\"}",limit="max_series",user="1"} 3 + cortex_ingester_usage_per_labelset{labelset="{label2=\"value2\"}",limit="max_series",user="1"} 2 + `), "cortex_ingester_usage_per_labelset", "cortex_ingester_limits_per_labelset")) + services.StopAndAwaitTerminated(context.Background(), ing) //nolint:errcheck + +} + +// Referred from https://github.com/prometheus/prometheus/blob/v2.52.1/model/histogram/histogram_test.go#L985. +func TestIngesterPRW2_PushNativeHistogramErrors(t *testing.T) { + metricLabelAdapters := []cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}} + metricLabels := cortexpb.FromLabelAdaptersToLabels(metricLabelAdapters) + for _, tc := range []struct { + name string + histograms []cortexpbv2.Histogram + expectedErr error + }{ + { + name: "rejects histogram with NaN observations that has its Count (2) lower than the actual total of buckets (2 + 1)", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + ZeroCount: 2, + Count: 2, + Sum: math.NaN(), + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{1}, + }), + }, + expectedErr: fmt.Errorf("3 observations found in buckets, but the Count field is 2: %w", histogram.ErrHistogramCountNotBigEnough), + }, + { + name: "rejects histogram without NaN observations that has its Count (4) higher than the actual total of buckets (2 + 1)", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + ZeroCount: 2, + Count: 4, + Sum: 333, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{1}, + }), + }, + expectedErr: fmt.Errorf("3 observations found in buckets, but the Count field is 4: %w", histogram.ErrHistogramCountMismatch), + }, + { + name: "rejects histogram that has too few negative buckets", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + NegativeSpans: []histogram.Span{{Offset: 0, Length: 1}}, + NegativeBuckets: []int64{}, + }), + }, + expectedErr: fmt.Errorf("negative side: spans need 1 buckets, have 0 buckets: %w", histogram.ErrHistogramSpansBucketsMismatch), + }, + { + name: "rejects histogram that has too few positive buckets", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{}, + }), + }, + expectedErr: fmt.Errorf("positive side: spans need 1 buckets, have 0 buckets: %w", histogram.ErrHistogramSpansBucketsMismatch), + }, + { + name: "rejects histogram that has too many negative buckets", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + NegativeSpans: []histogram.Span{{Offset: 0, Length: 1}}, + NegativeBuckets: []int64{1, 2}, + }), + }, + expectedErr: fmt.Errorf("negative side: spans need 1 buckets, have 2 buckets: %w", histogram.ErrHistogramSpansBucketsMismatch), + }, + { + name: "rejects histogram that has too many positive buckets", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{1, 2}, + }), + }, + expectedErr: fmt.Errorf("positive side: spans need 1 buckets, have 2 buckets: %w", histogram.ErrHistogramSpansBucketsMismatch), + }, + { + name: "rejects a histogram that has a negative span with a negative offset", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}, {Offset: -1, Length: 1}}, + NegativeBuckets: []int64{1, 2}, + }), + }, + expectedErr: fmt.Errorf("negative side: span number 2 with offset -1: %w", histogram.ErrHistogramSpanNegativeOffset), + }, + { + name: "rejects a histogram that has a positive span with a negative offset", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}, {Offset: -1, Length: 1}}, + PositiveBuckets: []int64{1, 2}, + }), + }, + expectedErr: fmt.Errorf("positive side: span number 2 with offset -1: %w", histogram.ErrHistogramSpanNegativeOffset), + }, + { + name: "rejects a histogram that has a negative span with a negative count", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}}, + NegativeBuckets: []int64{-1}, + }), + }, + expectedErr: fmt.Errorf("negative side: bucket number 1 has observation count of -1: %w", histogram.ErrHistogramNegativeBucketCount), + }, + { + name: "rejects a histogram that has a positive span with a negative count", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}}, + PositiveBuckets: []int64{-1}, + }), + }, + expectedErr: fmt.Errorf("positive side: bucket number 1 has observation count of -1: %w", histogram.ErrHistogramNegativeBucketCount), + }, + { + name: "rejects a histogram that has a lower count than count in buckets", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + Count: 0, + NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}}, + PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}}, + NegativeBuckets: []int64{1}, + PositiveBuckets: []int64{1}, + }), + }, + expectedErr: fmt.Errorf("2 observations found in buckets, but the Count field is 0: %w", histogram.ErrHistogramCountMismatch), + }, + { + name: "rejects a histogram that doesn't count the zero bucket in its count", + histograms: []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(10, &histogram.Histogram{ + Count: 2, + ZeroCount: 1, + NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}}, + PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}}, + NegativeBuckets: []int64{1}, + PositiveBuckets: []int64{1}, + }), + }, + expectedErr: fmt.Errorf("3 observations found in buckets, but the Count field is 2: %w", histogram.ErrHistogramCountMismatch), + }, + } { + t.Run(tc.name, func(t *testing.T) { + registry := prometheus.NewRegistry() + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + + limits := defaultLimitsTestConfig() + i, err := prepareIngesterWithBlocksStorageAndLimits(t, cfg, limits, nil, "", registry, true) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + ctx := user.InjectOrgID(context.Background(), userID) + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + req := cortexpbv2.ToWriteRequestV2([]labels.Labels{metricLabels}, nil, tc.histograms, nil, cortexpbv2.API) + // Push timeseries + _, err = i.PushV2(ctx, req) + assert.Equal(t, httpgrpc.Errorf(http.StatusBadRequest, wrapWithUser(wrappedTSDBIngestErr(tc.expectedErr, model.Time(10), metricLabelAdapters), userID).Error()), err) + + require.Equal(t, testutil.ToFloat64(i.metrics.ingestedSamplesFail), float64(1)) + }) + } +} + +func TestIngesterPRW2_Push_ShouldCorrectlyTrackMetricsInMultiTenantScenario(t *testing.T) { + metricLabelAdapters := []cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}} + metricLabels := cortexpb.FromLabelAdaptersToLabels(metricLabelAdapters) + metricNames := []string{ + "cortex_ingester_ingested_samples_total", + "cortex_ingester_ingested_samples_failures_total", + "cortex_ingester_memory_series", + "cortex_ingester_memory_users", + "cortex_ingester_memory_series_created_total", + "cortex_ingester_memory_series_removed_total", + "cortex_ingester_active_series", + } + + registry := prometheus.NewRegistry() + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + + i, err := prepareIngesterWithBlocksStorage(t, cfg, registry) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push timeseries for each user + for _, userID := range []string{"test-1", "test-2"} { + reqs := []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + } + + for _, req := range reqs { + ctx := user.InjectOrgID(context.Background(), userID) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + } + + // Update active series for metrics check. + i.updateActiveSeries(context.Background()) + + // Check tracked Prometheus metrics + expectedMetrics := ` + # HELP cortex_ingester_ingested_samples_total The total number of samples ingested. + # TYPE cortex_ingester_ingested_samples_total counter + cortex_ingester_ingested_samples_total 4 + # HELP cortex_ingester_ingested_samples_failures_total The total number of samples that errored on ingestion. + # TYPE cortex_ingester_ingested_samples_failures_total counter + cortex_ingester_ingested_samples_failures_total 0 + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 2 + # HELP cortex_ingester_memory_series The current number of series in memory. + # TYPE cortex_ingester_memory_series gauge + cortex_ingester_memory_series 2 + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test-1"} 1 + cortex_ingester_memory_series_created_total{user="test-2"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test-1"} 0 + cortex_ingester_memory_series_removed_total{user="test-2"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test-1"} 1 + cortex_ingester_active_series{user="test-2"} 1 + ` + + assert.NoError(t, testutil.GatherAndCompare(registry, strings.NewReader(expectedMetrics), metricNames...)) +} + +func TestIngesterPRW2_Push_DecreaseInactiveSeries(t *testing.T) { + metricLabelAdapters := []cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}} + metricLabels := cortexpb.FromLabelAdaptersToLabels(metricLabelAdapters) + metricNames := []string{ + "cortex_ingester_memory_series_created_total", + "cortex_ingester_memory_series_removed_total", + "cortex_ingester_active_series", + } + + registry := prometheus.NewRegistry() + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.ActiveSeriesMetricsIdleTimeout = 100 * time.Millisecond + cfg.LifecyclerConfig.JoinAfter = 0 + + i, err := prepareIngesterWithBlocksStorage(t, cfg, registry) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push timeseries for each user + for _, userID := range []string{"test-1", "test-2"} { + reqs := []*cortexpbv2.WriteRequest{ + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 2, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + } + + for _, req := range reqs { + ctx := user.InjectOrgID(context.Background(), userID) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + } + + // Wait a bit to make series inactive (set to 100ms above). + time.Sleep(200 * time.Millisecond) + + // Update active series for metrics check. This will remove inactive series. + i.updateActiveSeries(context.Background()) + + // Check tracked Prometheus metrics + expectedMetrics := ` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="test-1"} 1 + cortex_ingester_memory_series_created_total{user="test-2"} 1 + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="test-1"} 0 + cortex_ingester_memory_series_removed_total{user="test-2"} 0 + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="test-1"} 0 + cortex_ingester_active_series{user="test-2"} 0 + ` + + assert.NoError(t, testutil.GatherAndCompare(registry, strings.NewReader(expectedMetrics), metricNames...)) +} + +func BenchmarkIngesterPRW2Push(b *testing.B) { + limits := defaultLimitsTestConfig() + benchmarkIngesterPRW2Push(b, limits, false) +} + +func benchmarkIngesterPRW2Push(b *testing.B, limits validation.Limits, errorsExpected bool) { + registry := prometheus.NewRegistry() + ctx := user.InjectOrgID(context.Background(), userID) + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(b) + cfg.LifecyclerConfig.JoinAfter = 0 + + ingester, err := prepareIngesterWithBlocksStorage(b, cfg, registry) + require.NoError(b, err) + require.NoError(b, services.StartAndAwaitRunning(context.Background(), ingester)) + defer services.StopAndAwaitTerminated(context.Background(), ingester) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(b, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return ingester.lifecycler.GetState() + }) + + // Push a single time series to set the TSDB min time. + metricLabelAdapters := []cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}} + metricLabels := cortexpb.FromLabelAdaptersToLabels(metricLabelAdapters) + startTime := util.TimeToMillis(time.Now()) + + currTimeReq := cortexpbv2.ToWriteRequestV2( + []labels.Labels{metricLabels}, + []cortexpbv2.Sample{{Value: 1, Timestamp: startTime}}, + nil, + nil, + cortexpbv2.API) + _, err = ingester.PushV2(ctx, currTimeReq) + require.NoError(b, err) + + const ( + series = 10000 + samples = 10 + ) + + allLabels, allSamples := benchmarkDataV2(series) + + b.ResetTimer() + for iter := 0; iter < b.N; iter++ { + // Bump the timestamp on each of our test samples each time round the loop + for j := 0; j < samples; j++ { + for i := range allSamples { + allSamples[i].Timestamp = startTime + int64(iter*samples+j+1) + } + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(allLabels, allSamples, nil, nil, cortexpbv2.API)) + if !errorsExpected { + require.NoError(b, err) + } + } + } +} + +func Benchmark_IngesterPRW2_PushOnError(b *testing.B) { + var ( + ctx = user.InjectOrgID(context.Background(), userID) + sampleTimestamp = int64(100) + metricName = "test" + ) + + scenarios := map[string]struct { + numSeriesPerRequest int + numConcurrentClients int + }{ + "no concurrency": { + numSeriesPerRequest: 1000, + numConcurrentClients: 1, + }, + "low concurrency": { + numSeriesPerRequest: 1000, + numConcurrentClients: 100, + }, + "high concurrency": { + numSeriesPerRequest: 1000, + numConcurrentClients: 1000, + }, + } + + instanceLimits := map[string]*InstanceLimits{ + "no limits": nil, + "limits set": {MaxIngestionRate: 1000, MaxInMemoryTenants: 1, MaxInMemorySeries: 1000, MaxInflightPushRequests: 1000}, // these match max values from scenarios + } + + tests := map[string]struct { + // If this returns false, test is skipped. + prepareConfig func(limits *validation.Limits, instanceLimits *InstanceLimits) bool + beforeBenchmark func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) + runBenchmark func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) + }{ + "out of bound samples": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { return true }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // Push a single time series to set the TSDB min time. + currTimeReq := cortexpbv2.ToWriteRequestV2( + []labels.Labels{{{Name: labels.MetricName, Value: metricName}}}, + []cortexpbv2.Sample{{Value: 1, Timestamp: util.TimeToMillis(time.Now())}}, + nil, + nil, + cortexpbv2.API) + _, err := ingester.PushV2(ctx, currTimeReq) + require.NoError(b, err) + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + expectedErr := storage.ErrOutOfBounds.Error() + // Push out of bound samples. + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) // nolint:errcheck + + verifyErrorString(b, err, expectedErr) + } + }, + }, + "out of order samples": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { return true }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // For each series, push a single sample with a timestamp greater than next pushes. + for i := 0; i < numSeriesPerRequest; i++ { + currTimeReq := cortexpbv2.ToWriteRequestV2( + []labels.Labels{{{Name: labels.MetricName, Value: metricName}, {Name: "cardinality", Value: strconv.Itoa(i)}}}, + []cortexpbv2.Sample{{Value: 1, Timestamp: sampleTimestamp + 1}}, + nil, + nil, + cortexpbv2.API) + + _, err := ingester.PushV2(ctx, currTimeReq) + require.NoError(b, err) + } + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + expectedErr := storage.ErrOutOfOrderSample.Error() + + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + // Push out of order samples. + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) // nolint:errcheck + + verifyErrorString(b, err, expectedErr) + } + }, + }, + "per-user series limit reached": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + limits.MaxLocalSeriesPerUser = 1 + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // Push a series with a metric name different than the one used during the benchmark. + currTimeReq := cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(labels.MetricName, "another")}, + []cortexpbv2.Sample{{Value: 1, Timestamp: sampleTimestamp + 1}}, + nil, + nil, + cortexpbv2.API) + _, err := ingester.PushV2(ctx, currTimeReq) + require.NoError(b, err) + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + // Push series with a different name than the one already pushed. + + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) // nolint:errcheck + verifyErrorString(b, err, "per-user series limit") + } + }, + }, + "per-metric series limit reached": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + limits.MaxLocalSeriesPerMetric = 1 + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // Push a series with the same metric name but different labels than the one used during the benchmark. + currTimeReq := cortexpbv2.ToWriteRequestV2( + []labels.Labels{labels.FromStrings(labels.MetricName, metricName, "cardinality", "another")}, + []cortexpbv2.Sample{{Value: 1, Timestamp: sampleTimestamp + 1}}, + nil, + nil, + cortexpbv2.API) + _, err := ingester.PushV2(ctx, currTimeReq) + require.NoError(b, err) + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + // Push series with different labels than the one already pushed. + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) // nolint:errcheck + verifyErrorString(b, err, "per-metric series limit") + } + }, + }, + "very low ingestion rate limit": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + if instanceLimits == nil { + return false + } + instanceLimits.MaxIngestionRate = 0.00001 // very low + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // Send a lot of samples + _, err := ingester.PushV2(ctx, generateSamplesForLabelV2(labels.FromStrings(labels.MetricName, "test"), 10000)) + require.NoError(b, err) + + ingester.ingestionRate.Tick() + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + // Push series with different labels than the one already pushed. + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) + verifyErrorString(b, err, "push rate reached") + } + }, + }, + "max number of tenants reached": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + if instanceLimits == nil { + return false + } + instanceLimits.MaxInMemoryTenants = 1 + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + // Send some samples for one tenant (not the same that is used during the test) + ctx := user.InjectOrgID(context.Background(), "different_tenant") + _, err := ingester.PushV2(ctx, generateSamplesForLabelV2(labels.FromStrings(labels.MetricName, "test"), 10000)) + require.NoError(b, err) + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + // Push series with different labels than the one already pushed. + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) + verifyErrorString(b, err, "max tenants limit reached") + } + }, + }, + "max number of series reached": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + if instanceLimits == nil { + return false + } + instanceLimits.MaxInMemorySeries = 1 + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + _, err := ingester.PushV2(ctx, generateSamplesForLabelV2(labels.FromStrings(labels.MetricName, "test"), 10000)) + require.NoError(b, err) + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) + verifyErrorString(b, err, "max series limit reached") + } + }, + }, + "max inflight requests reached": { + prepareConfig: func(limits *validation.Limits, instanceLimits *InstanceLimits) bool { + if instanceLimits == nil { + return false + } + instanceLimits.MaxInflightPushRequests = 1 + return true + }, + beforeBenchmark: func(b *testing.B, ingester *Ingester, numSeriesPerRequest int) { + ingester.inflightPushRequests.Inc() + }, + runBenchmark: func(b *testing.B, ingester *Ingester, metrics []labels.Labels, samples []cortexpbv2.Sample) { + st := writev2.NewSymbolTable() + for _, lbs := range metrics { + st.SymbolizeLabels(lbs, nil) + } + for n := 0; n < b.N; n++ { + _, err := ingester.PushV2(ctx, cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API)) + verifyErrorString(b, err, "too many inflight push requests") + } + }, + }, + } + + for testName, testData := range tests { + for scenarioName, scenario := range scenarios { + for limitsName, limits := range instanceLimits { + b.Run(fmt.Sprintf("failure: %s, scenario: %s, limits: %s", testName, scenarioName, limitsName), func(b *testing.B) { + registry := prometheus.NewRegistry() + + instanceLimits := limits + if instanceLimits != nil { + // make a copy, to avoid changing value in the instanceLimits map. + newLimits := &InstanceLimits{} + *newLimits = *instanceLimits + instanceLimits = newLimits + } + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(b) + cfg.LifecyclerConfig.JoinAfter = 0 + + limits := defaultLimitsTestConfig() + if !testData.prepareConfig(&limits, instanceLimits) { + b.SkipNow() + } + + cfg.InstanceLimitsFn = func() *InstanceLimits { + return instanceLimits + } + + ingester, err := prepareIngesterWithBlocksStorageAndLimits(b, cfg, limits, nil, "", registry, true) + require.NoError(b, err) + require.NoError(b, services.StartAndAwaitRunning(context.Background(), ingester)) + defer services.StopAndAwaitTerminated(context.Background(), ingester) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(b, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return ingester.lifecycler.GetState() + }) + + testData.beforeBenchmark(b, ingester, scenario.numSeriesPerRequest) + + // Prepare the request. + metrics := make([]labels.Labels, 0, scenario.numSeriesPerRequest) + samples := make([]cortexpbv2.Sample, 0, scenario.numSeriesPerRequest) + for i := 0; i < scenario.numSeriesPerRequest; i++ { + metrics = append(metrics, labels.Labels{{Name: labels.MetricName, Value: metricName}, {Name: "cardinality", Value: strconv.Itoa(i)}}) + samples = append(samples, cortexpbv2.Sample{Value: float64(i), Timestamp: sampleTimestamp}) + } + + // Run the benchmark. + wg := sync.WaitGroup{} + wg.Add(scenario.numConcurrentClients) + start := make(chan struct{}) + + b.ReportAllocs() + b.ResetTimer() + + for c := 0; c < scenario.numConcurrentClients; c++ { + go func() { + defer wg.Done() + <-start + + testData.runBenchmark(b, ingester, metrics, samples) + }() + } + + b.ResetTimer() + close(start) + wg.Wait() + }) + } + } + } +} + +func TestIngesterPRW2_LabelNames(t *testing.T) { + series := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "500"}, {Name: "route", Value: "get_user"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + } + + expected := []string{"__name__", "route", "status"} + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range series { + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + tests := map[string]struct { + limit int + expected []string + }{ + "should return all label names if no limit is set": { + expected: expected, + }, + "should return limited label names if a limit is set": { + limit: 2, + expected: expected[:2], + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + // Get label names + res, err := i.LabelNames(ctx, &client.LabelNamesRequest{Limit: int64(testData.limit)}) + require.NoError(t, err) + assert.ElementsMatch(t, testData.expected, res.LabelNames) + }) + } +} + +func TestIngesterPRW2_LabelValues(t *testing.T) { + series := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "500"}, {Name: "route", Value: "get_user"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + } + + expected := map[string][]string{ + "__name__": {"test_1", "test_2"}, + "status": {"200", "500"}, + "route": {"get_user"}, + "unknown": {}, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range series { + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + tests := map[string]struct { + limit int64 + match []*labels.Matcher + }{ + "should return all label values if no limit is set": { + limit: 0, + }, + "should return limited label values if a limit is set": { + limit: 1, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + for labelName, expectedValues := range expected { + req := &client.LabelValuesRequest{LabelName: labelName, Limit: testData.limit} + res, err := i.LabelValues(ctx, req) + require.NoError(t, err) + if testData.limit > 0 && len(expectedValues) > int(testData.limit) { + expectedValues = expectedValues[:testData.limit] + } + assert.ElementsMatch(t, expectedValues, res.LabelValues) + } + }) + } + +} + +func TestIngesterPRW2_LabelValue_MaxInflightQueryRequest(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.DefaultLimits.MaxInflightQueryRequests = 1 + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + i.inflightQueryRequests.Add(1) + + // Mock request + ctx := user.InjectOrgID(context.Background(), "test") + + wreq, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000) + _, err = i.PushV2(ctx, wreq) + require.NoError(t, err) + + rreq := &client.LabelValuesRequest{} + _, err = i.LabelValues(ctx, rreq) + require.Error(t, err) + require.Equal(t, err, errTooManyInflightQueryRequests) +} + +func Test_IngesterPRW2_Query(t *testing.T) { + series := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "route", Value: "get_user"}, {Name: "status", Value: "200"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "route", Value: "get_user"}, {Name: "status", Value: "500"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + } + + tests := map[string]struct { + from int64 + to int64 + matchers []*client.LabelMatcher + expected []cortexpb.TimeSeries + }{ + "should return an empty response if no metric matches": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "unknown"}, + }, + expected: []cortexpb.TimeSeries{}, + }, + "should filter series by == matcher": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[0].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 100000}}}, + {Labels: cortexpb.FromLabelsToLabelAdapters(series[1].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 110000}}}, + }, + }, + "should filter series by != matcher": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.NOT_EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[2].lbls), Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 200000}}}, + }, + }, + "should filter series by =~ matcher": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.REGEX_MATCH, Name: model.MetricNameLabel, Value: ".*_1"}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[0].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 100000}}}, + {Labels: cortexpb.FromLabelsToLabelAdapters(series[1].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 110000}}}, + }, + }, + "should filter series by !~ matcher": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.REGEX_NO_MATCH, Name: model.MetricNameLabel, Value: ".*_1"}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[2].lbls), Samples: []cortexpb.Sample{{Value: 2, TimestampMs: 200000}}}, + }, + }, + "should filter series by multiple matchers": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + {Type: client.REGEX_MATCH, Name: "status", Value: "5.."}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[1].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 110000}}}, + }, + }, + "should filter series by matcher and time range": { + from: 100000, + to: 100000, + matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + expected: []cortexpb.TimeSeries{ + {Labels: cortexpb.FromLabelsToLabelAdapters(series[0].lbls), Samples: []cortexpb.Sample{{Value: 1, TimestampMs: 100000}}}, + }, + }, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range series { + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + // Run tests + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + req := &client.QueryRequest{ + StartTimestampMs: testData.from, + EndTimestampMs: testData.to, + Matchers: testData.matchers, + } + + s := &mockQueryStreamServer{ctx: ctx} + err = i.QueryStream(req, s) + require.NoError(t, err) + set, err := seriesSetFromResponseStream(s) + require.NoError(t, err) + r, err := client.SeriesSetToQueryResponse(set) + require.NoError(t, err) + fmt.Println("testData.expected", testData.expected) + fmt.Println("r.Timeseries", r.Timeseries) + assert.ElementsMatch(t, testData.expected, r.Timeseries) + }) + } +} + +func TestIngesterPRW2_Query_MaxInflightQueryRequest(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.DefaultLimits.MaxInflightQueryRequests = 1 + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + i.inflightQueryRequests.Add(1) + + // Mock request + ctx := user.InjectOrgID(context.Background(), "test") + + wreq, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000) + _, err = i.PushV2(ctx, wreq) + require.NoError(t, err) + + rreq := &client.QueryRequest{} + s := &mockQueryStreamServer{ctx: ctx} + err = i.QueryStream(rreq, s) + require.Error(t, err) + require.Equal(t, err, errTooManyInflightQueryRequests) +} + +func TestIngesterPRW2_Push_ShouldNotCreateTSDBIfNotInActiveState(t *testing.T) { + // Configure the lifecycler to not immediately join the ring, to make sure + // the ingester will NOT be in the ACTIVE state when we'll push samples. + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 10 * time.Second + + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + require.Equal(t, ring.PENDING, i.lifecycler.GetState()) + + // Mock request + userID := "test" + ctx := user.InjectOrgID(context.Background(), userID) + req := &cortexpbv2.WriteRequest{} + + res, err := i.PushV2(ctx, req) + assert.Equal(t, wrapWithUser(fmt.Errorf(errTSDBCreateIncompatibleState, "PENDING"), userID).Error(), err.Error()) + assert.Nil(t, res) + + // Check if the TSDB has been created + _, tsdbCreated := i.TSDBState.dbs[userID] + assert.False(t, tsdbCreated) +} + +func TestIngesterPRW2_MetricsForLabelMatchers(t *testing.T) { + fixtures := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "500"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + // The two following series have the same FastFingerprint=e002a3a451262627 + {labels.Labels{{Name: labels.MetricName, Value: "collision"}, {Name: "app", Value: "l"}, {Name: "uniq0", Value: "0"}, {Name: "uniq1", Value: "1"}}, 1, 300000}, + {labels.Labels{{Name: labels.MetricName, Value: "collision"}, {Name: "app", Value: "m"}, {Name: "uniq0", Value: "1"}, {Name: "uniq1", Value: "1"}}, 1, 300000}, + } + + tests := map[string]struct { + from int64 + to int64 + limit int64 + matchers []*client.LabelMatchers + expected []*cortexpb.Metric + queryIngestersWithin time.Duration + }{ + "should return an empty response if no metric match": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatchers{{ + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "unknown"}, + }, + }}, + expected: []*cortexpb.Metric{}, + }, + "should filter metrics by single matcher": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatchers{{ + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + }}, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[1].lbls)}, + }, + }, + "should filter metrics by multiple matchers": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatchers{ + { + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: "status", Value: "200"}, + }, + }, + { + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_2"}, + }, + }, + }, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[2].lbls)}, + }, + }, + "should NOT filter metrics by time range to always return known metrics even when queried for older time ranges": { + from: 100, + to: 1000, + matchers: []*client.LabelMatchers{{ + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + }}, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[1].lbls)}, + }, + }, + "should filter metrics by time range if queryIngestersWithin is enabled": { + from: 99999, + to: 100001, + matchers: []*client.LabelMatchers{{ + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + }}, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + }, + queryIngestersWithin: time.Hour, + }, + "should not return duplicated metrics on overlapping matchers": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatchers{ + { + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_1"}, + }, + }, + { + Matchers: []*client.LabelMatcher{ + {Type: client.REGEX_MATCH, Name: model.MetricNameLabel, Value: "test.*"}, + }, + }, + }, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[1].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[2].lbls)}, + }, + }, + "should return all matching metrics even if their FastFingerprint collide": { + from: math.MinInt64, + to: math.MaxInt64, + matchers: []*client.LabelMatchers{{ + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "collision"}, + }, + }}, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[3].lbls)}, + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[4].lbls)}, + }, + }, + "should return only limited results": { + from: math.MinInt64, + to: math.MaxInt64, + limit: 1, + matchers: []*client.LabelMatchers{ + { + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: "status", Value: "200"}, + }, + }, + { + Matchers: []*client.LabelMatcher{ + {Type: client.EQUAL, Name: model.MetricNameLabel, Value: "test_2"}, + }, + }, + }, + expected: []*cortexpb.Metric{ + {Labels: cortexpb.FromLabelsToLabelAdapters(fixtures[0].lbls)}, + }, + }, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push fixtures + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range fixtures { + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + // Run tests + for testName, testData := range tests { + testData := testData + + t.Run(testName, func(t *testing.T) { + req := &client.MetricsForLabelMatchersRequest{ + StartTimestampMs: testData.from, + EndTimestampMs: testData.to, + MatchersSet: testData.matchers, + Limit: testData.limit, + } + i.cfg.QueryIngestersWithin = testData.queryIngestersWithin + res, err := i.MetricsForLabelMatchers(ctx, req) + require.NoError(t, err) + assert.ElementsMatch(t, testData.expected, res.Metric) + }) + } +} + +func TestIngesterPRW2_MetricsForLabelMatchers_Deduplication(t *testing.T) { + const ( + userID = "test" + numSeries = 100000 + ) + + now := util.TimeToMillis(time.Now()) + i := createIngesterWithSeriesV2(t, userID, numSeries, 1, now, 1) + ctx := user.InjectOrgID(context.Background(), "test") + + req := &client.MetricsForLabelMatchersRequest{ + StartTimestampMs: now, + EndTimestampMs: now, + // Overlapping matchers to make sure series are correctly deduplicated. + MatchersSet: []*client.LabelMatchers{ + {Matchers: []*client.LabelMatcher{ + {Type: client.REGEX_MATCH, Name: model.MetricNameLabel, Value: "test.*"}, + }}, + {Matchers: []*client.LabelMatcher{ + {Type: client.REGEX_MATCH, Name: model.MetricNameLabel, Value: "test.*0"}, + }}, + }, + } + + res, err := i.MetricsForLabelMatchers(ctx, req) + require.NoError(t, err) + require.Len(t, res.GetMetric(), numSeries) +} + +func BenchmarkIngesterPRW2_MetricsForLabelMatchers(b *testing.B) { + var ( + userID = "test" + numSeries = 10000 + numSamplesPerSeries = 60 * 6 // 6h on 1 sample per minute + startTimestamp = util.TimeToMillis(time.Now()) + step = int64(60000) // 1 sample per minute + ) + + i := createIngesterWithSeriesV2(b, userID, numSeries, numSamplesPerSeries, startTimestamp, step) + ctx := user.InjectOrgID(context.Background(), "test") + + // Flush the ingester to ensure blocks have been compacted, so we'll test + // fetching labels from blocks. + i.Flush() + + b.ResetTimer() + b.ReportAllocs() + + for n := 0; n < b.N; n++ { + req := &client.MetricsForLabelMatchersRequest{ + StartTimestampMs: math.MinInt64, + EndTimestampMs: math.MaxInt64, + MatchersSet: []*client.LabelMatchers{{Matchers: []*client.LabelMatcher{ + {Type: client.REGEX_MATCH, Name: model.MetricNameLabel, Value: "test.*"}, + }}}, + } + + res, err := i.MetricsForLabelMatchers(ctx, req) + require.NoError(b, err) + require.Len(b, res.GetMetric(), numSeries) + } +} + +func TestIngesterPRW2_QueryStream(t *testing.T) { + // Create ingester. + cfg := defaultIngesterTestConfig(t) + + for _, enc := range encodings { + t.Run(enc.String(), func(t *testing.T) { + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE. + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series. + ctx := user.InjectOrgID(context.Background(), userID) + lbls := labels.Labels{{Name: labels.MetricName, Value: "foo"}} + var ( + req *cortexpbv2.WriteRequest + expectedResponseChunks *client.QueryStreamResponse + ) + switch enc { + case encoding.PrometheusXorChunk: + req, expectedResponseChunks = mockWriteRequestV2(t, lbls, 123000, 456) + case encoding.PrometheusHistogramChunk: + req, expectedResponseChunks = mockHistogramWriteRequestV2(t, lbls, 123000, 456, false) + case encoding.PrometheusFloatHistogramChunk: + req, expectedResponseChunks = mockHistogramWriteRequestV2(t, lbls, 123000, 456, true) + } + _, err = i.PushV2(ctx, req) + require.NoError(t, err) + + // Create a GRPC server used to query back the data. + serv := grpc.NewServer(grpc.StreamInterceptor(middleware.StreamServerUserHeaderInterceptor)) + defer serv.GracefulStop() + client.RegisterIngesterServer(serv, i) + + listener, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + + go func() { + require.NoError(t, serv.Serve(listener)) + }() + + // Query back the series using GRPC streaming. + c, err := client.MakeIngesterClient(listener.Addr().String(), defaultClientTestConfig()) + require.NoError(t, err) + defer c.Close() + + queryRequest := &client.QueryRequest{ + StartTimestampMs: 0, + EndTimestampMs: 200000, + Matchers: []*client.LabelMatcher{{ + Type: client.EQUAL, + Name: model.MetricNameLabel, + Value: "foo", + }}, + } + + chunksTest := func(t *testing.T) { + s, err := c.QueryStream(ctx, queryRequest) + require.NoError(t, err) + + count := 0 + var lastResp *client.QueryStreamResponse + for { + resp, err := s.Recv() + if err == io.EOF { + break + } + require.NoError(t, err) + count += len(resp.Chunkseries) + lastResp = resp + } + require.Equal(t, 1, count) + require.Equal(t, expectedResponseChunks, lastResp) + } + + t.Run("chunks", chunksTest) + }) + } +} + +func TestIngesterPRW2_QueryStreamManySamplesChunks(t *testing.T) { + // Create ingester. + cfg := defaultIngesterTestConfig(t) + + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE. + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series. + ctx := user.InjectOrgID(context.Background(), userID) + + const samplesCount = 1000000 + samples := make([]cortexpbv2.Sample, 0, samplesCount) + + for i := 0; i < samplesCount; i++ { + samples = append(samples, cortexpbv2.Sample{ + Value: float64(i), + Timestamp: int64(i), + }) + } + + // 100k samples in chunks use about 154 KiB, + _, err = i.PushV2(ctx, writeRequestSingleSeriesV2(labels.Labels{{Name: labels.MetricName, Value: "foo"}, {Name: "l", Value: "1"}}, samples[0:100000])) + require.NoError(t, err) + + // 1M samples in chunks use about 1.51 MiB, + _, err = i.PushV2(ctx, writeRequestSingleSeriesV2(labels.Labels{{Name: labels.MetricName, Value: "foo"}, {Name: "l", Value: "2"}}, samples)) + require.NoError(t, err) + + // 500k samples in chunks need 775 KiB, + _, err = i.PushV2(ctx, writeRequestSingleSeriesV2(labels.Labels{{Name: labels.MetricName, Value: "foo"}, {Name: "l", Value: "3"}}, samples[0:500000])) + require.NoError(t, err) + + // Create a GRPC server used to query back the data. + serv := grpc.NewServer(grpc.StreamInterceptor(middleware.StreamServerUserHeaderInterceptor)) + defer serv.GracefulStop() + client.RegisterIngesterServer(serv, i) + + listener, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + + go func() { + require.NoError(t, serv.Serve(listener)) + }() + + // Query back the series using GRPC streaming. + c, err := client.MakeIngesterClient(listener.Addr().String(), defaultClientTestConfig()) + require.NoError(t, err) + defer c.Close() + + s, err := c.QueryStream(ctx, &client.QueryRequest{ + StartTimestampMs: 0, + EndTimestampMs: samplesCount + 1, + + Matchers: []*client.LabelMatcher{{ + Type: client.EQUAL, + Name: model.MetricNameLabel, + Value: "foo", + }}, + }) + require.NoError(t, err) + + recvMsgs := 0 + series := 0 + totalSamples := 0 + + for { + resp, err := s.Recv() + if err == io.EOF { + break + } + require.NoError(t, err) + require.True(t, len(resp.Chunkseries) > 0) // No empty messages. + + recvMsgs++ + series += len(resp.Chunkseries) + + for _, ts := range resp.Chunkseries { + for _, c := range ts.Chunks { + enc := encoding.Encoding(c.Encoding).PromChunkEncoding() + require.True(t, enc != chunkenc.EncNone) + chk, err := chunkenc.FromData(enc, c.Data) + require.NoError(t, err) + totalSamples += chk.NumSamples() + } + } + } + + // As ingester doesn't guarantee sorting of series, we can get 2 (100k + 500k in first, 1M in second) + // or 3 messages (100k or 500k first, 1M second, and 500k or 100k last). + + require.True(t, 2 <= recvMsgs && recvMsgs <= 3) + require.Equal(t, 3, series) + require.Equal(t, 100000+500000+samplesCount, totalSamples) +} + +func BenchmarkIngesterPRW2_QueryStream_Chunks(b *testing.B) { + benchmarkQueryStreamV2(b) +} + +func benchmarkQueryStreamV2(b *testing.B) { + cfg := defaultIngesterTestConfig(b) + + // Create ingester. + i, err := prepareIngesterWithBlocksStorage(b, cfg, prometheus.NewRegistry()) + require.NoError(b, err) + require.NoError(b, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE. + test.Poll(b, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series. + ctx := user.InjectOrgID(context.Background(), userID) + + const samplesCount = 1000 + samples := make([]cortexpbv2.Sample, 0, samplesCount) + + for i := 0; i < samplesCount; i++ { + samples = append(samples, cortexpbv2.Sample{ + Value: float64(i), + Timestamp: int64(i), + }) + } + + const seriesCount = 100 + for s := 0; s < seriesCount; s++ { + _, err = i.PushV2(ctx, writeRequestSingleSeriesV2(labels.Labels{{Name: labels.MetricName, Value: "foo"}, {Name: "l", Value: strconv.Itoa(s)}}, samples)) + require.NoError(b, err) + } + + req := &client.QueryRequest{ + StartTimestampMs: 0, + EndTimestampMs: samplesCount + 1, + + Matchers: []*client.LabelMatcher{{ + Type: client.EQUAL, + Name: model.MetricNameLabel, + Value: "foo", + }}, + } + + mockStream := &mockQueryStreamServer{ctx: ctx} + + b.ResetTimer() + + for ix := 0; ix < b.N; ix++ { + err := i.QueryStream(req, mockStream) + require.NoError(b, err) + } +} + +func TestIngesterPRW2_dontShipBlocksWhenTenantDeletionMarkerIsPresent(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 2 + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + + // Use in-memory bucket. + bucket := objstore.NewInMemBucket() + + i.TSDBState.bucket = bucket + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + pushSingleSampleWithMetadataV2(t, i) + require.Equal(t, int64(1), i.TSDBState.seriesCount.Load()) + i.compactBlocks(context.Background(), true, nil) + require.Equal(t, int64(0), i.TSDBState.seriesCount.Load()) + i.shipBlocks(context.Background(), nil) + + numObjects := len(bucket.Objects()) + require.NotZero(t, numObjects) + + require.NoError(t, cortex_tsdb.WriteTenantDeletionMark(context.Background(), objstore.WithNoopInstr(bucket), userID, cortex_tsdb.NewTenantDeletionMark(time.Now()))) + numObjects++ // For deletion marker + + db := i.getTSDB(userID) + require.NotNil(t, db) + db.lastDeletionMarkCheck.Store(0) + + // After writing tenant deletion mark, + pushSingleSampleWithMetadataV2(t, i) + require.Equal(t, int64(1), i.TSDBState.seriesCount.Load()) + i.compactBlocks(context.Background(), true, nil) + require.Equal(t, int64(0), i.TSDBState.seriesCount.Load()) + i.shipBlocks(context.Background(), nil) + + numObjectsAfterMarkingTenantForDeletion := len(bucket.Objects()) + require.Equal(t, numObjects, numObjectsAfterMarkingTenantForDeletion) + require.Equal(t, tsdbTenantMarkedForDeletion, i.closeAndDeleteUserTSDBIfIdle(userID)) +} + +func TestIngesterPRW2_seriesCountIsCorrectAfterClosingTSDBForDeletedTenant(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 2 + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + + // Use in-memory bucket. + bucket := objstore.NewInMemBucket() + + // Write tenant deletion mark. + require.NoError(t, cortex_tsdb.WriteTenantDeletionMark(context.Background(), objstore.WithNoopInstr(bucket), userID, cortex_tsdb.NewTenantDeletionMark(time.Now()))) + + i.TSDBState.bucket = bucket + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + pushSingleSampleWithMetadataV2(t, i) + require.Equal(t, int64(1), i.TSDBState.seriesCount.Load()) + + // We call shipBlocks to check for deletion marker (it happens inside this method). + i.shipBlocks(context.Background(), nil) + + // Verify that tenant deletion mark was found. + db := i.getTSDB(userID) + require.NotNil(t, db) + require.True(t, db.deletionMarkFound.Load()) + + // If we try to close TSDB now, it should succeed, even though TSDB is not idle and empty. + require.Equal(t, uint64(1), db.Head().NumSeries()) + require.Equal(t, tsdbTenantMarkedForDeletion, i.closeAndDeleteUserTSDBIfIdle(userID)) + + // Closing should decrease series count. + require.Equal(t, int64(0), i.TSDBState.seriesCount.Load()) +} + +func TestIngesterPRW2_invalidSamplesDontChangeLastUpdateTime(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + ctx := user.InjectOrgID(context.Background(), userID) + sampleTimestamp := int64(model.Now()) + + { + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, sampleTimestamp) + _, err = i.PushV2(ctx, req) + require.NoError(t, err) + } + + db := i.getTSDB(userID) + lastUpdate := db.lastUpdate.Load() + + // Wait until 1 second passes. + test.Poll(t, 1*time.Second, time.Now().Unix()+1, func() interface{} { + return time.Now().Unix() + }) + + // Push another sample to the same metric and timestamp, with different value. We expect to get error. + { + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 1, sampleTimestamp) + _, err = i.PushV2(ctx, req) + require.Error(t, err) + } + + // Make sure last update hasn't changed. + require.Equal(t, lastUpdate, db.lastUpdate.Load()) +} + +func TestIngesterPRW2_flushing(t *testing.T) { + for name, tc := range map[string]struct { + setupIngester func(cfg *Config) + action func(t *testing.T, i *Ingester, reg *prometheus.Registry) + }{ + "ingesterShutdown": { + setupIngester: func(cfg *Config) { + cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown = true + cfg.BlocksStorageConfig.TSDB.KeepUserTSDBOpenOnShutdown = true + }, + action: func(t *testing.T, i *Ingester, reg *prometheus.Registry) { + pushSingleSampleWithMetadataV2(t, i) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + // Shutdown ingester. This triggers flushing of the block. + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) + + verifyCompactedHead(t, i, true) + + // Verify that block has been shipped. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 1 + `), "cortex_ingester_shipper_uploads_total")) + }, + }, + + "shutdownHandler": { + setupIngester: func(cfg *Config) { + cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown = false + cfg.BlocksStorageConfig.TSDB.KeepUserTSDBOpenOnShutdown = true + }, + + action: func(t *testing.T, i *Ingester, reg *prometheus.Registry) { + pushSingleSampleWithMetadataV2(t, i) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + i.ShutdownHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/shutdown", nil)) + + verifyCompactedHead(t, i, true) + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 1 + `), "cortex_ingester_shipper_uploads_total")) + }, + }, + + "flushHandler": { + setupIngester: func(cfg *Config) { + cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown = false + }, + + action: func(t *testing.T, i *Ingester, reg *prometheus.Registry) { + pushSingleSampleWithMetadataV2(t, i) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + // Using wait=true makes this a synchronous call. + i.FlushHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/flush?wait=true", nil)) + + verifyCompactedHead(t, i, true) + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 1 + `), "cortex_ingester_shipper_uploads_total")) + }, + }, + + "flushHandlerWithListOfTenants": { + setupIngester: func(cfg *Config) { + cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown = false + }, + + action: func(t *testing.T, i *Ingester, reg *prometheus.Registry) { + pushSingleSampleWithMetadataV2(t, i) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + users := url.Values{} + users.Add(tenantParam, "unknown-user") + users.Add(tenantParam, "another-unknown-user") + + // Using wait=true makes this a synchronous call. + i.FlushHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/flush?wait=true&"+users.Encode(), nil)) + + // Still nothing shipped or compacted. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + verifyCompactedHead(t, i, false) + + users = url.Values{} + users.Add(tenantParam, "different-user") + users.Add(tenantParam, userID) // Our user + users.Add(tenantParam, "yet-another-user") + + i.FlushHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/flush?wait=true&"+users.Encode(), nil)) + + verifyCompactedHead(t, i, true) + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 1 + `), "cortex_ingester_shipper_uploads_total")) + }, + }, + + "flushMultipleBlocksWithDataSpanning3Days": { + setupIngester: func(cfg *Config) { + cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown = false + }, + + action: func(t *testing.T, i *Ingester, reg *prometheus.Registry) { + // Pushing 5 samples, spanning over 3 days. + // First block + pushSingleSampleAtTimeV2(t, i, 23*time.Hour.Milliseconds()) + pushSingleSampleAtTimeV2(t, i, 24*time.Hour.Milliseconds()-1) + + // Second block + pushSingleSampleAtTimeV2(t, i, 24*time.Hour.Milliseconds()+1) + pushSingleSampleAtTimeV2(t, i, 25*time.Hour.Milliseconds()) + + // Third block, far in the future. + pushSingleSampleAtTimeV2(t, i, 50*time.Hour.Milliseconds()) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + i.FlushHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/flush?wait=true", nil)) + + verifyCompactedHead(t, i, true) + + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 3 + `), "cortex_ingester_shipper_uploads_total")) + + userDB := i.getTSDB(userID) + require.NotNil(t, userDB) + + blocks := userDB.Blocks() + require.Equal(t, 3, len(blocks)) + require.Equal(t, 23*time.Hour.Milliseconds(), blocks[0].Meta().MinTime) + require.Equal(t, 24*time.Hour.Milliseconds(), blocks[0].Meta().MaxTime) // Block maxt is exclusive. + + require.Equal(t, 24*time.Hour.Milliseconds()+1, blocks[1].Meta().MinTime) + require.Equal(t, 26*time.Hour.Milliseconds(), blocks[1].Meta().MaxTime) + + require.Equal(t, 50*time.Hour.Milliseconds()+1, blocks[2].Meta().MaxTime) // Block maxt is exclusive. + }, + }, + } { + t.Run(name, func(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 1 + cfg.BlocksStorageConfig.TSDB.ShipInterval = 1 * time.Minute // Long enough to not be reached during the test. + + if tc.setupIngester != nil { + tc.setupIngester(&cfg) + } + + // Create ingester + reg := prometheus.NewPedanticRegistry() + i, err := prepareIngesterWithBlocksStorage(t, cfg, reg) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // mock user's shipper + tc.action(t, i, reg) + }) + } +} + +func TestIngesterPRW2_ForFlush(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 1 + cfg.BlocksStorageConfig.TSDB.ShipInterval = 10 * time.Minute // Long enough to not be reached during the test. + + // Create ingester + reg := prometheus.NewPedanticRegistry() + i, err := prepareIngesterWithBlocksStorage(t, cfg, reg) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push some data. + pushSingleSampleWithMetadataV2(t, i) + + // Stop ingester. + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) + + // Nothing shipped yet. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 0 + `), "cortex_ingester_shipper_uploads_total")) + + // Restart ingester in "For Flusher" mode. We reuse the same config (esp. same dir) + reg = prometheus.NewPedanticRegistry() + i, err = NewForFlusher(i.cfg, i.limits, reg, log.NewNopLogger()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + + // Our single sample should be reloaded from WAL + verifyCompactedHead(t, i, false) + i.Flush() + + // Head should be empty after flushing. + verifyCompactedHead(t, i, true) + + // Verify that block has been shipped. + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_ingester_shipper_uploads_total Total number of uploaded TSDB blocks + # TYPE cortex_ingester_shipper_uploads_total counter + cortex_ingester_shipper_uploads_total 1 + `), "cortex_ingester_shipper_uploads_total")) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) +} + +func TestIngesterPRW2_UserStats(t *testing.T) { + series := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "500"}, {Name: "route", Value: "get_user"}}, 1, 110000}, + {labels.Labels{{Name: labels.MetricName, Value: "test_2"}}, 2, 200000}, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range series { + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + // force update statistics + for _, db := range i.TSDBState.dbs { + db.ingestedAPISamples.Tick() + db.ingestedRuleSamples.Tick() + } + + // Get label names + res, err := i.UserStats(ctx, &client.UserStatsRequest{}) + require.NoError(t, err) + assert.InDelta(t, 0.2, res.ApiIngestionRate, 0.0001) + assert.InDelta(t, float64(0), res.RuleIngestionRate, 0.0001) + assert.Equal(t, uint64(3), res.NumSeries) +} + +func TestIngesterPRW2_AllUserStats(t *testing.T) { + series := []struct { + user string + lbls labels.Labels + value float64 + timestamp int64 + }{ + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000}, + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_1"}, {Name: "status", Value: "500"}, {Name: "route", Value: "get_user"}}, 1, 110000}, + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_2"}}, 2, 200000}, + {"user-2", labels.Labels{{Name: labels.MetricName, Value: "test_2_1"}}, 2, 200000}, + {"user-2", labels.Labels{{Name: labels.MetricName, Value: "test_2_2"}}, 2, 200000}, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + for _, series := range series { + ctx := user.InjectOrgID(context.Background(), series.user) + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + // force update statistics + for _, db := range i.TSDBState.dbs { + db.ingestedAPISamples.Tick() + db.ingestedRuleSamples.Tick() + } + + // Get label names + res, err := i.AllUserStats(context.Background(), &client.UserStatsRequest{}) + require.NoError(t, err) + + expect := []*client.UserIDStatsResponse{ + { + UserId: "user-1", + Data: &client.UserStatsResponse{ + IngestionRate: 0.2, + NumSeries: 3, + ApiIngestionRate: 0.2, + RuleIngestionRate: 0, + ActiveSeries: 3, + LoadedBlocks: 0, + }, + }, + { + UserId: "user-2", + Data: &client.UserStatsResponse{ + IngestionRate: 0.13333333333333333, + NumSeries: 2, + ApiIngestionRate: 0.13333333333333333, + RuleIngestionRate: 0, + ActiveSeries: 2, + LoadedBlocks: 0, + }, + }, + } + assert.ElementsMatch(t, expect, res.Stats) +} + +func TestIngesterPRW2_AllUserStatsHandler(t *testing.T) { + series := []struct { + user string + lbls labels.Labels + value float64 + timestamp int64 + }{ + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000}, + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_1"}, {Name: "status", Value: "500"}, {Name: "route", Value: "get_user"}}, 1, 110000}, + {"user-1", labels.Labels{{Name: labels.MetricName, Value: "test_1_2"}}, 2, 200000}, + {"user-2", labels.Labels{{Name: labels.MetricName, Value: "test_2_1"}}, 2, 200000}, + {"user-2", labels.Labels{{Name: labels.MetricName, Value: "test_2_2"}}, 2, 200000}, + } + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + for _, series := range series { + ctx := user.InjectOrgID(context.Background(), series.user) + req, _ := mockWriteRequestV2(t, series.lbls, series.value, series.timestamp) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + // Force compaction to test loaded blocks + compactionCallbackCh := make(chan struct{}) + i.TSDBState.forceCompactTrigger <- requestWithUsersAndCallback{users: nil, callback: compactionCallbackCh} + <-compactionCallbackCh + + // force update statistics + for _, db := range i.TSDBState.dbs { + db.ingestedAPISamples.Tick() + db.ingestedRuleSamples.Tick() + } + + // Get label names + response := httptest.NewRecorder() + request := httptest.NewRequest("GET", "/all_user_stats", nil) + request.Header.Add("Accept", "application/json") + i.AllUserStatsHandler(response, request) + var resp UserStatsByTimeseries + err = json.Unmarshal(response.Body.Bytes(), &resp) + require.NoError(t, err) + + expect := UserStatsByTimeseries{ + { + UserID: "user-1", + UserStats: UserStats{ + IngestionRate: 0.2, + NumSeries: 0, + APIIngestionRate: 0.2, + RuleIngestionRate: 0, + ActiveSeries: 3, + LoadedBlocks: 1, + }, + }, + { + UserID: "user-2", + UserStats: UserStats{ + IngestionRate: 0.13333333333333333, + NumSeries: 0, + APIIngestionRate: 0.13333333333333333, + RuleIngestionRate: 0, + ActiveSeries: 2, + LoadedBlocks: 1, + }, + }, + } + assert.ElementsMatch(t, expect, resp) +} + +func TestIngesterPRW2_CompactIdleBlock(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 1 + cfg.BlocksStorageConfig.TSDB.HeadCompactionInterval = 1 * time.Hour // Long enough to not be reached during the test. + cfg.BlocksStorageConfig.TSDB.HeadCompactionIdleTimeout = 1 * time.Second // Testing this. + + r := prometheus.NewRegistry() + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, r) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + pushSingleSampleWithMetadataV2(t, i) + + i.compactBlocks(context.Background(), false, nil) + verifyCompactedHead(t, i, false) + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="1"} 1 + + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="1"} 0 + + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + `), memSeriesCreatedTotalName, memSeriesRemovedTotalName, "cortex_ingester_memory_users")) + + // wait one second (plus maximum jitter) -- TSDB is now idle. + time.Sleep(time.Duration(float64(cfg.BlocksStorageConfig.TSDB.HeadCompactionIdleTimeout) * (1 + compactionIdleTimeoutJitter))) + + i.compactBlocks(context.Background(), false, nil) + verifyCompactedHead(t, i, true) + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="1"} 1 + + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="1"} 1 + + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + `), memSeriesCreatedTotalName, memSeriesRemovedTotalName, "cortex_ingester_memory_users")) + + // Pushing another sample still works. + pushSingleSampleWithMetadataV2(t, i) + verifyCompactedHead(t, i, false) + + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="1"} 2 + + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="1"} 1 + + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + `), memSeriesCreatedTotalName, memSeriesRemovedTotalName, "cortex_ingester_memory_users")) +} + +func TestIngesterPRW2_CompactAndCloseIdleTSDB(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipInterval = 1 * time.Second // Required to enable shipping. + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 1 + cfg.BlocksStorageConfig.TSDB.HeadCompactionInterval = 1 * time.Second + cfg.BlocksStorageConfig.TSDB.HeadCompactionIdleTimeout = 1 * time.Second + cfg.BlocksStorageConfig.TSDB.CloseIdleTSDBTimeout = 1 * time.Second + cfg.BlocksStorageConfig.TSDB.CloseIdleTSDBInterval = 100 * time.Millisecond + + r := prometheus.NewRegistry() + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, r) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + pushSingleSampleWithMetadataV2(t, i) + i.updateActiveSeries(context.Background()) + + require.Equal(t, int64(1), i.TSDBState.seriesCount.Load()) + + userMetrics := []string{memSeriesCreatedTotalName, memSeriesRemovedTotalName, "cortex_ingester_active_series"} + + globalMetrics := []string{"cortex_ingester_memory_users", "cortex_ingester_memory_metadata"} + metricsToCheck := append(userMetrics, globalMetrics...) + + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="1"} 1 + + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="1"} 0 + + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="1"} 1 + + # HELP cortex_ingester_memory_metadata The current number of metadata in memory. + # TYPE cortex_ingester_memory_metadata gauge + cortex_ingester_memory_metadata 1 + + # HELP cortex_ingester_memory_metadata_created_total The total number of metadata that were created per user + # TYPE cortex_ingester_memory_metadata_created_total counter + cortex_ingester_memory_metadata_created_total{user="1"} 1 + `), metricsToCheck...)) + + // Wait until TSDB has been closed and removed. + test.Poll(t, 10*time.Second, 0, func() interface{} { + i.stoppedMtx.Lock() + defer i.stoppedMtx.Unlock() + return len(i.TSDBState.dbs) + }) + + require.Greater(t, testutil.ToFloat64(i.TSDBState.idleTsdbChecks.WithLabelValues(string(tsdbIdleClosed))), float64(0)) + i.updateActiveSeries(context.Background()) + require.Equal(t, int64(0), i.TSDBState.seriesCount.Load()) // Flushing removed all series from memory. + + // Verify that user has disappeared from metrics. + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(""), userMetrics...)) + + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 0 + + # HELP cortex_ingester_memory_metadata The current number of metadata in memory. + # TYPE cortex_ingester_memory_metadata gauge + cortex_ingester_memory_metadata 0 + `), "cortex_ingester_memory_users", "cortex_ingester_memory_metadata")) + + // Pushing another sample will recreate TSDB. + pushSingleSampleWithMetadataV2(t, i) + i.updateActiveSeries(context.Background()) + + // User is back. + require.NoError(t, testutil.GatherAndCompare(r, strings.NewReader(` + # HELP cortex_ingester_memory_series_created_total The total number of series that were created per user. + # TYPE cortex_ingester_memory_series_created_total counter + cortex_ingester_memory_series_created_total{user="1"} 1 + + # HELP cortex_ingester_memory_series_removed_total The total number of series that were removed per user. + # TYPE cortex_ingester_memory_series_removed_total counter + cortex_ingester_memory_series_removed_total{user="1"} 0 + + # HELP cortex_ingester_memory_users The current number of users in memory. + # TYPE cortex_ingester_memory_users gauge + cortex_ingester_memory_users 1 + + # HELP cortex_ingester_active_series Number of currently active series per user. + # TYPE cortex_ingester_active_series gauge + cortex_ingester_active_series{user="1"} 1 + + # HELP cortex_ingester_memory_metadata The current number of metadata in memory. + # TYPE cortex_ingester_memory_metadata gauge + cortex_ingester_memory_metadata 1 + + # HELP cortex_ingester_memory_metadata_created_total The total number of metadata that were created per user + # TYPE cortex_ingester_memory_metadata_created_total counter + cortex_ingester_memory_metadata_created_total{user="1"} 1 + `), metricsToCheck...)) +} + +func TestIngesterPRW2_CloseTSDBsOnShutdown(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push some data. + pushSingleSampleWithMetadataV2(t, i) + + db := i.getTSDB(userID) + require.NotNil(t, db) + + // Stop ingester. + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) + + // Verify that DB is no longer in memory, but was closed + db = i.getTSDB(userID) + require.Nil(t, db) +} + +func TestIngesterPRW2_NotDeleteUnshippedBlocks(t *testing.T) { + chunkRange := 2 * time.Hour + chunkRangeMilliSec := chunkRange.Milliseconds() + cfg := defaultIngesterTestConfig(t) + cfg.BlocksStorageConfig.TSDB.BlockRanges = []time.Duration{chunkRange} + cfg.BlocksStorageConfig.TSDB.Retention = time.Millisecond // Which means delete all but first block. + cfg.LifecyclerConfig.JoinAfter = 0 + + // Create ingester + reg := prometheus.NewPedanticRegistry() + i, err := prepareIngesterWithBlocksStorage(t, cfg, reg) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_ingester_oldest_unshipped_block_timestamp_seconds Unix timestamp of the oldest TSDB block not shipped to the storage yet. 0 if ingester has no blocks or all blocks have been shipped. + # TYPE cortex_ingester_oldest_unshipped_block_timestamp_seconds gauge + cortex_ingester_oldest_unshipped_block_timestamp_seconds 0 + `), "cortex_ingester_oldest_unshipped_block_timestamp_seconds")) + + // Push some data to create 3 blocks. + ctx := user.InjectOrgID(context.Background(), userID) + for j := int64(0); j < 5; j++ { + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, j*chunkRangeMilliSec) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + + db := i.getTSDB(userID) + require.NotNil(t, db) + require.Nil(t, db.Compact(ctx)) + + oldBlocks := db.Blocks() + require.Equal(t, 3, len(oldBlocks)) + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(fmt.Sprintf(` + # HELP cortex_ingester_oldest_unshipped_block_timestamp_seconds Unix timestamp of the oldest TSDB block not shipped to the storage yet. 0 if ingester has no blocks or all blocks have been shipped. + # TYPE cortex_ingester_oldest_unshipped_block_timestamp_seconds gauge + cortex_ingester_oldest_unshipped_block_timestamp_seconds %d + `, oldBlocks[0].Meta().ULID.Time()/1000)), "cortex_ingester_oldest_unshipped_block_timestamp_seconds")) + + // Saying that we have shipped the second block, so only that should get deleted. + require.Nil(t, shipper.WriteMetaFile(nil, db.shipperMetadataFilePath, &shipper.Meta{ + Version: shipper.MetaVersion1, + Uploaded: []ulid.ULID{oldBlocks[1].Meta().ULID}, + })) + require.NoError(t, db.updateCachedShippedBlocks()) + + // Add more samples that could trigger another compaction and hence reload of blocks. + for j := int64(5); j < 6; j++ { + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, j*chunkRangeMilliSec) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + require.Nil(t, db.Compact(ctx)) + + // Only the second block should be gone along with a new block. + newBlocks := db.Blocks() + require.Equal(t, 3, len(newBlocks)) + require.Equal(t, oldBlocks[0].Meta().ULID, newBlocks[0].Meta().ULID) // First block remains same. + require.Equal(t, oldBlocks[2].Meta().ULID, newBlocks[1].Meta().ULID) // 3rd block becomes 2nd now. + require.NotEqual(t, oldBlocks[1].Meta().ULID, newBlocks[2].Meta().ULID) // The new block won't match previous 2nd block. + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(fmt.Sprintf(` + # HELP cortex_ingester_oldest_unshipped_block_timestamp_seconds Unix timestamp of the oldest TSDB block not shipped to the storage yet. 0 if ingester has no blocks or all blocks have been shipped. + # TYPE cortex_ingester_oldest_unshipped_block_timestamp_seconds gauge + cortex_ingester_oldest_unshipped_block_timestamp_seconds %d + `, newBlocks[0].Meta().ULID.Time()/1000)), "cortex_ingester_oldest_unshipped_block_timestamp_seconds")) + + // Shipping 2 more blocks, hence all the blocks from first round. + require.Nil(t, shipper.WriteMetaFile(nil, db.shipperMetadataFilePath, &shipper.Meta{ + Version: shipper.MetaVersion1, + Uploaded: []ulid.ULID{oldBlocks[1].Meta().ULID, newBlocks[0].Meta().ULID, newBlocks[1].Meta().ULID}, + })) + require.NoError(t, db.updateCachedShippedBlocks()) + + // Add more samples that could trigger another compaction and hence reload of blocks. + for j := int64(6); j < 7; j++ { + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, j*chunkRangeMilliSec) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + require.Nil(t, db.Compact(ctx)) + + // All blocks from the old blocks should be gone now. + newBlocks2 := db.Blocks() + require.Equal(t, 2, len(newBlocks2)) + + require.Equal(t, newBlocks[2].Meta().ULID, newBlocks2[0].Meta().ULID) // Block created in last round. + for _, b := range oldBlocks { + // Second block is not one among old blocks. + require.NotEqual(t, b.Meta().ULID, newBlocks2[1].Meta().ULID) + } + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(fmt.Sprintf(` + # HELP cortex_ingester_oldest_unshipped_block_timestamp_seconds Unix timestamp of the oldest TSDB block not shipped to the storage yet. 0 if ingester has no blocks or all blocks have been shipped. + # TYPE cortex_ingester_oldest_unshipped_block_timestamp_seconds gauge + cortex_ingester_oldest_unshipped_block_timestamp_seconds %d + `, newBlocks2[0].Meta().ULID.Time()/1000)), "cortex_ingester_oldest_unshipped_block_timestamp_seconds")) +} + +func TestIngesterPRW2_PushErrorDuringForcedCompaction(t *testing.T) { + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push a sample, it should succeed. + pushSingleSampleWithMetadataV2(t, i) + + // We mock a flushing by setting the boolean. + db := i.getTSDB(userID) + require.NotNil(t, db) + require.True(t, db.casState(active, forceCompacting)) + + // Ingestion should fail with a 503. + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, util.TimeToMillis(time.Now())) + ctx := user.InjectOrgID(context.Background(), userID) + _, err = i.PushV2(ctx, req) + require.Equal(t, httpgrpc.Errorf(http.StatusServiceUnavailable, wrapWithUser(errors.New("forced compaction in progress"), userID).Error()), err) + + // Ingestion is successful after a flush. + require.True(t, db.casState(forceCompacting, active)) + pushSingleSampleWithMetadata(t, i) +} + +func TestIngesterPRW2_NoFlushWithInFlightRequest(t *testing.T) { + registry := prometheus.NewRegistry() + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), registry) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + _ = services.StopAndAwaitTerminated(context.Background(), i) + }) + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push few samples. + for j := 0; j < 5; j++ { + pushSingleSampleWithMetadataV2(t, i) + } + + // Verifying that compaction won't happen when a request is in flight. + + // This mocks a request in flight. + db := i.getTSDB(userID) + require.NoError(t, db.acquireAppendLock()) + + // Flush handler only triggers compactions, but doesn't wait for them to finish. We cannot use ?wait=true here, + // because it would deadlock -- flush will wait for appendLock to be released. + i.FlushHandler(httptest.NewRecorder(), httptest.NewRequest("POST", "/flush", nil)) + + // Flushing should not have succeeded even after 5 seconds. + time.Sleep(5 * time.Second) + require.NoError(t, testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_ingester_tsdb_compactions_total Total number of TSDB compactions that were executed. + # TYPE cortex_ingester_tsdb_compactions_total counter + cortex_ingester_tsdb_compactions_total 0 + `), "cortex_ingester_tsdb_compactions_total")) + + // No requests in flight after this. + db.releaseAppendLock() + + // Let's wait until all head series have been flushed. + test.Poll(t, 5*time.Second, uint64(0), func() interface{} { + db := i.getTSDB(userID) + if db == nil { + return false + } + return db.Head().NumSeries() + }) + + require.NoError(t, testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_ingester_tsdb_compactions_total Total number of TSDB compactions that were executed. + # TYPE cortex_ingester_tsdb_compactions_total counter + cortex_ingester_tsdb_compactions_total 1 + `), "cortex_ingester_tsdb_compactions_total")) +} + +func TestIngesterPRW2_PushInstanceLimits(t *testing.T) { + tests := map[string]struct { + limits InstanceLimits + reqs map[string][]*cortexpbv2.WriteRequest + expectedErr error + expectedErrType interface{} + }{ + "should succeed creating one user and series": { + limits: InstanceLimits{MaxInMemorySeries: 1, MaxInMemoryTenants: 1}, + reqs: map[string][]*cortexpbv2.WriteRequest{ + "test": { + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test"}})}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + []cortexpbv2.Metadata{ + {Type: cortexpbv2.METRIC_TYPE_COUNTER, HelpRef: 3}, + }, + cortexpbv2.API, + "a help for metric_name_1"), + }, + }, + expectedErr: nil, + }, + + "should fail creating two series": { + limits: InstanceLimits{MaxInMemorySeries: 1, MaxInMemoryTenants: 1}, + reqs: map[string][]*cortexpbv2.WriteRequest{ + "test": { + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test1"}})}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test2"}})}, // another series + []cortexpbv2.Sample{{Value: 1, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + }, + }, + + expectedErr: wrapWithUser(errMaxSeriesLimitReached, "test"), + }, + + "should fail creating two users": { + limits: InstanceLimits{MaxInMemorySeries: 1, MaxInMemoryTenants: 1}, + reqs: map[string][]*cortexpbv2.WriteRequest{ + "user1": { + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test1"}})}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + }, + + "user2": { + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test2"}})}, // another series + []cortexpbv2.Sample{{Value: 1, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + }, + }, + expectedErr: wrapWithUser(errMaxUsersLimitReached, "user2"), + }, + + "should fail pushing samples in two requests due to rate limit": { + limits: InstanceLimits{MaxInMemorySeries: 1, MaxInMemoryTenants: 1, MaxIngestionRate: 0.001}, + reqs: map[string][]*cortexpbv2.WriteRequest{ + "user1": { + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test1"}})}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 9}}, + nil, + nil, + cortexpbv2.API), + + cortexpbv2.ToWriteRequestV2( + []labels.Labels{cortexpb.FromLabelAdaptersToLabels([]cortexpb.LabelAdapter{{Name: labels.MetricName, Value: "test1"}})}, + []cortexpbv2.Sample{{Value: 1, Timestamp: 10}}, + nil, + nil, + cortexpbv2.API), + }, + }, + expectedErr: errMaxSamplesPushRateLimitReached, + }, + } + + defaultInstanceLimits = nil + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.InstanceLimitsFn = func() *InstanceLimits { + return &testData.limits + } + + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Iterate through users in sorted order (by username). + uids := []string{} + totalPushes := 0 + for uid, requests := range testData.reqs { + uids = append(uids, uid) + totalPushes += len(requests) + } + sort.Strings(uids) + + pushIdx := 0 + for _, uid := range uids { + ctx := user.InjectOrgID(context.Background(), uid) + + for _, req := range testData.reqs[uid] { + pushIdx++ + _, err := i.PushV2(ctx, req) + + if pushIdx < totalPushes { + require.NoError(t, err) + } else { + // Last push may expect error. + if testData.expectedErr != nil { + assert.Equal(t, testData.expectedErr, err) + } else if testData.expectedErrType != nil { + assert.True(t, errors.As(err, testData.expectedErrType), "expected error type %T, got %v", testData.expectedErrType, err) + } else { + assert.NoError(t, err) + } + } + + // imitate time ticking between each push + i.ingestionRate.Tick() + + rate := testutil.ToFloat64(i.metrics.ingestionRate) + require.NotZero(t, rate) + } + } + }) + } +} + +func TestIngesterPRW2_inflightPushRequests(t *testing.T) { + limits := InstanceLimits{MaxInflightPushRequests: 1} + + // Create a mocked ingester + cfg := defaultIngesterTestConfig(t) + cfg.InstanceLimitsFn = func() *InstanceLimits { return &limits } + cfg.LifecyclerConfig.JoinAfter = 0 + + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until the ingester is ACTIVE + test.Poll(t, 100*time.Millisecond, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + ctx := user.InjectOrgID(context.Background(), "test") + + startCh := make(chan struct{}) + + g, ctx := errgroup.WithContext(ctx) + g.Go(func() error { + count := 3500000 + req := generateSamplesForLabelV2(labels.FromStrings(labels.MetricName, fmt.Sprintf("real-%d", count)), count) + // Signal that we're going to do the real push now. + close(startCh) + + _, err := i.PushV2(ctx, req) + return err + }) + + g.Go(func() error { + select { + case <-ctx.Done(): + // failed to setup + case <-startCh: + // we can start the test. + } + + time.Sleep(10 * time.Millisecond) // Give first goroutine a chance to start pushing... + req := generateSamplesForLabelV2(labels.FromStrings(labels.MetricName, "testcase"), 1024) + + _, err := i.PushV2(ctx, req) + require.Equal(t, errTooManyInflightPushRequests, err) + return nil + }) + + require.NoError(t, g.Wait()) +} + +func TestIngesterPRW2_QueryExemplar_MaxInflightQueryRequest(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + cfg.DefaultLimits.MaxInflightQueryRequests = 1 + i, err := prepareIngesterWithBlocksStorage(t, cfg, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + i.inflightQueryRequests.Add(1) + + // Mock request + ctx := user.InjectOrgID(context.Background(), "test") + + wreq, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "status", Value: "200"}, {Name: "route", Value: "get_user"}}, 1, 100000) + _, err = i.PushV2(ctx, wreq) + require.NoError(t, err) + + rreq := &client.ExemplarQueryRequest{} + _, err = i.QueryExemplars(ctx, rreq) + require.Error(t, err) + require.Equal(t, err, errTooManyInflightQueryRequests) +} + +func generateSamplesForLabelV2(lbs labels.Labels, count int) *cortexpbv2.WriteRequest { + var lbls = make([]labels.Labels, 0, count) + var samples = make([]cortexpbv2.Sample, 0, count) + + for i := 0; i < count; i++ { + samples = append(samples, cortexpbv2.Sample{ + Value: float64(i), + Timestamp: int64(i), + }) + lbls = append(lbls, lbs) + } + + return cortexpbv2.ToWriteRequestV2(lbls, samples, nil, nil, cortexpbv2.API) +} + +func mockWriteRequestWithMetadataV2(t *testing.T, lbls labels.Labels, value float64, timestamp int64, metadata cortexpbv2.Metadata, additionalSymbols ...string) (*cortexpbv2.WriteRequest, *client.QueryStreamResponse) { + samples := []cortexpbv2.Sample{ + { + Timestamp: timestamp, + Value: value, + }, + } + + req := cortexpbv2.ToWriteRequestV2([]labels.Labels{lbls}, samples, nil, []cortexpbv2.Metadata{metadata}, cortexpbv2.API, additionalSymbols...) + + chunk := chunkenc.NewXORChunk() + app, err := chunk.Appender() + require.NoError(t, err) + app.Append(timestamp, value) + chunk.Compact() + + expectedQueryStreamResChunks := &client.QueryStreamResponse{ + Chunkseries: []client.TimeSeriesChunk{ + { + Labels: cortexpb.FromLabelsToLabelAdapters(lbls), + Chunks: []client.Chunk{ + { + StartTimestampMs: timestamp, + EndTimestampMs: timestamp, + Encoding: int32(encoding.PrometheusXorChunk), + Data: chunk.Bytes(), + }, + }, + }, + }, + } + + return req, expectedQueryStreamResChunks +} + +func mockHistogramWriteRequestV2(t *testing.T, lbls labels.Labels, value int, timestampMs int64, float bool) (*cortexpbv2.WriteRequest, *client.QueryStreamResponse) { + var ( + histograms []cortexpbv2.Histogram + h *histogram.Histogram + fh *histogram.FloatHistogram + c chunkenc.Chunk + ) + if float { + fh = tsdbutil.GenerateTestFloatHistogram(value) + histograms = []cortexpbv2.Histogram{ + cortexpbv2.FloatHistogramToHistogramProto(timestampMs, fh), + } + c = chunkenc.NewFloatHistogramChunk() + } else { + h = tsdbutil.GenerateTestHistogram(value) + histograms = []cortexpbv2.Histogram{ + cortexpbv2.HistogramToHistogramProto(timestampMs, h), + } + c = chunkenc.NewHistogramChunk() + } + + app, err := c.Appender() + require.NoError(t, err) + if float { + _, _, _, err = app.AppendFloatHistogram(nil, timestampMs, fh, true) + } else { + _, _, _, err = app.AppendHistogram(nil, timestampMs, h, true) + } + require.NoError(t, err) + c.Compact() + + req := cortexpbv2.ToWriteRequestV2([]labels.Labels{lbls}, nil, histograms, nil, cortexpbv2.API) + enc := int32(encoding.PrometheusHistogramChunk) + if float { + enc = int32(encoding.PrometheusFloatHistogramChunk) + } + expectedQueryStreamResChunks := &client.QueryStreamResponse{ + Chunkseries: []client.TimeSeriesChunk{ + { + Labels: cortexpb.FromLabelsToLabelAdapters(lbls), + Chunks: []client.Chunk{ + { + StartTimestampMs: timestampMs, + EndTimestampMs: timestampMs, + Encoding: enc, + Data: c.Bytes(), + }, + }, + }, + }, + } + + return req, expectedQueryStreamResChunks +} + +func mockWriteRequestV2(t *testing.T, lbls labels.Labels, value float64, timestamp int64) (*cortexpbv2.WriteRequest, *client.QueryStreamResponse) { + samples := []cortexpbv2.Sample{ + { + Timestamp: timestamp, + Value: value, + }, + } + + req := cortexpbv2.ToWriteRequestV2([]labels.Labels{lbls}, samples, nil, nil, cortexpbv2.API) + + chunk := chunkenc.NewXORChunk() + app, err := chunk.Appender() + require.NoError(t, err) + app.Append(timestamp, value) + chunk.Compact() + + expectedQueryStreamResChunks := &client.QueryStreamResponse{ + Chunkseries: []client.TimeSeriesChunk{ + { + Labels: cortexpb.FromLabelsToLabelAdapters(lbls), + Chunks: []client.Chunk{ + { + StartTimestampMs: timestamp, + EndTimestampMs: timestamp, + Encoding: int32(encoding.PrometheusXorChunk), + Data: chunk.Bytes(), + }, + }, + }, + }, + } + + return req, expectedQueryStreamResChunks +} + +func pushSingleSampleWithMetadataV2(t *testing.T, i *Ingester) { + ctx := user.InjectOrgID(context.Background(), userID) + metadata := cortexpbv2.Metadata{ + Type: cortexpbv2.METRIC_TYPE_COUNTER, + HelpRef: 3, + UnitRef: 0, + } + + req, _ := mockWriteRequestWithMetadataV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, util.TimeToMillis(time.Now()), metadata, "a help for metric") + _, err := i.PushV2(ctx, req) + require.NoError(t, err) +} + +func pushSingleSampleAtTimeV2(t *testing.T, i *Ingester, ts int64) { + ctx := user.InjectOrgID(context.Background(), userID) + req, _ := mockWriteRequestV2(t, labels.Labels{{Name: labels.MetricName, Value: "test"}}, 0, ts) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) +} + +func writeRequestSingleSeriesV2(lbls labels.Labels, samples []cortexpbv2.Sample) *cortexpbv2.WriteRequest { + req := &cortexpbv2.WriteRequest{ + Source: cortexpbv2.API, + } + + st := writev2.NewSymbolTable() + ts := cortexpbv2.TimeSeries{} + ts.Samples = samples + ts.LabelsRefs = st.SymbolizeLabels(lbls, nil) + req.Timeseries = append(req.Timeseries, cortexpbv2.PreallocTimeseriesV2{TimeSeries: &ts}) + req.Symbols = st.Symbols() + + return req +} + +// createIngesterWithSeries creates an ingester and push numSeries with numSamplesPerSeries each. +func createIngesterWithSeriesV2(t testing.TB, userID string, numSeries, numSamplesPerSeries int, startTimestamp, step int64) *Ingester { + const maxBatchSize = 1000 + + // Create ingester. + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), i)) + }) + + // Wait until it's ACTIVE. + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push fixtures. + ctx := user.InjectOrgID(context.Background(), userID) + + for ts := startTimestamp; ts < startTimestamp+(step*int64(numSamplesPerSeries)); ts += step { + for o := 0; o < numSeries; o += maxBatchSize { + batchSize := min(maxBatchSize, numSeries-o) + + // Generate metrics and samples (1 for each series). + metrics := make([]labels.Labels, 0, batchSize) + samples := make([]cortexpbv2.Sample, 0, batchSize) + + for s := 0; s < batchSize; s++ { + metrics = append(metrics, labels.Labels{ + {Name: labels.MetricName, Value: fmt.Sprintf("test_%d", o+s)}, + }) + + samples = append(samples, cortexpbv2.Sample{ + Timestamp: ts, + Value: 1, + }) + } + + // Send metrics to the ingester. + req := cortexpbv2.ToWriteRequestV2(metrics, samples, nil, nil, cortexpbv2.API) + _, err := i.PushV2(ctx, req) + require.NoError(t, err) + } + } + + return i +} + +func benchmarkDataV2(nSeries int) (allLabels []labels.Labels, allSamples []cortexpbv2.Sample) { + for j := 0; j < nSeries; j++ { + labels := chunk.BenchmarkLabels.Copy() + for i := range labels { + if labels[i].Name == "cpu" { + labels[i].Value = fmt.Sprintf("cpu%02d", j) + } + } + allLabels = append(allLabels, labels) + allSamples = append(allSamples, cortexpbv2.Sample{Timestamp: 0, Value: float64(j)}) + } + return +} diff --git a/pkg/util/validation/validate.go b/pkg/util/validation/validate.go index be94cfa2f1..dda50dddb3 100644 --- a/pkg/util/validation/validate.go +++ b/pkg/util/validation/validate.go @@ -12,9 +12,12 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/cortexpbv2" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/extract" ) @@ -24,6 +27,7 @@ const ( errMetadataMissingMetricName = "metadata missing metric name" errMetadataTooLong = "metadata '%s' value too long: %.200q metric %.200q" + errMetadataV2TooLong = "metadata '%s' value too long: %.200q" typeMetricName = "METRIC_NAME" typeHelp = "HELP" @@ -148,6 +152,48 @@ func ValidateSampleTimestamp(validateMetrics *ValidateMetrics, limits *Limits, u return nil } +// ValidateExemplarV2 returns an error if the exemplar is invalid. +// The returned error may retain the provided series labels. +func ValidateExemplarV2(validateMetrics *ValidateMetrics, symbols []string, userID string, seriesLabels []cortexpb.LabelAdapter, e *cortexpbv2.Exemplar, b labels.ScratchBuilder, st *writev2.SymbolsTable) ValidationError { + lbs := e.ToLabels(&b, symbols) + // symbolize examplar labels + e.LabelsRefs = st.SymbolizeLabels(lbs, nil) + exemplarLabels := cortexpb.FromLabelsToLabelAdapters(lbs) + + if len(exemplarLabels) <= 0 { + validateMetrics.DiscardedExemplars.WithLabelValues(exemplarLabelsMissing, userID).Inc() + return newExemplarEmtpyLabelsError(seriesLabels, []cortexpb.LabelAdapter{}, e.Timestamp) + } + + if e.Timestamp == 0 { + validateMetrics.DiscardedExemplars.WithLabelValues(exemplarTimestampInvalid, userID).Inc() + return newExemplarMissingTimestampError( + seriesLabels, + exemplarLabels, + e.Timestamp, + ) + } + + // Exemplar label length does not include chars involved in text + // rendering such as quotes, commas, etc. See spec and const definition. + labelSetLen := 0 + for _, l := range exemplarLabels { + labelSetLen += utf8.RuneCountInString(l.Name) + labelSetLen += utf8.RuneCountInString(l.Value) + } + + if labelSetLen > ExemplarMaxLabelSetLength { + validateMetrics.DiscardedExemplars.WithLabelValues(exemplarLabelsTooLong, userID).Inc() + return newExemplarLabelLengthError( + seriesLabels, + exemplarLabels, + e.Timestamp, + ) + } + + return nil +} + // ValidateExemplar returns an error if the exemplar is invalid. // The returned error may retain the provided series labels. func ValidateExemplar(validateMetrics *ValidateMetrics, userID string, ls []cortexpb.LabelAdapter, e cortexpb.Exemplar) ValidationError { @@ -243,6 +289,37 @@ func ValidateLabels(validateMetrics *ValidateMetrics, limits *Limits, userID str return nil } +// ValidateMetadata returns an err if a metric metadata is invalid. +func ValidateMetadataV2(validateMetrics *ValidateMetrics, cfg *Limits, userID string, symbols []string, metadata *cortexpbv2.Metadata, st *writev2.SymbolsTable) error { + help := symbols[metadata.HelpRef] + unit := symbols[metadata.UnitRef] + + // symbolize help and unit + metadata.HelpRef = st.Symbolize(help) + metadata.UnitRef = st.Symbolize(unit) + + maxMetadataValueLength := cfg.MaxMetadataLength + var reason string + var cause string + var metadataType string + if len(help) > maxMetadataValueLength { + metadataType = typeHelp + reason = helpTooLong + cause = help + } else if len(unit) > maxMetadataValueLength { + metadataType = typeUnit + reason = unitTooLong + cause = unit + } + + if reason != "" { + validateMetrics.DiscardedMetadata.WithLabelValues(reason, userID).Inc() + return httpgrpc.Errorf(http.StatusBadRequest, errMetadataV2TooLong, metadataType, cause) + } + + return nil +} + // ValidateMetadata returns an err if a metric metadata is invalid. func ValidateMetadata(validateMetrics *ValidateMetrics, cfg *Limits, userID string, metadata *cortexpb.MetricMetadata) error { if cfg.EnforceMetadataMetricName && metadata.GetMetricFamilyName() == "" { @@ -276,6 +353,67 @@ func ValidateMetadata(validateMetrics *ValidateMetrics, cfg *Limits, userID stri return nil } +func ValidateNativeHistogramV2(validateMetrics *ValidateMetrics, limits *Limits, userID string, ls []cortexpb.LabelAdapter, histogramSample cortexpbv2.Histogram) (cortexpbv2.Histogram, error) { + if limits.MaxNativeHistogramBuckets == 0 { + return histogramSample, nil + } + + var ( + exceedLimit bool + ) + if histogramSample.IsFloatHistogram() { + // Initial check to see if the bucket limit is exceeded or not. If not, we can avoid type casting. + exceedLimit = len(histogramSample.PositiveCounts)+len(histogramSample.NegativeCounts) > limits.MaxNativeHistogramBuckets + if !exceedLimit { + return histogramSample, nil + } + // Exceed limit. + if histogramSample.Schema <= histogram.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() + return cortexpbv2.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + fh := cortexpbv2.FloatHistogramProtoToFloatHistogram(histogramSample) + oBuckets := len(fh.PositiveBuckets) + len(fh.NegativeBuckets) + for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > limits.MaxNativeHistogramBuckets { + if fh.Schema <= histogram.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() + return cortexpbv2.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + fh = fh.ReduceResolution(fh.Schema - 1) + } + if oBuckets != len(fh.PositiveBuckets)+len(fh.NegativeBuckets) { + validateMetrics.HistogramSamplesReducedResolution.WithLabelValues(userID).Inc() + } + // If resolution reduced, convert new float histogram to protobuf type again. + return cortexpbv2.FloatHistogramToHistogramProto(histogramSample.Timestamp, fh), nil + } + + // Initial check to see if bucket limit is exceeded or not. If not, we can avoid type casting. + exceedLimit = len(histogramSample.PositiveDeltas)+len(histogramSample.NegativeDeltas) > limits.MaxNativeHistogramBuckets + if !exceedLimit { + return histogramSample, nil + } + // Exceed limit. + if histogramSample.Schema <= histogram.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() + return cortexpbv2.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + h := cortexpbv2.HistogramProtoToHistogram(histogramSample) + oBuckets := len(h.PositiveBuckets) + len(h.NegativeBuckets) + for len(h.PositiveBuckets)+len(h.NegativeBuckets) > limits.MaxNativeHistogramBuckets { + if h.Schema <= histogram.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() + return cortexpbv2.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + h = h.ReduceResolution(h.Schema - 1) + } + if oBuckets != len(h.PositiveBuckets)+len(h.NegativeBuckets) { + validateMetrics.HistogramSamplesReducedResolution.WithLabelValues(userID).Inc() + } + // If resolution reduced, convert new histogram to protobuf type again. + return cortexpbv2.HistogramToHistogramProto(histogramSample.Timestamp, h), nil +} + func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, userID string, ls []cortexpb.LabelAdapter, histogramSample cortexpb.Histogram) (cortexpb.Histogram, error) { if limits.MaxNativeHistogramBuckets == 0 { return histogramSample, nil