[PRW-2.0] (part 2) Removed automatic negotiation, updates for the latest spec semantics in remote pkg (#14329)

* [PRW-2.0] (part2) Moved to latest basic negotiation & spec semantics.

Spec: https://github.com/prometheus/docs/pull/2462

Supersedes https://github.com/prometheus/prometheus/pull/13968

Signed-off-by: bwplotka <bwplotka@gmail.com>

# Conflicts:
#	config/config.go
#	docs/configuration/configuration.md
#	storage/remote/queue_manager_test.go
#	storage/remote/write.go
#	web/api/v1/api.go

* Addressed comments.

Signed-off-by: bwplotka <bwplotka@gmail.com>

---------

Signed-off-by: bwplotka <bwplotka@gmail.com>
This commit is contained in:
Bartlomiej Plotka 2024-06-26 12:20:01 +02:00 committed by GitHub
parent 2b348d43b7
commit 4dbcdd19da
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
19 changed files with 1016 additions and 1092 deletions

View file

@ -1747,6 +1747,7 @@ func rwProtoMsgFlagValue(msgs *[]config.RemoteWriteProtoMsg) kingpin.Value {
return &rwProtoMsgFlagParser{msgs: msgs}
}
// IsCumulative is used by kingpin to tell if it's an array or not.
func (p *rwProtoMsgFlagParser) IsCumulative() bool {
return true
}

View file

@ -116,7 +116,7 @@ func parseAndPushMetrics(client *remote.Client, data []byte, labels map[string]s
// Encode the request body into snappy encoding.
compressed := snappy.Encode(nil, raw)
err = client.Store(context.Background(), compressed, 0, remote.Version1, "snappy")
err = client.Store(context.Background(), compressed, 0)
if err != nil {
fmt.Fprintln(os.Stderr, " FAILED:", err)
return false

View file

@ -1026,9 +1026,6 @@ func CheckTargetAddress(address model.LabelValue) error {
return nil
}
// TODO(bwplotka): Remove in the next PRs (review split PR for readability).
type RemoteWriteFormat int64
// RemoteWriteProtoMsg represents the known protobuf message for the remote write
// 1.0 and 2.0 specs.
type RemoteWriteProtoMsg string
@ -1054,7 +1051,7 @@ func (m RemoteWriteProtoMsgs) Strings() []string {
}
func (m RemoteWriteProtoMsgs) String() string {
return strings.Join(m.Strings(), ",")
return strings.Join(m.Strings(), ", ")
}
var (

View file

@ -1798,7 +1798,7 @@ var expectedErrors = []struct {
},
{
filename: "remote_write_wrong_msg.bad.yml",
errMsg: `invalid protobuf_message value: unknown remote write protobuf message io.prometheus.writet.v2.Request, supported: prometheus.WriteRequest,io.prometheus.write.v2.Request`,
errMsg: `invalid protobuf_message value: unknown remote write protobuf message io.prometheus.writet.v2.Request, supported: prometheus.WriteRequest, io.prometheus.write.v2.Request`,
},
{
filename: "remote_write_url_missing.bad.yml",

View file

@ -26,7 +26,7 @@ The Prometheus monitoring server
| <code class="text-nowrap">--web.enable-lifecycle</code> | Enable shutdown and reload via HTTP request. | `false` |
| <code class="text-nowrap">--web.enable-admin-api</code> | Enable API endpoints for admin control actions. | `false` |
| <code class="text-nowrap">--web.enable-remote-write-receiver</code> | Enable API endpoint accepting remote write requests. | `false` |
| <code class="text-nowrap">--web.remote-write-receiver.accepted-protobuf-messages</code> | List of the remote write protobuf messages to accept when receiving the remote writes. Supported values: prometheus.WriteRequest,io.prometheus.write.v2.Request | `prometheus.WriteRequest` |
| <code class="text-nowrap">--web.remote-write-receiver.accepted-protobuf-messages</code> | List of the remote write protobuf messages to accept when receiving the remote writes. Supported values: prometheus.WriteRequest, io.prometheus.write.v2.Request | `prometheus.WriteRequest` |
| <code class="text-nowrap">--web.console.templates</code> | Path to the console template directory, available at /consoles. | `consoles` |
| <code class="text-nowrap">--web.console.libraries</code> | Path to the console library directory. | `console_libraries` |
| <code class="text-nowrap">--web.page-title</code> | Document title of Prometheus instance. | `Prometheus Time Series Collection and Processing Server` |

View file

@ -0,0 +1,83 @@
// Copyright 2024 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package writev2
import "github.com/prometheus/prometheus/model/labels"
// SymbolsTable implements table for easy symbol use.
type SymbolsTable struct {
strings []string
symbolsMap map[string]uint32
}
// NewSymbolTable returns a symbol table.
func NewSymbolTable() SymbolsTable {
return SymbolsTable{
// Empty string is required as a first element.
symbolsMap: map[string]uint32{"": 0},
strings: []string{""},
}
}
// Symbolize adds (if not added before) a string to the symbols table,
// while returning its reference number.
func (t *SymbolsTable) Symbolize(str string) uint32 {
if ref, ok := t.symbolsMap[str]; ok {
return ref
}
ref := uint32(len(t.strings))
t.strings = append(t.strings, str)
t.symbolsMap[str] = ref
return ref
}
// SymbolizeLabels symbolize Prometheus labels.
func (t *SymbolsTable) SymbolizeLabels(lbls labels.Labels, buf []uint32) []uint32 {
result := buf[:0]
lbls.Range(func(l labels.Label) {
off := t.Symbolize(l.Name)
result = append(result, off)
off = t.Symbolize(l.Value)
result = append(result, off)
})
return result
}
// Symbols returns computes symbols table to put in e.g. Request.Symbols.
// As per spec, order does not matter.
func (t *SymbolsTable) Symbols() []string {
return t.strings
}
// Reset clears symbols table.
func (t *SymbolsTable) Reset() {
// NOTE: Make sure to keep empty symbol.
t.strings = t.strings[:1]
for k := range t.symbolsMap {
if k == "" {
continue
}
delete(t.symbolsMap, k)
}
}
// DesymbolizeLabels decodes label references, with given symbols to labels.
func DesymbolizeLabels(labelRefs []uint32, symbols []string) labels.Labels {
b := labels.NewScratchBuilder(len(labelRefs))
for i := 0; i < len(labelRefs); i += 2 {
b.Add(symbols[labelRefs[i]], symbols[labelRefs[i+1]])
}
b.Sort()
return b.Labels()
}

View file

@ -0,0 +1,59 @@
// Copyright 2024 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package writev2
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/prometheus/prometheus/model/labels"
)
func TestSymbolsTable(t *testing.T) {
s := NewSymbolTable()
require.Equal(t, []string{""}, s.Symbols(), "required empty reference does not exist")
require.Equal(t, uint32(0), s.Symbolize(""))
require.Equal(t, []string{""}, s.Symbols())
require.Equal(t, uint32(1), s.Symbolize("abc"))
require.Equal(t, []string{"", "abc"}, s.Symbols())
require.Equal(t, uint32(2), s.Symbolize("__name__"))
require.Equal(t, []string{"", "abc", "__name__"}, s.Symbols())
require.Equal(t, uint32(3), s.Symbolize("foo"))
require.Equal(t, []string{"", "abc", "__name__", "foo"}, s.Symbols())
s.Reset()
require.Equal(t, []string{""}, s.Symbols(), "required empty reference does not exist")
require.Equal(t, uint32(0), s.Symbolize(""))
require.Equal(t, uint32(1), s.Symbolize("__name__"))
require.Equal(t, []string{"", "__name__"}, s.Symbols())
require.Equal(t, uint32(2), s.Symbolize("abc"))
require.Equal(t, []string{"", "__name__", "abc"}, s.Symbols())
ls := labels.FromStrings("__name__", "qwer", "zxcv", "1234")
encoded := s.SymbolizeLabels(ls, nil)
require.Equal(t, []uint32{1, 3, 4, 5}, encoded)
decoded := DesymbolizeLabels(encoded, s.Symbols())
require.Equal(t, ls, decoded)
// Different buf.
ls = labels.FromStrings("__name__", "qwer", "zxcv2222", "1234")
encoded = s.SymbolizeLabels(ls, []uint32{1, 3, 4, 5})
require.Equal(t, []uint32{1, 3, 6, 5}, encoded)
}

View file

@ -19,7 +19,6 @@ import (
"context"
"fmt"
"io"
"math/rand"
"net/http"
"strconv"
"strings"
@ -44,32 +43,33 @@ import (
const maxErrMsgLen = 1024
var UserAgent = fmt.Sprintf("Prometheus/%s", version.Version)
const (
RemoteWriteVersionHeader = "X-Prometheus-Remote-Write-Version"
RemoteWriteVersion1HeaderValue = "0.1.0"
RemoteWriteVersion20HeaderValue = "2.0.0"
appProtoContentType = "application/x-protobuf"
)
// If we send a Remote Write 2.0 request to a Remote Write endpoint that only understands
// Remote Write 1.0 it will respond with an error. We need to handle these errors
// accordingly. Any 5xx errors will just need to be retried as they are considered
// transient/recoverable errors. A 4xx error will need to be passed back to the queue
// manager in order to be re-encoded in a suitable format.
// Compression represents the encoding. Currently remote storage supports only
// one, but we experiment with more, thus leaving the compression scaffolding
// for now.
// NOTE(bwplotka): Keeping it public, as a non-stable help for importers to use.
type Compression string
// A Remote Write 2.0 request sent to, for example, a Prometheus 2.50 receiver (which does
// not understand Remote Write 2.0) will result in an HTTP 400 status code from the receiver.
const (
// SnappyBlockCompression represents https://github.com/google/snappy/blob/2c94e11145f0b7b184b831577c93e5a41c4c0346/format_description.txt
SnappyBlockCompression Compression = "snappy"
)
// A Remote Write 2.0 request sent to a remote write receiver may (depending on receiver version)
// result in an HTTP 406 status code to indicate that it does not accept the protocol or
// encoding of that request and that the sender should retry with a more suitable protocol
// version or encoding.
var (
// UserAgent represents Prometheus version to use for user agent header.
UserAgent = fmt.Sprintf("Prometheus/%s", version.Version)
// We bundle any error we want to return to the queue manager to trigger a renegotiation into
// this custom error.
type ErrRenegotiate struct {
FirstLine string
StatusCode int
}
func (r *ErrRenegotiate) Error() string {
return fmt.Sprintf("HTTP %d: msg: %s", r.StatusCode, r.FirstLine)
}
remoteWriteContentTypeHeaders = map[config.RemoteWriteProtoMsg]string{
config.RemoteWriteProtoMsgV1: appProtoContentType, // Also application/x-protobuf;proto=prometheus.WriteRequest but simplified for compatibility with 1.x spec.
config.RemoteWriteProtoMsgV2: appProtoContentType + ";proto=io.prometheus.write.v2.Request",
}
)
var (
remoteReadQueriesTotal = prometheus.NewCounterVec(
@ -111,29 +111,31 @@ func init() {
// Client allows reading and writing from/to a remote HTTP endpoint.
type Client struct {
remoteName string // Used to differentiate clients in metrics.
urlString string // url.String()
lastRWHeader string
Client *http.Client
timeout time.Duration
remoteName string // Used to differentiate clients in metrics.
urlString string // url.String()
Client *http.Client
timeout time.Duration
retryOnRateLimit bool
readQueries prometheus.Gauge
readQueriesTotal *prometheus.CounterVec
readQueriesDuration prometheus.Observer
writeProtoMsg config.RemoteWriteProtoMsg
writeCompression Compression // Not exposed by ClientConfig for now.
}
// ClientConfig configures a client.
type ClientConfig struct {
URL *config_util.URL
RemoteWriteFormat config.RemoteWriteFormat
Timeout model.Duration
HTTPClientConfig config_util.HTTPClientConfig
SigV4Config *sigv4.SigV4Config
AzureADConfig *azuread.AzureADConfig
Headers map[string]string
RetryOnRateLimit bool
URL *config_util.URL
Timeout model.Duration
HTTPClientConfig config_util.HTTPClientConfig
SigV4Config *sigv4.SigV4Config
AzureADConfig *azuread.AzureADConfig
Headers map[string]string
RetryOnRateLimit bool
WriteProtoMsg config.RemoteWriteProtoMsg
}
// ReadClient uses the SAMPLES method of remote read to read series samples from remote server.
@ -192,14 +194,20 @@ func NewWriteClient(name string, conf *ClientConfig) (WriteClient, error) {
}
}
httpClient.Transport = otelhttp.NewTransport(t)
writeProtoMsg := config.RemoteWriteProtoMsgV1
if conf.WriteProtoMsg != "" {
writeProtoMsg = conf.WriteProtoMsg
}
httpClient.Transport = otelhttp.NewTransport(t)
return &Client{
remoteName: name,
urlString: conf.URL.String(),
Client: httpClient,
retryOnRateLimit: conf.RetryOnRateLimit,
timeout: time.Duration(conf.Timeout),
writeProtoMsg: writeProtoMsg,
writeCompression: SnappyBlockCompression,
}, nil
}
@ -226,58 +234,9 @@ type RecoverableError struct {
retryAfter model.Duration
}
// Attempt a HEAD request against a remote write endpoint to see what it supports.
func (c *Client) probeRemoteVersions(ctx context.Context) error {
// We assume we are in Version2 mode otherwise we shouldn't be calling this.
httpReq, err := http.NewRequest(http.MethodHead, c.urlString, nil)
if err != nil {
// Errors from NewRequest are from unparsable URLs, so are not
// recoverable.
return err
}
// Set the version header to be nice.
httpReq.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
httpReq.Header.Set("User-Agent", UserAgent)
ctx, cancel := context.WithTimeout(ctx, c.timeout)
defer cancel()
httpResp, err := c.Client.Do(httpReq.WithContext(ctx))
if err != nil {
// We don't attempt a retry here.
return err
}
// See if we got a header anyway.
promHeader := httpResp.Header.Get(RemoteWriteVersionHeader)
// Only update lastRWHeader if the X-Prometheus-Remote-Write header is not blank.
if promHeader != "" {
c.lastRWHeader = promHeader
}
// Check for an error.
if httpResp.StatusCode != http.StatusOK {
if httpResp.StatusCode == http.StatusMethodNotAllowed {
// If we get a 405 (MethodNotAllowed) error then it means the endpoint doesn't
// understand Remote Write 2.0, so we allow the lastRWHeader to be overwritten
// even if it is blank.
// This will make subsequent sends use RemoteWrite 1.0 until the endpoint gives
// a response that confirms it can speak 2.0.
c.lastRWHeader = promHeader
}
return fmt.Errorf(httpResp.Status)
}
// All ok, return no error.
return nil
}
// Store sends a batch of samples to the HTTP endpoint, the request is the proto marshalled
// and encoded bytes from codec.go.
func (c *Client) Store(ctx context.Context, req []byte, attempt int, rwFormat config.RemoteWriteFormat, compression string) error {
func (c *Client) Store(ctx context.Context, req []byte, attempt int) error {
httpReq, err := http.NewRequest(http.MethodPost, c.urlString, bytes.NewReader(req))
if err != nil {
// Errors from NewRequest are from unparsable URLs, so are not
@ -285,14 +244,13 @@ func (c *Client) Store(ctx context.Context, req []byte, attempt int, rwFormat co
return err
}
httpReq.Header.Add("Content-Encoding", compression)
httpReq.Header.Set("Content-Type", "application/x-protobuf")
httpReq.Header.Add("Content-Encoding", string(c.writeCompression))
httpReq.Header.Set("Content-Type", remoteWriteContentTypeHeaders[c.writeProtoMsg])
httpReq.Header.Set("User-Agent", UserAgent)
if rwFormat == Version1 {
if c.writeProtoMsg == config.RemoteWriteProtoMsgV1 {
// Compatibility mode for 1.0.
httpReq.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion1HeaderValue)
} else {
// Set the right header if we're using v2.0 remote write protocol.
httpReq.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
}
@ -317,32 +275,13 @@ func (c *Client) Store(ctx context.Context, req []byte, attempt int, rwFormat co
httpResp.Body.Close()
}()
// See if we got a X-Prometheus-Remote-Write header in the response.
if promHeader := httpResp.Header.Get(RemoteWriteVersionHeader); promHeader != "" {
// Only update lastRWHeader if the X-Prometheus-Remote-Write header is not blank.
// (It's blank if it wasn't present, we don't care about that distinction.)
c.lastRWHeader = promHeader
}
if httpResp.StatusCode/100 != 2 {
scanner := bufio.NewScanner(io.LimitReader(httpResp.Body, maxErrMsgLen))
line := ""
if scanner.Scan() {
line = scanner.Text()
}
switch httpResp.StatusCode {
case http.StatusBadRequest:
// Return an unrecoverable error to indicate the 400.
// This then gets passed up the chain so we can react to it properly.
return &ErrRenegotiate{line, httpResp.StatusCode}
case http.StatusNotAcceptable:
// Return an unrecoverable error to indicate the 406.
// This then gets passed up the chain so we can react to it properly.
return &ErrRenegotiate{line, httpResp.StatusCode}
default:
// We want to end up returning a non-specific error.
err = fmt.Errorf("server returned HTTP status %s: %s", httpResp.Status, line)
}
err = fmt.Errorf("server returned HTTP status %s: %s", httpResp.Status, line)
}
if httpResp.StatusCode/100 == 5 ||
(c.retryOnRateLimit && httpResp.StatusCode == http.StatusTooManyRequests) {
@ -368,19 +307,15 @@ func retryAfterDuration(t string) model.Duration {
}
// Name uniquely identifies the client.
func (c Client) Name() string {
func (c *Client) Name() string {
return c.remoteName
}
// Endpoint is the remote read or write endpoint.
func (c Client) Endpoint() string {
func (c *Client) Endpoint() string {
return c.urlString
}
func (c *Client) GetLastRWHeader() string {
return c.lastRWHeader
}
// Read reads from a remote endpoint.
func (c *Client) Read(ctx context.Context, query *prompb.Query) (*prompb.QueryResult, error) {
c.readQueries.Inc()
@ -453,34 +388,3 @@ func (c *Client) Read(ctx context.Context, query *prompb.Query) (*prompb.QueryRe
return resp.Results[0], nil
}
type TestClient struct {
name string
url string
}
func NewTestClient(name, url string) WriteClient {
return &TestClient{name: name, url: url}
}
func (c *TestClient) probeRemoteVersions(_ context.Context) error {
return nil
}
func (c *TestClient) Store(_ context.Context, req []byte, _ int, _ config.RemoteWriteFormat, _ string) error {
r := rand.Intn(200-100) + 100
time.Sleep(time.Duration(r) * time.Millisecond)
return nil
}
func (c *TestClient) Name() string {
return c.name
}
func (c *TestClient) Endpoint() string {
return c.url
}
func (c *TestClient) GetLastRWHeader() string {
return "2.0;snappy,0.1.0"
}

View file

@ -73,7 +73,7 @@ func TestStoreHTTPErrorHandling(t *testing.T) {
c, err := NewWriteClient(hash, conf)
require.NoError(t, err)
err = c.Store(context.Background(), []byte{}, 0, Version1, "snappy")
err = c.Store(context.Background(), []byte{}, 0)
if test.err != nil {
require.EqualError(t, err, test.err.Error())
} else {
@ -133,7 +133,7 @@ func TestClientRetryAfter(t *testing.T) {
c := getClient(getClientConfig(serverURL, tc.retryOnRateLimit))
var recErr RecoverableError
err = c.Store(context.Background(), []byte{}, 0, Version1, "snappy")
err = c.Store(context.Background(), []byte{}, 0)
require.Equal(t, tc.expectedRecoverable, errors.As(err, &recErr), "Mismatch in expected recoverable error status.")
if tc.expectedRecoverable {
require.Equal(t, tc.expectedRetryAfter, recErr.retryAfter)
@ -203,7 +203,7 @@ func TestClientHeaders(t *testing.T) {
c, err := NewWriteClient("c", conf)
require.NoError(t, err)
err = c.Store(context.Background(), []byte{}, 0, Version1, "snappy")
err = c.Store(context.Background(), []byte{}, 0)
require.NoError(t, err)
require.True(t, called, "The remote server wasn't called")

View file

@ -634,7 +634,7 @@ func exemplarProtoV2ToExemplar(ep writev2.Exemplar, symbols []string) exemplar.E
timestamp := ep.Timestamp
return exemplar.Exemplar{
Labels: labelProtosV2ToLabels(ep.LabelsRefs, symbols),
Labels: writev2.DesymbolizeLabels(ep.LabelsRefs, symbols),
Value: ep.Value,
Ts: timestamp,
HasTs: timestamp != 0,
@ -756,7 +756,7 @@ func HistogramProtoToFloatHistogram(hp prompb.Histogram) *histogram.FloatHistogr
}
}
func FloatMinHistogramProtoToFloatHistogram(hp writev2.Histogram) *histogram.FloatHistogram {
func FloatV2HistogramProtoToFloatHistogram(hp writev2.Histogram) *histogram.FloatHistogram {
if !hp.IsFloatHistogram() {
panic("FloatHistogramProtoToFloatHistogram called with an integer histogram")
}
@ -774,10 +774,10 @@ func FloatMinHistogramProtoToFloatHistogram(hp writev2.Histogram) *histogram.Flo
}
}
// HistogramProtoToHistogram extracts a (normal integer) Histogram from the
// V2HistogramProtoToHistogram extracts a (normal integer) Histogram from the
// provided proto message. The caller has to make sure that the proto message
// represents an integer histogram and not a float histogram, or it panics.
func MinHistogramProtoToHistogram(hp writev2.Histogram) *histogram.Histogram {
func V2HistogramProtoToHistogram(hp writev2.Histogram) *histogram.Histogram {
if hp.IsFloatHistogram() {
panic("HistogramProtoToHistogram called with a float histogram")
}
@ -839,16 +839,16 @@ func HistogramToHistogramProto(timestamp int64, h *histogram.Histogram) prompb.H
}
}
func HistogramToMinHistogramProto(timestamp int64, h *histogram.Histogram) writev2.Histogram {
func HistogramToV2HistogramProto(timestamp int64, h *histogram.Histogram) writev2.Histogram {
return writev2.Histogram{
Count: &writev2.Histogram_CountInt{CountInt: h.Count},
Sum: h.Sum,
Schema: h.Schema,
ZeroThreshold: h.ZeroThreshold,
ZeroCount: &writev2.Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount},
NegativeSpans: spansToMinSpansProto(h.NegativeSpans),
NegativeSpans: spansToV2SpansProto(h.NegativeSpans),
NegativeDeltas: h.NegativeBuckets,
PositiveSpans: spansToMinSpansProto(h.PositiveSpans),
PositiveSpans: spansToV2SpansProto(h.PositiveSpans),
PositiveDeltas: h.PositiveBuckets,
ResetHint: writev2.Histogram_ResetHint(h.CounterResetHint),
Timestamp: timestamp,
@ -871,16 +871,16 @@ func FloatHistogramToHistogramProto(timestamp int64, fh *histogram.FloatHistogra
}
}
func FloatHistogramToMinHistogramProto(timestamp int64, fh *histogram.FloatHistogram) writev2.Histogram {
func FloatHistogramToV2HistogramProto(timestamp int64, fh *histogram.FloatHistogram) writev2.Histogram {
return writev2.Histogram{
Count: &writev2.Histogram_CountFloat{CountFloat: fh.Count},
Sum: fh.Sum,
Schema: fh.Schema,
ZeroThreshold: fh.ZeroThreshold,
ZeroCount: &writev2.Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount},
NegativeSpans: spansToMinSpansProto(fh.NegativeSpans),
NegativeSpans: spansToV2SpansProto(fh.NegativeSpans),
NegativeCounts: fh.NegativeBuckets,
PositiveSpans: spansToMinSpansProto(fh.PositiveSpans),
PositiveSpans: spansToV2SpansProto(fh.PositiveSpans),
PositiveCounts: fh.PositiveBuckets,
ResetHint: writev2.Histogram_ResetHint(fh.CounterResetHint),
Timestamp: timestamp,
@ -896,7 +896,7 @@ func spansToSpansProto(s []histogram.Span) []prompb.BucketSpan {
return spans
}
func spansToMinSpansProto(s []histogram.Span) []writev2.BucketSpan {
func spansToV2SpansProto(s []histogram.Span) []writev2.BucketSpan {
spans := make([]writev2.BucketSpan, len(s))
for i := 0; i < len(s); i++ {
spans[i] = writev2.BucketSpan{Offset: s[i].Offset, Length: s[i].Length}
@ -923,17 +923,6 @@ func labelProtosToLabels(b *labels.ScratchBuilder, labelPairs []prompb.Label) la
return b.Labels()
}
// labelProtosV2ToLabels transforms v2 proto labels references, which are uint32 values, into labels via
// indexing into the symbols slice.
func labelProtosV2ToLabels(labelRefs []uint32, symbols []string) labels.Labels {
b := labels.NewScratchBuilder(len(labelRefs))
for i := 0; i < len(labelRefs); i += 2 {
b.Add(symbols[labelRefs[i]], symbols[labelRefs[i+1]])
}
b.Sort()
return b.Labels()
}
// labelsToLabelsProto transforms labels into prompb labels. The buffer slice
// will be used to avoid allocations if it is big enough to store the labels.
func labelsToLabelsProto(lbls labels.Labels, buf []prompb.Label) []prompb.Label {
@ -947,17 +936,6 @@ func labelsToLabelsProto(lbls labels.Labels, buf []prompb.Label) []prompb.Label
return result
}
func labelsToLabelsProtoV2Refs(lbls labels.Labels, symbolTable *rwSymbolTable, buf []uint32) []uint32 {
result := buf[:0]
lbls.Range(func(l labels.Label) {
off := symbolTable.RefStr(l.Name)
result = append(result, off)
off = symbolTable.RefStr(l.Value)
result = append(result, off)
})
return result
}
// metricTypeToMetricTypeProto transforms a Prometheus metricType into prompb metricType. Since the former is a string we need to transform it to an enum.
func metricTypeToMetricTypeProto(t model.MetricType) prompb.MetricMetadata_MetricType {
mt := strings.ToUpper(string(t))
@ -1059,7 +1037,7 @@ func DecodeOTLPWriteRequest(r *http.Request) (pmetricotlp.ExportRequest, error)
return otlpReq, nil
}
func DecodeMinimizedWriteRequestStr(r io.Reader) (*writev2.Request, error) {
func DecodeV2WriteRequestStr(r io.Reader) (*writev2.Request, error) {
compressed, err := io.ReadAll(r)
if err != nil {
return nil, err
@ -1078,14 +1056,14 @@ func DecodeMinimizedWriteRequestStr(r io.Reader) (*writev2.Request, error) {
return &req, nil
}
func MinimizedWriteRequestToWriteRequest(redReq *writev2.Request) (*prompb.WriteRequest, error) {
func V2WriteRequestToWriteRequest(redReq *writev2.Request) (*prompb.WriteRequest, error) {
req := &prompb.WriteRequest{
Timeseries: make([]prompb.TimeSeries, len(redReq.Timeseries)),
// TODO handle metadata?
}
for i, rts := range redReq.Timeseries {
labelProtosV2ToLabels(rts.LabelsRefs, redReq.Symbols).Range(func(l labels.Label) {
writev2.DesymbolizeLabels(rts.LabelsRefs, redReq.Symbols).Range(func(l labels.Label) {
req.Timeseries[i].Labels = append(req.Timeseries[i].Labels, prompb.Label{
Name: l.Name,
Value: l.Value,
@ -1096,7 +1074,7 @@ func MinimizedWriteRequestToWriteRequest(redReq *writev2.Request) (*prompb.Write
for j, e := range rts.Exemplars {
exemplars[j].Value = e.Value
exemplars[j].Timestamp = e.Timestamp
labelProtosV2ToLabels(e.LabelsRefs, redReq.Symbols).Range(func(l labels.Label) {
writev2.DesymbolizeLabels(e.LabelsRefs, redReq.Symbols).Range(func(l labels.Label) {
exemplars[j].Labels = append(exemplars[j].Labels, prompb.Label{
Name: l.Name,
Value: l.Value,

View file

@ -76,9 +76,9 @@ var writeRequestFixture = &prompb.WriteRequest{
},
}
// writeRequestMinimizedFixture represents the same request as writeRequestFixture, but using the minimized representation.
var writeRequestMinimizedFixture = func() *writev2.Request {
st := newRwSymbolTable()
// writeV2RequestFixture represents the same request as writeRequestFixture, but using the v2 representation.
var writeV2RequestFixture = func() *writev2.Request {
st := writev2.NewSymbolTable()
var labels []uint32
for _, s := range []string{
"__name__", "test_metric1",
@ -87,14 +87,14 @@ var writeRequestMinimizedFixture = func() *writev2.Request {
"d", "e",
"foo", "bar",
} {
ref := st.RefStr(s)
ref := st.Symbolize(s)
labels = append(labels, ref)
}
for _, s := range []string{
"f", "g", // 10, 11
"h", "i", // 12, 13
} {
st.RefStr(s)
_ = st.Symbolize(s)
}
return &writev2.Request{
@ -103,16 +103,16 @@ var writeRequestMinimizedFixture = func() *writev2.Request {
LabelsRefs: labels,
Samples: []writev2.Sample{{Value: 1, Timestamp: 0}},
Exemplars: []writev2.Exemplar{{LabelsRefs: []uint32{10, 11}, Value: 1, Timestamp: 0}},
Histograms: []writev2.Histogram{HistogramToMinHistogramProto(0, &testHistogram), FloatHistogramToMinHistogramProto(1, testHistogram.ToFloat(nil))},
Histograms: []writev2.Histogram{HistogramToV2HistogramProto(0, &testHistogram), FloatHistogramToV2HistogramProto(1, testHistogram.ToFloat(nil))},
},
{
LabelsRefs: labels,
Samples: []writev2.Sample{{Value: 2, Timestamp: 1}},
Exemplars: []writev2.Exemplar{{LabelsRefs: []uint32{12, 13}, Value: 2, Timestamp: 1}},
Histograms: []writev2.Histogram{HistogramToMinHistogramProto(2, &testHistogram), FloatHistogramToMinHistogramProto(3, testHistogram.ToFloat(nil))},
Histograms: []writev2.Histogram{HistogramToV2HistogramProto(2, &testHistogram), FloatHistogramToV2HistogramProto(3, testHistogram.ToFloat(nil))},
},
},
Symbols: st.LabelsStrings(),
Symbols: st.Symbols(),
}
}()
@ -567,13 +567,13 @@ func TestDecodeWriteRequest(t *testing.T) {
require.Equal(t, writeRequestFixture, actual)
}
func TestDecodeMinWriteRequest(t *testing.T) {
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeRequestMinimizedFixture.Timeseries, writeRequestMinimizedFixture.Symbols, nil, nil, nil, "snappy")
func TestDecodeV2WriteRequest(t *testing.T) {
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeV2RequestFixture.Timeseries, writeV2RequestFixture.Symbols, nil, nil, nil, "snappy")
require.NoError(t, err)
actual, err := DecodeMinimizedWriteRequestStr(bytes.NewReader(buf))
actual, err := DecodeV2WriteRequestStr(bytes.NewReader(buf))
require.NoError(t, err)
require.Equal(t, writeRequestMinimizedFixture, actual)
require.Equal(t, writeV2RequestFixture, actual)
}
func TestNilHistogramProto(t *testing.T) {
@ -895,11 +895,3 @@ func (c *mockChunkIterator) Next() bool {
func (c *mockChunkIterator) Err() error {
return nil
}
func TestStrFormat(t *testing.T) {
r := newRwSymbolTable()
ls := labels.FromStrings("asdf", "qwer", "zxcv", "1234")
encoded := labelsToLabelsProtoV2Refs(ls, &r, nil)
decoded := labelProtosV2ToLabels(encoded, r.LabelsStrings())
require.Equal(t, ls, decoded)
}

View file

@ -19,7 +19,6 @@ import (
"fmt"
"math"
"strconv"
"strings"
"sync"
"time"
@ -393,22 +392,13 @@ func (m *queueManagerMetrics) unregister() {
// external timeseries database.
type WriteClient interface {
// Store stores the given samples in the remote storage.
Store(ctx context.Context, req []byte, retryAttempt int, rwFormat config.RemoteWriteFormat, compression string) error
Store(ctx context.Context, req []byte, retryAttempt int) error
// Name uniquely identifies the remote storage.
Name() string
// Endpoint is the remote read or write endpoint for the storage client.
Endpoint() string
// Get the protocol versions supported by the endpoint.
probeRemoteVersions(ctx context.Context) error
// Get the last RW header received from the endpoint.
GetLastRWHeader() string
}
const (
Version1 config.RemoteWriteFormat = iota // 1.0, 0.1, etc.
Version2 // symbols are indices into an array of strings.
)
// QueueManager manages a queue of samples to be sent to the Storage
// indicated by the provided WriteClient. Implements writeTo interface
// used by WAL Watcher.
@ -428,10 +418,11 @@ type QueueManager struct {
sendNativeHistograms bool
watcher *wlog.Watcher
metadataWatcher *MetadataWatcher
rwFormat config.RemoteWriteFormat
clientMtx sync.RWMutex
storeClient WriteClient
protoMsg config.RemoteWriteProtoMsg
enc Compression
seriesMtx sync.Mutex // Covers seriesLabels, seriesMetadata, droppedSeries and builder.
seriesLabels map[chunks.HeadSeriesRef]labels.Labels
@ -478,7 +469,7 @@ func NewQueueManager(
sm ReadyScrapeManager,
enableExemplarRemoteWrite bool,
enableNativeHistogramRemoteWrite bool,
rwFormat config.RemoteWriteFormat,
protoMsg config.RemoteWriteProtoMsg,
) *QueueManager {
if logger == nil {
logger = log.NewNopLogger()
@ -501,7 +492,6 @@ func NewQueueManager(
storeClient: client,
sendExemplars: enableExemplarRemoteWrite,
sendNativeHistograms: enableNativeHistogramRemoteWrite,
rwFormat: rwFormat,
seriesLabels: make(map[chunks.HeadSeriesRef]labels.Labels),
seriesMetadata: make(map[chunks.HeadSeriesRef]*metadata.Metadata),
@ -521,10 +511,13 @@ func NewQueueManager(
metrics: metrics,
interner: interner,
highestRecvTimestamp: highestRecvTimestamp,
protoMsg: protoMsg,
enc: SnappyBlockCompression, // Hardcoded for now, but scaffolding exists for likely future use.
}
walMetadata := false
if t.rwFormat > Version1 {
if t.protoMsg != config.RemoteWriteProtoMsgV1 {
walMetadata = true
}
t.watcher = wlog.NewWatcher(watcherMetrics, readerMetrics, logger, client.Name(), t, dir, enableExemplarRemoteWrite, enableNativeHistogramRemoteWrite, walMetadata)
@ -533,7 +526,7 @@ func NewQueueManager(
// with the new approach, which stores metadata as WAL records and
// ships them alongside series. If both mechanisms are set, the new one
// takes precedence by implicitly disabling the older one.
if t.mcfg.Send && t.rwFormat > Version1 {
if t.mcfg.Send && t.protoMsg != config.RemoteWriteProtoMsgV1 {
level.Warn(logger).Log("msg", "usage of 'metadata_config.send' is redundant when using remote write v2 (or higher) as metadata will always be gathered from the WAL and included for every series within each write request")
t.mcfg.Send = false
}
@ -550,9 +543,10 @@ func NewQueueManager(
// This is only used for the metadata_config.send setting and 1.x Remote Write.
func (t *QueueManager) AppendWatcherMetadata(ctx context.Context, metadata []scrape.MetricMetadata) {
// no op for any newer proto format, which will cache metadata sent to it from the WAL watcher.
if t.rwFormat > Version2 {
if t.protoMsg != config.RemoteWriteProtoMsgV1 {
return
}
// 1.X will still get metadata in batches.
mm := make([]prompb.MetricMetadata, 0, len(metadata))
for _, entry := range metadata {
@ -580,12 +574,8 @@ func (t *QueueManager) AppendWatcherMetadata(ctx context.Context, metadata []scr
}
func (t *QueueManager) sendMetadataWithBackoff(ctx context.Context, metadata []prompb.MetricMetadata, pBuf *proto.Buffer) error {
// Build the WriteRequest with no samples.
// Get compression to use from content negotiation based on last header seen (defaults to snappy).
compression, _ := negotiateRWProto(t.rwFormat, t.storeClient.GetLastRWHeader())
req, _, _, err := buildWriteRequest(t.logger, nil, metadata, pBuf, nil, nil, compression)
// Build the WriteRequest with no samples (v1 flow).
req, _, _, err := buildWriteRequest(t.logger, nil, metadata, pBuf, nil, nil, t.enc)
if err != nil {
return err
}
@ -608,7 +598,7 @@ func (t *QueueManager) sendMetadataWithBackoff(ctx context.Context, metadata []p
}
begin := time.Now()
err := t.storeClient.Store(ctx, req, try, Version1, compression)
err := t.storeClient.Store(ctx, req, try)
t.metrics.sentBatchDuration.Observe(time.Since(begin).Seconds())
if err != nil {
@ -1008,7 +998,7 @@ func (t *QueueManager) StoreSeries(series []record.RefSeries, index int) {
// StoreMetadata keeps track of known series' metadata for lookups when sending samples to remote.
func (t *QueueManager) StoreMetadata(meta []record.RefMetadata) {
if t.rwFormat < Version2 {
if t.protoMsg == config.RemoteWriteProtoMsgV1 {
return
}
@ -1511,40 +1501,6 @@ func (q *queue) newBatch(capacity int) []timeSeries {
return make([]timeSeries, 0, capacity)
}
func negotiateRWProto(rwFormat config.RemoteWriteFormat, lastHeaderSeen string) (string, config.RemoteWriteFormat) {
if rwFormat == Version1 {
// If we're only handling Version1 then all we can do is that with snappy compression.
return "snappy", Version1
}
if rwFormat != Version2 {
// If we get here then someone has added a new RemoteWriteFormat value but hasn't
// fixed this function to handle it. Panic!
panic(fmt.Sprintf("Unhandled RemoteWriteFormat %q", rwFormat))
}
if lastHeaderSeen == "" {
// We haven't had a valid header, so we just default to "0.1.0/snappy".
return "snappy", Version1
}
// We can currently handle:
// "2.0;snappy"
// "0.1.0" - implicit compression of snappy
// lastHeaderSeen should contain a list of tuples.
// If we find a match to something we can handle then we can return that.
for _, tuple := range strings.Split(lastHeaderSeen, ",") {
// Remove spaces from the tuple.
curr := strings.ReplaceAll(tuple, " ", "")
switch curr {
case "2.0;snappy":
return "snappy", Version2
case "0.1.0":
return "snappy", Version1
}
}
// Otherwise we have to default to "0.1.0".
return "snappy", Version1
}
func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
defer func() {
if s.running.Dec() == 0 {
@ -1553,7 +1509,7 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
}()
shardNum := strconv.Itoa(shardID)
symbolTable := newRwSymbolTable()
symbolTable := writev2.NewSymbolTable()
// Send batches of at most MaxSamplesPerSend samples to the remote storage.
// If we have fewer samples than that, flush them out after a deadline anyways.
@ -1569,7 +1525,7 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
max += int(float64(max) * 0.1)
}
// TODO we should make an interface for the timeseries type
// TODO: Dry all of this, we should make an interface/generic for the timeseries type.
batchQueue := queue.Chan()
pendingData := make([]prompb.TimeSeries, max)
for i := range pendingData {
@ -1578,10 +1534,9 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
pendingData[i].Exemplars = []prompb.Exemplar{{}}
}
}
pendingMinStrData := make([]writev2.TimeSeries, max)
for i := range pendingMinStrData {
pendingMinStrData[i].Samples = []writev2.Sample{{}}
pendingDataV2 := make([]writev2.TimeSeries, max)
for i := range pendingDataV2 {
pendingDataV2[i].Samples = []writev2.Sample{{}}
}
timer := time.NewTimer(time.Duration(s.qm.cfg.BatchSendDeadline))
@ -1595,24 +1550,22 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
}
defer stop()
attemptBatchSend := func(batch []timeSeries, rwFormat config.RemoteWriteFormat, compression string, timer bool) error {
switch rwFormat {
case Version1:
sendBatch := func(batch []timeSeries, protoMsg config.RemoteWriteProtoMsg, enc Compression, timer bool) {
switch protoMsg {
case config.RemoteWriteProtoMsgV1:
nPendingSamples, nPendingExemplars, nPendingHistograms := populateTimeSeries(batch, pendingData, s.qm.sendExemplars, s.qm.sendNativeHistograms)
n := nPendingSamples + nPendingExemplars + nPendingHistograms
if timer {
level.Debug(s.qm.logger).Log("msg", "runShard timer ticked, sending buffered data", "samples", nPendingSamples,
"exemplars", nPendingExemplars, "shard", shardNum, "histograms", nPendingHistograms)
}
return s.sendSamples(ctx, pendingData[:n], nPendingSamples, nPendingExemplars, nPendingHistograms, pBuf, &buf, compression)
case Version2:
nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata := populateV2TimeSeries(&symbolTable, batch, pendingMinStrData, s.qm.sendExemplars, s.qm.sendNativeHistograms)
_ = s.sendSamples(ctx, pendingData[:n], nPendingSamples, nPendingExemplars, nPendingHistograms, pBuf, &buf, enc)
case config.RemoteWriteProtoMsgV2:
nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata := populateV2TimeSeries(&symbolTable, batch, pendingDataV2, s.qm.sendExemplars, s.qm.sendNativeHistograms)
n := nPendingSamples + nPendingExemplars + nPendingHistograms
err := s.sendV2Samples(ctx, pendingMinStrData[:n], symbolTable.LabelsStrings(), nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata, &pBufRaw, &buf, compression)
symbolTable.clear()
return err
_ = s.sendV2Samples(ctx, pendingDataV2[:n], symbolTable.Symbols(), nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata, &pBufRaw, &buf, enc)
symbolTable.Reset()
}
return nil
}
for {
@ -1638,20 +1591,9 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
if !ok {
return
}
// Work out what version to send based on the last header seen and the QM's rwFormat setting.
for attemptNos := 1; attemptNos <= 3; attemptNos++ {
lastHeaderSeen := s.qm.storeClient.GetLastRWHeader()
compression, rwFormat := negotiateRWProto(s.qm.rwFormat, lastHeaderSeen)
sendErr := attemptBatchSend(batch, rwFormat, compression, false)
pErr := &ErrRenegotiate{}
if sendErr == nil || !errors.As(sendErr, &pErr) {
// No error, or error wasn't a 406 or 400, so we can stop trying.
break
}
// If we get either of the two errors (406, 400) bundled in ErrRenegotiate we loop and re-negotiate.
// TODO(alexg) - add retry/renegotiate metrics here
}
sendBatch(batch, s.qm.protoMsg, s.qm.enc, false)
// TODO(bwplotka): Previously the return was between popular and send, double check.
queue.ReturnForReuse(batch)
stop()
@ -1660,19 +1602,7 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
case <-timer.C:
batch := queue.Batch()
if len(batch) > 0 {
for attemptNos := 1; attemptNos <= 3; attemptNos++ {
// Work out what version to send based on the last header seen and the QM's rwFormat setting.
lastHeaderSeen := s.qm.storeClient.GetLastRWHeader()
compression, rwFormat := negotiateRWProto(s.qm.rwFormat, lastHeaderSeen)
sendErr := attemptBatchSend(batch, rwFormat, compression, true)
pErr := &ErrRenegotiate{}
if sendErr == nil || !errors.As(sendErr, &pErr) {
// No error, or error wasn't a 406 or 400, so we can stop trying.
break
}
// If we get either of the two errors (406, 400) bundled in ErrRenegotiate we loop and re-negotiate.
}
// TODO(alexg) - add retry/renegotiate metrics here
sendBatch(batch, s.qm.protoMsg, s.qm.enc, true)
}
queue.ReturnForReuse(batch)
timer.Reset(time.Duration(s.qm.cfg.BatchSendDeadline))
@ -1721,25 +1651,21 @@ func populateTimeSeries(batch []timeSeries, pendingData []prompb.TimeSeries, sen
return nPendingSamples, nPendingExemplars, nPendingHistograms
}
func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount int, pBuf *proto.Buffer, buf *[]byte, compression string) error {
func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount int, pBuf *proto.Buffer, buf *[]byte, enc Compression) error {
begin := time.Now()
err := s.sendSamplesWithBackoff(ctx, samples, sampleCount, exemplarCount, histogramCount, 0, pBuf, buf, compression)
err := s.sendSamplesWithBackoff(ctx, samples, sampleCount, exemplarCount, histogramCount, 0, pBuf, buf, enc)
s.updateMetrics(ctx, err, sampleCount, exemplarCount, histogramCount, 0, time.Since(begin))
// Return the error in case it is a 406 and we need to reformat the data.
return err
}
func (s *shards) sendV2Samples(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, compression string) error {
func (s *shards) sendV2Samples(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, enc Compression) error {
begin := time.Now()
err := s.sendV2SamplesWithBackoff(ctx, samples, labels, sampleCount, exemplarCount, histogramCount, metadataCount, pBuf, buf, compression)
err := s.sendV2SamplesWithBackoff(ctx, samples, labels, sampleCount, exemplarCount, histogramCount, metadataCount, pBuf, buf, enc)
s.updateMetrics(ctx, err, sampleCount, exemplarCount, histogramCount, metadataCount, time.Since(begin))
// Return the error in case it is a 406 and we need to reformat the data.
return err
}
func (s *shards) updateMetrics(ctx context.Context, err error, sampleCount, exemplarCount, histogramCount, metadataCount int, duration time.Duration) {
func (s *shards) updateMetrics(_ context.Context, err error, sampleCount, exemplarCount, histogramCount, metadataCount int, duration time.Duration) {
if err != nil {
level.Error(s.qm.logger).Log("msg", "non-recoverable error", "count", sampleCount, "exemplarCount", exemplarCount, "err", err)
s.qm.metrics.failedSamplesTotal.Add(float64(sampleCount))
@ -1763,9 +1689,9 @@ func (s *shards) updateMetrics(ctx context.Context, err error, sampleCount, exem
}
// sendSamples to the remote storage with backoff for recoverable errors.
func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf *proto.Buffer, buf *[]byte, compression string) error {
func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf *proto.Buffer, buf *[]byte, enc Compression) error {
// Build the WriteRequest with no metadata.
req, highest, lowest, err := buildWriteRequest(s.qm.logger, samples, nil, pBuf, buf, nil, compression)
req, highest, lowest, err := buildWriteRequest(s.qm.logger, samples, nil, pBuf, buf, nil, enc)
s.qm.buildRequestLimitTimestamp.Store(lowest)
if err != nil {
// Failing to build the write request is non-recoverable, since it will
@ -1791,7 +1717,7 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti
pBuf,
buf,
isTimeSeriesOldFilter(s.qm.metrics, currentTime, time.Duration(s.qm.cfg.SampleAgeLimit)),
compression,
enc,
)
s.qm.buildRequestLimitTimestamp.Store(lowest)
if err != nil {
@ -1823,7 +1749,7 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti
s.qm.metrics.exemplarsTotal.Add(float64(exemplarCount))
s.qm.metrics.histogramsTotal.Add(float64(histogramCount))
s.qm.metrics.metadataTotal.Add(float64(metadataCount))
err := s.qm.client().Store(ctx, *buf, try, Version1, compression)
err := s.qm.client().Store(ctx, *buf, try)
s.qm.metrics.sentBatchDuration.Observe(time.Since(begin).Seconds())
if err != nil {
@ -1854,9 +1780,9 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti
}
// sendV2Samples to the remote storage with backoff for recoverable errors.
func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, compression string) error {
func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, enc Compression) error {
// Build the WriteRequest with no metadata.
req, highest, lowest, err := buildV2WriteRequest(s.qm.logger, samples, labels, pBuf, buf, nil, compression)
req, highest, lowest, err := buildV2WriteRequest(s.qm.logger, samples, labels, pBuf, buf, nil, enc)
s.qm.buildRequestLimitTimestamp.Store(lowest)
if err != nil {
// Failing to build the write request is non-recoverable, since it will
@ -1882,7 +1808,7 @@ func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2
pBuf,
buf,
isV2TimeSeriesOldFilter(s.qm.metrics, currentTime, time.Duration(s.qm.cfg.SampleAgeLimit)),
compression,
enc,
)
s.qm.buildRequestLimitTimestamp.Store(lowest)
if err != nil {
@ -1914,7 +1840,7 @@ func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2
s.qm.metrics.exemplarsTotal.Add(float64(exemplarCount))
s.qm.metrics.histogramsTotal.Add(float64(histogramCount))
s.qm.metrics.metadataTotal.Add(float64(metadataCount))
err := s.qm.client().Store(ctx, *buf, try, Version2, compression)
err := s.qm.client().Store(ctx, *buf, try)
s.qm.metrics.sentBatchDuration.Observe(time.Since(begin).Seconds())
if err != nil {
@ -1944,15 +1870,15 @@ func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2
return err
}
func populateV2TimeSeries(symbolTable *rwSymbolTable, batch []timeSeries, pendingData []writev2.TimeSeries, sendExemplars, sendNativeHistograms bool) (int, int, int, int) {
func populateV2TimeSeries(symbolTable *writev2.SymbolsTable, batch []timeSeries, pendingData []writev2.TimeSeries, sendExemplars, sendNativeHistograms bool) (int, int, int, int) {
var nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata int
for nPending, d := range batch {
pendingData[nPending].Samples = pendingData[nPending].Samples[:0]
// todo: should we also safeguard against empty metadata here?
if d.metadata != nil {
pendingData[nPending].Metadata.Type = metricTypeToMetricTypeProtoV2(d.metadata.Type)
pendingData[nPending].Metadata.HelpRef = symbolTable.RefStr(d.metadata.Help)
pendingData[nPending].Metadata.HelpRef = symbolTable.RefStr(d.metadata.Unit)
pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize(d.metadata.Help)
pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize(d.metadata.Unit)
nPendingMetadata++
}
@ -1966,9 +1892,7 @@ func populateV2TimeSeries(symbolTable *rwSymbolTable, batch []timeSeries, pendin
// Number of pending samples is limited by the fact that sendSamples (via sendSamplesWithBackoff)
// retries endlessly, so once we reach max samples, if we can never send to the endpoint we'll
// stop reading from the queue. This makes it safe to reference pendingSamples by index.
// pendingData[nPending].Labels = labelsToLabelsProto(d.seriesLabels, pendingData[nPending].Labels)
pendingData[nPending].LabelsRefs = labelsToLabelsProtoV2Refs(d.seriesLabels, symbolTable, pendingData[nPending].LabelsRefs)
pendingData[nPending].LabelsRefs = symbolTable.SymbolizeLabels(d.seriesLabels, pendingData[nPending].LabelsRefs)
switch d.sType {
case tSample:
pendingData[nPending].Samples = append(pendingData[nPending].Samples, writev2.Sample{
@ -1978,16 +1902,16 @@ func populateV2TimeSeries(symbolTable *rwSymbolTable, batch []timeSeries, pendin
nPendingSamples++
case tExemplar:
pendingData[nPending].Exemplars = append(pendingData[nPending].Exemplars, writev2.Exemplar{
LabelsRefs: labelsToLabelsProtoV2Refs(d.exemplarLabels, symbolTable, nil), // TODO: optimize, reuse slice
LabelsRefs: symbolTable.SymbolizeLabels(d.exemplarLabels, nil), // TODO: optimize, reuse slice
Value: d.value,
Timestamp: d.timestamp,
})
nPendingExemplars++
case tHistogram:
pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, HistogramToMinHistogramProto(d.timestamp, d.histogram))
pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, HistogramToV2HistogramProto(d.timestamp, d.histogram))
nPendingHistograms++
case tFloatHistogram:
pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, FloatHistogramToMinHistogramProto(d.timestamp, d.floatHistogram))
pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, FloatHistogramToV2HistogramProto(d.timestamp, d.floatHistogram))
nPendingHistograms++
case tMetadata:
// TODO: log or return an error?
@ -2138,11 +2062,9 @@ func buildTimeSeries(timeSeries []prompb.TimeSeries, filter func(prompb.TimeSeri
return highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms
}
func compressPayload(tmpbuf *[]byte, inp []byte, compression string) ([]byte, error) {
var compressed []byte
switch compression {
case "snappy":
func compressPayload(tmpbuf *[]byte, inp []byte, enc Compression) (compressed []byte, _ error) {
switch enc {
case SnappyBlockCompression:
compressed = snappy.Encode(*tmpbuf, inp)
if n := snappy.MaxEncodedLen(len(inp)); n > len(*tmpbuf) {
// grow the buffer for the next time
@ -2150,11 +2072,11 @@ func compressPayload(tmpbuf *[]byte, inp []byte, compression string) ([]byte, er
}
return compressed, nil
default:
return compressed, fmt.Errorf("Unknown compression scheme [%s]", compression)
return compressed, fmt.Errorf("Unknown compression scheme [%v]", enc)
}
}
func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metadata []prompb.MetricMetadata, pBuf *proto.Buffer, buf *[]byte, filter func(prompb.TimeSeries) bool, compression string) ([]byte, int64, int64, error) {
func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metadata []prompb.MetricMetadata, pBuf *proto.Buffer, buf *[]byte, filter func(prompb.TimeSeries) bool, enc Compression) (compressed []byte, highest, lowest int64, _ error) {
highest, lowest, timeSeries,
droppedSamples, droppedExemplars, droppedHistograms := buildTimeSeries(timeSeries, filter)
@ -2185,49 +2107,14 @@ func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metada
buf = &[]byte{}
}
var compressed []byte
compressed, err = compressPayload(buf, pBuf.Bytes(), compression)
compressed, err = compressPayload(buf, pBuf.Bytes(), enc)
if err != nil {
return nil, highest, lowest, err
}
return compressed, highest, lowest, nil
}
type rwSymbolTable struct {
strings []string
symbolsMap map[string]uint32
}
func newRwSymbolTable() rwSymbolTable {
return rwSymbolTable{
symbolsMap: make(map[string]uint32),
}
}
func (r *rwSymbolTable) RefStr(str string) uint32 {
if ref, ok := r.symbolsMap[str]; ok {
return ref
}
ref := uint32(len(r.strings))
r.strings = append(r.strings, str)
r.symbolsMap[str] = ref
return ref
}
func (r *rwSymbolTable) LabelsStrings() []string {
return r.strings
}
func (r *rwSymbolTable) clear() {
r.strings = r.strings[:0]
for k := range r.symbolsMap {
delete(r.symbolsMap, k)
}
}
func buildV2WriteRequest(logger log.Logger, samples []writev2.TimeSeries, labels []string, pBuf, buf *[]byte, filter func(writev2.TimeSeries) bool, compression string) ([]byte, int64, int64, error) {
func buildV2WriteRequest(logger log.Logger, samples []writev2.TimeSeries, labels []string, pBuf, buf *[]byte, filter func(writev2.TimeSeries) bool, enc Compression) (compressed []byte, highest, lowest int64, _ error) {
highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms := buildV2TimeSeries(samples, filter)
if droppedSamples > 0 || droppedExemplars > 0 || droppedHistograms > 0 {
@ -2257,13 +2144,10 @@ func buildV2WriteRequest(logger log.Logger, samples []writev2.TimeSeries, labels
buf = &[]byte{}
}
var compressed []byte
compressed, err = compressPayload(buf, data, compression)
compressed, err = compressPayload(buf, data, enc)
if err != nil {
return nil, highest, lowest, err
}
return compressed, highest, lowest, nil
}
@ -2299,7 +2183,7 @@ func buildV2TimeSeries(timeSeries []writev2.TimeSeries, filter func(writev2.Time
highest = ts.Histograms[0].Timestamp
}
// Get lowest timestamp
// Get the lowest timestamp.
if len(ts.Samples) > 0 && ts.Samples[0].Timestamp < lowest {
lowest = ts.Samples[0].Timestamp
}

View file

@ -15,6 +15,7 @@ package remote
import (
"context"
"errors"
"fmt"
"math"
"math/rand"
@ -64,74 +65,7 @@ func newHighestTimestampMetric() *maxTimestamp {
}
}
type contentNegotiationStep struct {
lastRWHeader string
compression string
behaviour error // or nil
attemptString string
}
func TestContentNegotiation(t *testing.T) {
testcases := []struct {
name string
success bool
qmRwFormat config.RemoteWriteProtoMsg
rwFormat config.RemoteWriteFormat
steps []contentNegotiationStep
}{
// Test a simple case where the v2 request we send is processed first time.
{
success: true, name: "v2 happy path", qmRwFormat: config.RemoteWriteProtoMsgV2, rwFormat: Version2, steps: []contentNegotiationStep{
{lastRWHeader: "2.0;snappy,0.1.0", compression: "snappy", behaviour: nil, attemptString: "0,1,snappy,ok"},
},
},
// Test a simple case where the v1 request we send is processed first time.
{
success: true, name: "v1 happy path", qmRwFormat: config.RemoteWriteProtoMsgV1, rwFormat: Version1, steps: []contentNegotiationStep{
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: nil, attemptString: "0,0,snappy,ok"},
},
},
// Test a case where the v1 request has a temporary delay but goes through on retry.
// There is no content re-negotiation between first and retry attempts.
{
success: true, name: "v1 happy path with one 5xx retry", qmRwFormat: config.RemoteWriteProtoMsgV1, rwFormat: Version1, steps: []contentNegotiationStep{
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: RecoverableError{fmt.Errorf("Pretend 500"), 1}, attemptString: "0,0,snappy,Pretend 500"},
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: nil, attemptString: "1,0,snappy,ok"},
},
},
// Repeat the above test but with v2. The request has a temporary delay but goes through on retry.
// There is no content re-negotiation between first and retry attempts.
{
success: true, name: "v2 happy path with one 5xx retry", qmRwFormat: config.RemoteWriteProtoMsgV2, rwFormat: Version2, steps: []contentNegotiationStep{
{lastRWHeader: "2.0;snappy,0.1.0", compression: "snappy", behaviour: RecoverableError{fmt.Errorf("Pretend 500"), 1}, attemptString: "0,1,snappy,Pretend 500"},
{lastRWHeader: "2.0;snappy,0.1.0", compression: "snappy", behaviour: nil, attemptString: "1,1,snappy,ok"},
},
},
// Now test where the server suddenly stops speaking 2.0 and we need to downgrade.
{
success: true, name: "v2 request to v2 server that has downgraded via 406", qmRwFormat: config.RemoteWriteProtoMsgV2, rwFormat: Version2, steps: []contentNegotiationStep{
{lastRWHeader: "2.0;snappy,0.1.0", compression: "snappy", behaviour: &ErrRenegotiate{"", 406}, attemptString: "0,1,snappy,HTTP 406: msg: "},
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: nil, attemptString: "0,0,snappy,ok"},
},
},
// Now test where the server suddenly stops speaking 2.0 and we need to downgrade because it returns a 400.
{
success: true, name: "v2 request to v2 server that has downgraded via 400", qmRwFormat: config.RemoteWriteProtoMsgV2, rwFormat: Version2, steps: []contentNegotiationStep{
{lastRWHeader: "2.0;snappy,0.1.0", compression: "snappy", behaviour: &ErrRenegotiate{"", 400}, attemptString: "0,1,snappy,HTTP 400: msg: "},
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: nil, attemptString: "0,0,snappy,ok"},
},
},
// Now test where the server flip flops between "2.0;snappy" and "0.1.0" only.
{
success: false, name: "flip flopping", qmRwFormat: config.RemoteWriteProtoMsgV2, rwFormat: Version2, steps: []contentNegotiationStep{
{lastRWHeader: "2.0;snappy", compression: "snappy", behaviour: &ErrRenegotiate{"", 406}, attemptString: "0,1,snappy,HTTP 406: msg: "},
{lastRWHeader: "0.1.0", compression: "snappy", behaviour: &ErrRenegotiate{"", 406}, attemptString: "0,0,snappy,HTTP 406: msg: "},
{lastRWHeader: "2.0;snappy", compression: "snappy", behaviour: &ErrRenegotiate{"", 406}, attemptString: "0,1,snappy,HTTP 406: msg: "},
// There's no 4th attempt as we do a maximum of 3 sending attempts (not counting retries).
},
},
}
func TestBasicContentNegotiation(t *testing.T) {
queueConfig := config.DefaultQueueConfig
queueConfig.BatchSendDeadline = model.Duration(100 * time.Millisecond)
queueConfig.MaxShards = 1
@ -147,7 +81,62 @@ func TestContentNegotiation(t *testing.T) {
},
}
for _, tc := range testcases {
for _, tc := range []struct {
name string
senderProtoMsg config.RemoteWriteProtoMsg
receiverProtoMsg config.RemoteWriteProtoMsg
injectErrs []error
expectFail bool
}{
{
name: "v2 happy path",
senderProtoMsg: config.RemoteWriteProtoMsgV2, receiverProtoMsg: config.RemoteWriteProtoMsgV2,
injectErrs: []error{nil},
},
{
name: "v1 happy path",
senderProtoMsg: config.RemoteWriteProtoMsgV1, receiverProtoMsg: config.RemoteWriteProtoMsgV1,
injectErrs: []error{nil},
},
// Test a case where the v1 request has a temporary delay but goes through on retry.
{
name: "v1 happy path with one 5xx retry",
senderProtoMsg: config.RemoteWriteProtoMsgV1, receiverProtoMsg: config.RemoteWriteProtoMsgV1,
injectErrs: []error{RecoverableError{errors.New("pretend 500"), 1}, nil},
},
// Repeat the above test but with v2. The request has a temporary delay but goes through on retry.
{
name: "v2 happy path with one 5xx retry",
senderProtoMsg: config.RemoteWriteProtoMsgV2, receiverProtoMsg: config.RemoteWriteProtoMsgV2,
injectErrs: []error{RecoverableError{errors.New("pretend 500"), 1}, nil},
},
// A few error cases of v2 talking to v1.
{
name: "v2 talks to v1 that gives 400 or 415",
senderProtoMsg: config.RemoteWriteProtoMsgV2, receiverProtoMsg: config.RemoteWriteProtoMsgV1,
injectErrs: []error{errors.New("pretend unrecoverable err")},
expectFail: true,
},
{
name: "v2 talks to v1 that tries to unmarshal v2 payload with v1 proto",
senderProtoMsg: config.RemoteWriteProtoMsgV2, receiverProtoMsg: config.RemoteWriteProtoMsgV1,
injectErrs: []error{nil},
expectFail: true, // invalid request, no timeseries
},
// Opposite, v1 talking to v2 only server.
{
name: "v1 talks to v2 that gives 400 or 415",
senderProtoMsg: config.RemoteWriteProtoMsgV1, receiverProtoMsg: config.RemoteWriteProtoMsgV2,
injectErrs: []error{errors.New("pretend unrecoverable err")},
expectFail: true,
},
{
name: "v1 talks to (broken) v2 that tries to unmarshal v1 payload with v2 proto",
senderProtoMsg: config.RemoteWriteProtoMsgV1, receiverProtoMsg: config.RemoteWriteProtoMsgV2,
injectErrs: []error{nil},
expectFail: true, // invalid request, no timeseries
},
} {
t.Run(tc.name, func(t *testing.T) {
dir := t.TempDir()
s := NewStorage(nil, nil, nil, dir, defaultFlushDeadline, nil, true)
@ -167,67 +156,56 @@ func TestContentNegotiation(t *testing.T) {
queueConfig.Capacity = len(samples)
queueConfig.MaxSamplesPerSend = len(samples)
// For now we only ever have a single rw config in this test.
conf.RemoteWriteConfigs[0].ProtobufMessage = tc.qmRwFormat
conf.RemoteWriteConfigs[0].ProtobufMessage = tc.senderProtoMsg
require.NoError(t, s.ApplyConfig(conf))
hash, err := toHash(writeConfig)
require.NoError(t, err)
qm := s.rws.queues[hash]
c := NewTestWriteClient(tc.rwFormat)
c.setSteps(tc.steps) // set expected behaviour.
c := NewTestWriteClient(tc.receiverProtoMsg)
c.injectErrors(tc.injectErrs)
qm.SetClient(c)
qm.StoreSeries(series, 0)
qm.StoreMetadata(metadata)
// Did we expect some data back?
if tc.success {
// Do we expect some data back?
if !tc.expectFail {
c.expectSamples(samples, series)
} else {
c.expectSamples(nil, nil)
}
// Schedule send.
qm.Append(samples)
if !tc.success {
// We just need to sleep for a bit to give it time to run.
time.Sleep(2 * time.Second)
// But we still need to check for data with no delay to avoid race.
c.waitForExpectedData(t, 0*time.Second)
} else {
// We expected data so wait for it.
if !tc.expectFail {
// No error expected, so wait for data.
c.waitForExpectedData(t, 5*time.Second)
require.Equal(t, 1, c.writesReceived)
require.Equal(t, 0.0, client_testutil.ToFloat64(qm.metrics.failedSamplesTotal))
} else {
// Wait for failure to be recorded in metrics.
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
require.NoError(t, runutil.Retry(500*time.Millisecond, ctx.Done(), func() error {
if client_testutil.ToFloat64(qm.metrics.failedSamplesTotal) != 1.0 {
return errors.New("expected one sample failed in qm metrics")
}
return nil
}))
require.Equal(t, 0, c.writesReceived)
}
require.Equal(t, len(c.sendAttempts), len(tc.steps))
for i, s := range c.sendAttempts {
require.Equal(t, s, tc.steps[i].attemptString)
}
// samplesTotal means attempts.
require.Equal(t, float64(len(tc.injectErrs)), client_testutil.ToFloat64(qm.metrics.samplesTotal))
require.Equal(t, float64(len(tc.injectErrs)-1), client_testutil.ToFloat64(qm.metrics.retriedSamplesTotal))
})
}
}
func TestSampleDelivery(t *testing.T) {
testcases := []struct {
name string
samples bool
exemplars bool
histograms bool
floatHistograms bool
rwFormat config.RemoteWriteFormat
}{
{samples: true, exemplars: false, histograms: false, floatHistograms: false, name: "samples only"},
{samples: true, exemplars: true, histograms: true, floatHistograms: true, name: "samples, exemplars, and histograms"},
{samples: false, exemplars: true, histograms: false, floatHistograms: false, name: "exemplars only"},
{samples: false, exemplars: false, histograms: true, floatHistograms: false, name: "histograms only"},
{samples: false, exemplars: false, histograms: false, floatHistograms: true, name: "float histograms only"},
// TODO(alexg): update some portion of this test to check for the 2.0 metadata
{samples: true, exemplars: false, histograms: false, floatHistograms: false, name: "samples only", rwFormat: Version2},
{samples: true, exemplars: true, histograms: true, floatHistograms: true, name: "samples, exemplars, and histograms", rwFormat: Version2},
{samples: false, exemplars: true, histograms: false, floatHistograms: false, name: "exemplars only", rwFormat: Version2},
{samples: false, exemplars: false, histograms: true, floatHistograms: false, name: "histograms only", rwFormat: Version2},
{samples: false, exemplars: false, histograms: false, floatHistograms: true, name: "float histograms only", rwFormat: Version2},
}
// Let's create an even number of send batches so we don't run into the
// Let's create an even number of send batches, so we don't run into the
// batch timeout case.
n := 3
@ -247,9 +225,29 @@ func TestSampleDelivery(t *testing.T) {
writeConfig,
},
}
for _, tc := range []struct {
protoMsg config.RemoteWriteProtoMsg
for _, tc := range testcases {
t.Run(tc.name, func(t *testing.T) {
name string
samples bool
exemplars bool
histograms bool
floatHistograms bool
}{
{protoMsg: config.RemoteWriteProtoMsgV1, samples: true, exemplars: false, histograms: false, floatHistograms: false, name: "samples only"},
{protoMsg: config.RemoteWriteProtoMsgV1, samples: true, exemplars: true, histograms: true, floatHistograms: true, name: "samples, exemplars, and histograms"},
{protoMsg: config.RemoteWriteProtoMsgV1, samples: false, exemplars: true, histograms: false, floatHistograms: false, name: "exemplars only"},
{protoMsg: config.RemoteWriteProtoMsgV1, samples: false, exemplars: false, histograms: true, floatHistograms: false, name: "histograms only"},
{protoMsg: config.RemoteWriteProtoMsgV1, samples: false, exemplars: false, histograms: false, floatHistograms: true, name: "float histograms only"},
// TODO(alexg): update some portion of this test to check for the 2.0 metadata
{protoMsg: config.RemoteWriteProtoMsgV2, samples: true, exemplars: false, histograms: false, floatHistograms: false, name: "samples only"},
{protoMsg: config.RemoteWriteProtoMsgV2, samples: true, exemplars: true, histograms: true, floatHistograms: true, name: "samples, exemplars, and histograms"},
{protoMsg: config.RemoteWriteProtoMsgV2, samples: false, exemplars: true, histograms: false, floatHistograms: false, name: "exemplars only"},
{protoMsg: config.RemoteWriteProtoMsgV2, samples: false, exemplars: false, histograms: true, floatHistograms: false, name: "histograms only"},
{protoMsg: config.RemoteWriteProtoMsgV2, samples: false, exemplars: false, histograms: false, floatHistograms: true, name: "float histograms only"},
} {
t.Run(fmt.Sprintf("%s-%s", tc.protoMsg, tc.name), func(t *testing.T) {
dir := t.TempDir()
s := NewStorage(nil, nil, nil, dir, defaultFlushDeadline, nil, true)
defer s.Close()
@ -281,12 +279,14 @@ func TestSampleDelivery(t *testing.T) {
// Apply new config.
queueConfig.Capacity = len(samples)
queueConfig.MaxSamplesPerSend = len(samples) / 2
// For now we only ever have a single rw config in this test.
conf.RemoteWriteConfigs[0].ProtobufMessage = tc.protoMsg
require.NoError(t, s.ApplyConfig(conf))
hash, err := toHash(writeConfig)
require.NoError(t, err)
qm := s.rws.queues[hash]
c := NewTestWriteClient(tc.rwFormat)
c := NewTestWriteClient(tc.protoMsg)
qm.SetClient(c)
qm.StoreSeries(series, 0)
@ -325,7 +325,7 @@ func testDefaultQueueConfig() config.QueueConfig {
}
func TestMetadataDelivery(t *testing.T) {
c := NewTestWriteClient(Version1)
c := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
dir := t.TempDir()
@ -333,7 +333,7 @@ func TestMetadataDelivery(t *testing.T) {
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.Start()
defer m.Stop()
@ -350,6 +350,7 @@ func TestMetadataDelivery(t *testing.T) {
m.AppendWatcherMetadata(context.Background(), metadata)
require.Equal(t, 0.0, client_testutil.ToFloat64(m.metrics.failedMetadataTotal))
require.Len(t, c.receivedMetadata, numMetadata)
// One more write than the rounded qoutient should be performed in order to get samples that didn't
// fit into MaxSamplesPerSend.
@ -387,7 +388,7 @@ func TestWALMetadataDelivery(t *testing.T) {
require.NoError(t, err)
qm := s.rws.queues[hash]
c := NewTestWriteClient(Version1)
c := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
qm.SetClient(c)
qm.StoreSeries(series, 0)
@ -400,12 +401,12 @@ func TestWALMetadataDelivery(t *testing.T) {
}
func TestSampleDeliveryTimeout(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
// Let's send one less sample than batch size, and wait the timeout duration
n := 9
samples, series := createTimeseries(n, n)
c := NewTestWriteClient(rwFormat)
c := NewTestWriteClient(protoMsg)
cfg := testDefaultQueueConfig()
mcfg := config.DefaultMetadataConfig
@ -414,7 +415,7 @@ func TestSampleDeliveryTimeout(t *testing.T) {
dir := t.TempDir()
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.StoreSeries(series, 0)
m.Start()
defer m.Stop()
@ -432,8 +433,8 @@ func TestSampleDeliveryTimeout(t *testing.T) {
}
func TestSampleDeliveryOrder(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
ts := 10
n := config.DefaultQueueConfig.MaxSamplesPerSend * ts
samples := make([]record.RefSample, 0, n)
@ -451,7 +452,7 @@ func TestSampleDeliveryOrder(t *testing.T) {
})
}
c := NewTestWriteClient(rwFormat)
c := NewTestWriteClient(protoMsg)
c.expectSamples(samples, series)
dir := t.TempDir()
@ -460,7 +461,7 @@ func TestSampleDeliveryOrder(t *testing.T) {
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.StoreSeries(series, 0)
m.Start()
@ -482,7 +483,7 @@ func TestShutdown(t *testing.T) {
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, deadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, deadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
n := 2 * config.DefaultQueueConfig.MaxSamplesPerSend
samples, series := createTimeseries(n, n)
m.StoreSeries(series, 0)
@ -520,7 +521,7 @@ func TestSeriesReset(t *testing.T) {
cfg := testDefaultQueueConfig()
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, deadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, deadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
for i := 0; i < numSegments; i++ {
series := []record.RefSeries{}
for j := 0; j < numSeries; j++ {
@ -534,14 +535,14 @@ func TestSeriesReset(t *testing.T) {
}
func TestReshard(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
size := 10 // Make bigger to find more races.
nSeries := 6
nSamples := config.DefaultQueueConfig.Capacity * size
samples, series := createTimeseries(nSamples, nSeries)
c := NewTestWriteClient(rwFormat)
c := NewTestWriteClient(protoMsg)
c.expectSamples(samples, series)
cfg := testDefaultQueueConfig()
@ -551,7 +552,7 @@ func TestReshard(t *testing.T) {
dir := t.TempDir()
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.StoreSeries(series, 0)
m.Start()
@ -577,9 +578,9 @@ func TestReshard(t *testing.T) {
}
func TestReshardRaceWithStop(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
c := NewTestWriteClient(rwFormat)
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
c := NewTestWriteClient(protoMsg)
var m *QueueManager
h := sync.Mutex{}
h.Lock()
@ -590,7 +591,7 @@ func TestReshardRaceWithStop(t *testing.T) {
go func() {
for {
metrics := newQueueManagerMetrics(nil, "", "")
m = NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m = NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.Start()
h.Unlock()
h.Lock()
@ -615,8 +616,8 @@ func TestReshardRaceWithStop(t *testing.T) {
}
func TestReshardPartialBatch(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
samples, series := createTimeseries(1, 10)
c := NewTestBlockedWriteClient()
@ -629,7 +630,7 @@ func TestReshardPartialBatch(t *testing.T) {
cfg.BatchSendDeadline = model.Duration(batchSendDeadline)
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, t.TempDir(), newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, flushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, t.TempDir(), newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, flushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.StoreSeries(series, 0)
m.Start()
@ -661,8 +662,8 @@ func TestReshardPartialBatch(t *testing.T) {
// where a large scrape (> capacity + max samples per send) is appended at the
// same time as a batch times out according to the batch send deadline.
func TestQueueFilledDeadlock(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
samples, series := createTimeseries(50, 1)
c := NewNopWriteClient()
@ -678,7 +679,7 @@ func TestQueueFilledDeadlock(t *testing.T) {
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, t.TempDir(), newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, flushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, t.TempDir(), newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, flushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.StoreSeries(series, 0)
m.Start()
defer m.Stop()
@ -703,13 +704,13 @@ func TestQueueFilledDeadlock(t *testing.T) {
}
func TestReleaseNoninternedString(t *testing.T) {
for _, rwFormat := range []config.RemoteWriteFormat{Version1, Version2} {
t.Run(fmt.Sprint(rwFormat), func(t *testing.T) {
for _, protoMsg := range []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2} {
t.Run(fmt.Sprint(protoMsg), func(t *testing.T) {
cfg := testDefaultQueueConfig()
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
c := NewTestWriteClient(rwFormat)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, rwFormat)
c := NewTestWriteClient(protoMsg)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, protoMsg)
m.Start()
defer m.Stop()
for i := 1; i < 1000; i++ {
@ -757,8 +758,8 @@ func TestShouldReshard(t *testing.T) {
for _, c := range cases {
metrics := newQueueManagerMetrics(nil, "", "")
// todo: test with new proto type(s)
client := NewTestWriteClient(Version1)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, client, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
client := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, client, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.numShards = c.startingShards
m.dataIn.incr(c.samplesIn)
m.dataOut.incr(c.samplesOut)
@ -804,7 +805,7 @@ func TestDisableReshardOnRetry(t *testing.T) {
}
)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, client, 0, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, client, 0, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.StoreSeries(fakeSeries, 0)
// Attempt to samples while the manager is running. We immediately stop the
@ -955,6 +956,8 @@ func getSeriesNameFromRef(r record.RefSeries) string {
return r.Labels.Get("__name__")
}
// TestWriteClient represents write client which does not call remote storage,
// but instead re-implements fake WriteHandler for test purposes.
type TestWriteClient struct {
receivedSamples map[string][]prompb.Sample
expectedSamples map[string][]prompb.Sample
@ -968,25 +971,25 @@ type TestWriteClient struct {
writesReceived int
mtx sync.Mutex
buf []byte
rwFormat config.RemoteWriteFormat
sendAttempts []string
steps []contentNegotiationStep
currstep int
protoMsg config.RemoteWriteProtoMsg
injectedErrs []error
currErr int
retry bool
}
func NewTestWriteClient(rwFormat config.RemoteWriteFormat) *TestWriteClient {
// NewTestWriteClient creates a new testing write client.
func NewTestWriteClient(protoMsg config.RemoteWriteProtoMsg) *TestWriteClient {
return &TestWriteClient{
receivedSamples: map[string][]prompb.Sample{},
expectedSamples: map[string][]prompb.Sample{},
receivedMetadata: map[string][]prompb.MetricMetadata{},
rwFormat: rwFormat,
protoMsg: protoMsg,
}
}
func (c *TestWriteClient) setSteps(steps []contentNegotiationStep) {
c.steps = steps
c.currstep = -1 // incremented by GetLastRWHeader()
func (c *TestWriteClient) injectErrors(injectedErrs []error) {
c.injectedErrs = injectedErrs
c.currErr = -1
c.retry = false
}
@ -1096,60 +1099,56 @@ func (c *TestWriteClient) waitForExpectedData(tb testing.TB, timeout time.Durati
}
}
func (c *TestWriteClient) Store(_ context.Context, req []byte, attemptNos int, rwFormat config.RemoteWriteFormat, compression string) error {
func (c *TestWriteClient) Store(_ context.Context, req []byte, _ int) error {
c.mtx.Lock()
defer c.mtx.Unlock()
// nil buffers are ok for snappy, ignore cast error.
if c.buf != nil {
c.buf = c.buf[:cap(c.buf)]
}
// TODO(bwplotka): Consider using WriteHandler instead?
reqBuf, err := snappy.Decode(c.buf, req)
c.buf = reqBuf
if err != nil {
return err
}
attemptString := fmt.Sprintf("%d,%d,%s", attemptNos, rwFormat, compression)
if attemptNos > 0 {
// If this is a second attempt then we need to bump to the next step otherwise we loop.
c.currstep++
}
// Check if we've been told to return something for this config.
if len(c.steps) > 0 {
if err = c.steps[c.currstep].behaviour; err != nil {
c.sendAttempts = append(c.sendAttempts, attemptString+","+fmt.Sprintf("%s", err))
// Check if we've been told to inject err for this call.
if len(c.injectedErrs) > 0 {
c.currErr++
if err = c.injectedErrs[c.currErr]; err != nil {
return err
}
}
var reqProto *prompb.WriteRequest
switch rwFormat {
case Version1:
switch c.protoMsg {
case config.RemoteWriteProtoMsgV1:
reqProto = &prompb.WriteRequest{}
err = proto.Unmarshal(reqBuf, reqProto)
case Version2:
var reqMin writev2.Request
err = proto.Unmarshal(reqBuf, &reqMin)
case config.RemoteWriteProtoMsgV2:
// NOTE(bwplotka): v1 msg can be unmarshaled to v2 sometimes, without
// errors.
var reqProtoV2 writev2.Request
err = proto.Unmarshal(reqBuf, &reqProtoV2)
if err == nil {
reqProto, err = MinimizedWriteRequestToWriteRequest(&reqMin)
reqProto, err = V2WriteRequestToWriteRequest(&reqProtoV2)
}
}
if err != nil {
c.sendAttempts = append(c.sendAttempts, attemptString+","+fmt.Sprintf("%s", err))
return err
}
if len(reqProto.Timeseries) == 0 && len(reqProto.Metadata) == 0 {
return errors.New("invalid request, no timeseries")
}
builder := labels.NewScratchBuilder(0)
count := 0
for _, ts := range reqProto.Timeseries {
labels := labelProtosToLabels(&builder, ts.Labels)
seriesName := labels.Get("__name__")
for _, sample := range ts.Samples {
count++
c.receivedSamples[seriesName] = append(c.receivedSamples[seriesName], sample)
}
c.receivedSamples[seriesName] = append(c.receivedSamples[seriesName], ts.Samples...)
if len(ts.Exemplars) > 0 {
c.receivedExemplars[seriesName] = append(c.receivedExemplars[seriesName], ts.Exemplars...)
}
@ -1166,7 +1165,6 @@ func (c *TestWriteClient) Store(_ context.Context, req []byte, attemptNos int, r
}
c.writesReceived++
c.sendAttempts = append(c.sendAttempts, attemptString+",ok")
return nil
}
@ -1178,20 +1176,6 @@ func (c *TestWriteClient) Endpoint() string {
return "http://test-remote.com/1234"
}
func (c *TestWriteClient) probeRemoteVersions(_ context.Context) error {
return nil
}
func (c *TestWriteClient) GetLastRWHeader() string {
c.mtx.Lock()
defer c.mtx.Unlock()
c.currstep++
if len(c.steps) > 0 {
return c.steps[c.currstep].lastRWHeader
}
return "2.0;snappy,0.1.0"
}
// TestBlockingWriteClient is a queue_manager WriteClient which will block
// on any calls to Store(), until the request's Context is cancelled, at which
// point the `numCalls` property will contain a count of how many times Store()
@ -1204,7 +1188,7 @@ func NewTestBlockedWriteClient() *TestBlockingWriteClient {
return &TestBlockingWriteClient{}
}
func (c *TestBlockingWriteClient) Store(ctx context.Context, _ []byte, _ int, _ config.RemoteWriteFormat, _ string) error {
func (c *TestBlockingWriteClient) Store(ctx context.Context, _ []byte, _ int) error {
c.numCalls.Inc()
<-ctx.Done()
return nil
@ -1222,27 +1206,15 @@ func (c *TestBlockingWriteClient) Endpoint() string {
return "http://test-remote-blocking.com/1234"
}
func (c *TestBlockingWriteClient) probeRemoteVersions(_ context.Context) error {
return nil
}
func (c *TestBlockingWriteClient) GetLastRWHeader() string {
return "2.0;snappy,0.1.0"
}
// For benchmarking the send and not the receive side.
type NopWriteClient struct{}
func NewNopWriteClient() *NopWriteClient { return &NopWriteClient{} }
func (c *NopWriteClient) Store(context.Context, []byte, int, config.RemoteWriteFormat, string) error {
func (c *NopWriteClient) Store(context.Context, []byte, int) error {
return nil
}
func (c *NopWriteClient) Name() string { return "nopwriteclient" }
func (c *NopWriteClient) Endpoint() string { return "http://test-remote.com/1234" }
func (c *NopWriteClient) probeRemoteVersions(_ context.Context) error {
return nil
}
func (c *NopWriteClient) GetLastRWHeader() string { return "2.0;snappy,0.1.0" }
type MockWriteClient struct {
StoreFunc func(context.Context, []byte, int) error
@ -1250,20 +1222,12 @@ type MockWriteClient struct {
EndpointFunc func() string
}
func (c *MockWriteClient) Store(ctx context.Context, bb []byte, n int, _ config.RemoteWriteFormat, _ string) error {
func (c *MockWriteClient) Store(ctx context.Context, bb []byte, n int) error {
return c.StoreFunc(ctx, bb, n)
}
func (c *MockWriteClient) Name() string { return c.NameFunc() }
func (c *MockWriteClient) Endpoint() string { return c.EndpointFunc() }
// TODO(bwplotka): Mock it if needed.
func (c *MockWriteClient) GetLastRWHeader() string { return "2.0;snappy,0.1.0" }
// TODO(bwplotka): Mock it if needed.
func (c *MockWriteClient) probeRemoteVersions(_ context.Context) error {
return nil
}
// Extra labels to make a more realistic workload - taken from Kubernetes' embedded cAdvisor metrics.
var extraLabels []labels.Label = []labels.Label{
{Name: "kubernetes_io_arch", Value: "amd64"},
@ -1302,7 +1266,7 @@ func BenchmarkSampleSend(b *testing.B) {
metrics := newQueueManagerMetrics(nil, "", "")
// todo: test with new proto type(s)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.StoreSeries(series, 0)
// These should be received by the client.
@ -1356,12 +1320,12 @@ func BenchmarkStoreSeries(b *testing.B) {
for _, tc := range testCases {
b.Run(tc.name, func(b *testing.B) {
for i := 0; i < b.N; i++ {
c := NewTestWriteClient(Version1)
c := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
dir := b.TempDir()
cfg := config.DefaultQueueConfig
mcfg := config.DefaultMetadataConfig
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.externalLabels = tc.externalLabels
m.relabelConfigs = tc.relabelConfigs
@ -1401,7 +1365,7 @@ func BenchmarkStartup(b *testing.B) {
// todo: test with new proto type(s)
m := NewQueueManager(metrics, nil, nil, logger, dir,
newEWMARate(ewmaWeight, shardUpdateDuration),
cfg, mcfg, labels.EmptyLabels(), nil, c, 1*time.Minute, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
cfg, mcfg, labels.EmptyLabels(), nil, c, 1*time.Minute, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.watcher.SetStartTime(timestamp.Time(math.MaxInt64))
m.watcher.MaxSegment = segments[len(segments)-2]
err := m.watcher.Run()
@ -1488,7 +1452,7 @@ func TestCalculateDesiredShards(t *testing.T) {
metrics := newQueueManagerMetrics(nil, "", "")
samplesIn := newEWMARate(ewmaWeight, shardUpdateDuration)
// todo: test with new proto type(s)
m := NewQueueManager(metrics, nil, nil, nil, dir, samplesIn, cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, samplesIn, cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
// Need to start the queue manager so the proper metrics are initialized.
// However we can stop it right away since we don't need to do any actual
@ -1557,7 +1521,7 @@ func TestCalculateDesiredShards(t *testing.T) {
}
func TestCalculateDesiredShardsDetail(t *testing.T) {
c := NewTestWriteClient(Version1)
c := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
cfg := config.DefaultQueueConfig
mcfg := config.DefaultMetadataConfig
@ -1566,7 +1530,7 @@ func TestCalculateDesiredShardsDetail(t *testing.T) {
metrics := newQueueManagerMetrics(nil, "", "")
samplesIn := newEWMARate(ewmaWeight, shardUpdateDuration)
// todo: test with new proto type(s)
m := NewQueueManager(metrics, nil, nil, nil, dir, samplesIn, cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, samplesIn, cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
for _, tc := range []struct {
name string
@ -1886,7 +1850,7 @@ func BenchmarkBuildWriteRequest(b *testing.B) {
})
}
func BenchmarkBuildMinimizedWriteRequest(b *testing.B) {
func BenchmarkBuildV2WriteRequest(b *testing.B) {
noopLogger := log.NewNopLogger()
type testcase struct {
batch []timeSeries
@ -1897,7 +1861,7 @@ func BenchmarkBuildMinimizedWriteRequest(b *testing.B) {
{createDummyTimeSeries(100)},
}
for _, tc := range testCases {
symbolTable := newRwSymbolTable()
symbolTable := writev2.NewSymbolTable()
buff := make([]byte, 0)
seriesBuff := make([]writev2.TimeSeries, len(tc.batch))
for i := range seriesBuff {
@ -1909,7 +1873,7 @@ func BenchmarkBuildMinimizedWriteRequest(b *testing.B) {
// Warmup buffers
for i := 0; i < 10; i++ {
populateV2TimeSeries(&symbolTable, tc.batch, seriesBuff, true, true)
buildV2WriteRequest(noopLogger, seriesBuff, symbolTable.LabelsStrings(), &pBuf, &buff, nil, "snappy")
buildV2WriteRequest(noopLogger, seriesBuff, symbolTable.Symbols(), &pBuf, &buff, nil, "snappy")
}
b.Run(fmt.Sprintf("%d-instances", len(tc.batch)), func(b *testing.B) {
@ -1917,11 +1881,11 @@ func BenchmarkBuildMinimizedWriteRequest(b *testing.B) {
for j := 0; j < b.N; j++ {
populateV2TimeSeries(&symbolTable, tc.batch, seriesBuff, true, true)
b.ResetTimer()
req, _, _, err := buildV2WriteRequest(noopLogger, seriesBuff, symbolTable.LabelsStrings(), &pBuf, &buff, nil, "snappy")
req, _, _, err := buildV2WriteRequest(noopLogger, seriesBuff, symbolTable.Symbols(), &pBuf, &buff, nil, "snappy")
if err != nil {
b.Fatal(err)
}
symbolTable.clear()
symbolTable.Reset()
totalSize += len(req)
b.ReportMetric(float64(totalSize)/float64(b.N), "compressedSize/op")
}
@ -1936,7 +1900,7 @@ func TestDropOldTimeSeries(t *testing.T) {
samples, newSamples, series := createTimeseriesWithOldSamples(nSamples, nSeries)
// TODO(alexg): test with new version
c := NewTestWriteClient(Version1)
c := NewTestWriteClient(config.RemoteWriteProtoMsgV1)
c.expectSamples(newSamples, series)
cfg := config.DefaultQueueConfig
@ -1946,7 +1910,7 @@ func TestDropOldTimeSeries(t *testing.T) {
dir := t.TempDir()
metrics := newQueueManagerMetrics(nil, "", "")
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, Version1)
m := NewQueueManager(metrics, nil, nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, mcfg, labels.EmptyLabels(), nil, c, defaultFlushDeadline, newPool(), newHighestTimestampMetric(), nil, false, false, config.RemoteWriteProtoMsgV1)
m.StoreSeries(series, 0)
m.Start()

View file

@ -169,25 +169,15 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error {
name = rwConf.Name
}
// TODO(bwplotka): Remove in the next PR (split for readability).
protoVersion := func() config.RemoteWriteFormat {
switch rwConf.ProtobufMessage {
case config.RemoteWriteProtoMsgV2:
return Version2
default:
return Version1
}
}()
c, err := NewWriteClient(name, &ClientConfig{
URL: rwConf.URL,
RemoteWriteFormat: protoVersion,
Timeout: rwConf.RemoteTimeout,
HTTPClientConfig: rwConf.HTTPClientConfig,
SigV4Config: rwConf.SigV4Config,
AzureADConfig: rwConf.AzureADConfig,
Headers: rwConf.Headers,
RetryOnRateLimit: rwConf.QueueConfig.RetryOnRateLimit,
URL: rwConf.URL,
WriteProtoMsg: rwConf.ProtobufMessage,
Timeout: rwConf.RemoteTimeout,
HTTPClientConfig: rwConf.HTTPClientConfig,
SigV4Config: rwConf.SigV4Config,
AzureADConfig: rwConf.AzureADConfig,
Headers: rwConf.Headers,
RetryOnRateLimit: rwConf.QueueConfig.RetryOnRateLimit,
})
if err != nil {
return err
@ -202,21 +192,6 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error {
continue
}
// Work out what protocol and compression to use for this endpoint.
// Default to Remote Write Version1.
rwFormat := Version1
switch protoVersion {
case Version1:
// We use the standard value as there's no negotiation to be had.
case Version2:
rwFormat = Version2
// If this newer remote write format is enabled then we need to probe the remote server
// to work out the desired protocol version and compressions.
// The value of the header is kept in the client so no need to see it here.
_ = c.probeRemoteVersions(context.Background())
// We ignore any error here, at some point we may choose to log it.
}
// Redacted to remove any passwords in the URL (that are
// technically accepted but not recommended) since this is
// only used for metric labels.
@ -239,7 +214,7 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error {
rws.scraper,
rwConf.SendExemplars,
rwConf.SendNativeHistograms,
rwFormat,
rwConf.ProtobufMessage,
)
// Keep track of which queues are new so we know which to start.
newHashes = append(newHashes, hash)

View file

@ -23,105 +23,39 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2"
"github.com/gogo/protobuf/proto"
"github.com/golang/snappy"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/exemplar"
"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"
otlptranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
)
const (
RemoteWriteVersionHeader = "X-Prometheus-Remote-Write-Version"
RemoteWriteVersion1HeaderValue = "0.1.0"
RemoteWriteVersion20HeaderValue = "2.0"
)
func rwHeaderNameValues(rwFormat config.RemoteWriteFormat) map[string]string {
// Return the correct remote write header name/values based on provided rwFormat.
ret := make(map[string]string, 1)
switch rwFormat {
case Version1:
ret[RemoteWriteVersionHeader] = RemoteWriteVersion1HeaderValue
case Version2:
// We need to add the supported protocol definitions in order:
tuples := make([]string, 0, 2)
// Add "2.0;snappy".
tuples = append(tuples, RemoteWriteVersion20HeaderValue+";snappy")
// Add default "0.1.0".
tuples = append(tuples, RemoteWriteVersion1HeaderValue)
ret[RemoteWriteVersionHeader] = strings.Join(tuples, ",")
}
return ret
}
type writeHeadHandler struct {
logger log.Logger
remoteWriteHeadRequests prometheus.Counter
// Experimental feature, new remote write proto format.
// The handler will accept the new format, but it can still accept the old one.
rwFormat config.RemoteWriteFormat
}
func NewWriteHeadHandler(logger log.Logger, reg prometheus.Registerer, rwFormat config.RemoteWriteFormat) http.Handler {
h := &writeHeadHandler{
logger: logger,
rwFormat: rwFormat,
remoteWriteHeadRequests: prometheus.NewCounter(prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "api",
Name: "remote_write_head_requests",
Help: "The number of remote write HEAD requests.",
}),
}
if reg != nil {
reg.MustRegister(h.remoteWriteHeadRequests)
}
return h
}
// Send a response to the HEAD request based on the format supported.
func (h *writeHeadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// Add appropriate header values for the specific rwFormat.
for hName, hValue := range rwHeaderNameValues(h.rwFormat) {
w.Header().Set(hName, hValue)
}
// Increment counter
h.remoteWriteHeadRequests.Inc()
w.WriteHeader(http.StatusOK)
}
type writeHandler struct {
logger log.Logger
appendable storage.Appendable
samplesWithInvalidLabelsTotal prometheus.Counter
// Experimental feature, new remote write proto format.
// The handler will accept the new format, but it can still accept the old one.
rwFormat config.RemoteWriteFormat
acceptedProtoMsgs map[config.RemoteWriteProtoMsg]struct{}
}
// NewWriteHandler creates a http.Handler that accepts remote write requests and
// writes them to the provided appendable.
func NewWriteHandler(logger log.Logger, reg prometheus.Registerer, appendable storage.Appendable, rwFormat config.RemoteWriteFormat) http.Handler {
// NewWriteHandler creates a http.Handler that accepts remote write requests with
// the given message in acceptedProtoMsgs and writes them to the provided appendable.
func NewWriteHandler(logger log.Logger, reg prometheus.Registerer, appendable storage.Appendable, acceptedProtoMsgs []config.RemoteWriteProtoMsg) http.Handler {
protoMsgs := map[config.RemoteWriteProtoMsg]struct{}{}
for _, acc := range acceptedProtoMsgs {
protoMsgs[acc] = struct{}{}
}
h := &writeHandler{
logger: logger,
appendable: appendable,
rwFormat: rwFormat,
logger: logger,
appendable: appendable,
acceptedProtoMsgs: protoMsgs,
samplesWithInvalidLabelsTotal: prometheus.NewCounter(prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "api",
@ -135,35 +69,67 @@ func NewWriteHandler(logger log.Logger, reg prometheus.Registerer, appendable st
return h
}
func (h *writeHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
var err error
func (h *writeHandler) parseProtoMsg(contentType string) (config.RemoteWriteProtoMsg, error) {
contentType = strings.TrimSpace(contentType)
// Set the header(s) in the response based on the rwFormat the server supports.
for hName, hValue := range rwHeaderNameValues(h.rwFormat) {
w.Header().Set(hName, hValue)
parts := strings.Split(contentType, ";")
if parts[0] != appProtoContentType {
return "", fmt.Errorf("expected %v as the first (media) part, got %v content-type", appProtoContentType, contentType)
}
// Parse potential https://www.rfc-editor.org/rfc/rfc9110#parameter
for _, p := range parts[1:] {
pair := strings.Split(p, "=")
if len(pair) != 2 {
return "", fmt.Errorf("as per https://www.rfc-editor.org/rfc/rfc9110#parameter expected parameters to be key-values, got %v in %v content-type", p, contentType)
}
if pair[0] == "proto" {
ret := config.RemoteWriteProtoMsg(pair[1])
if err := ret.Validate(); err != nil {
return "", fmt.Errorf("got %v content type; %w", contentType, err)
}
return ret, nil
}
}
// No "proto=" parameter, assuming v1.
return config.RemoteWriteProtoMsgV1, nil
}
func (h *writeHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
contentType := r.Header.Get("Content-Type")
if contentType == "" {
// Don't break yolo 1.0 clients if not needed. This is similar to what we did
// before 2.0: https://github.com/prometheus/prometheus/blob/d78253319daa62c8f28ed47e40bafcad2dd8b586/storage/remote/write_handler.go#L62
// We could give http.StatusUnsupportedMediaType, but let's assume 1.0 message by default.
contentType = appProtoContentType
}
// Parse the headers to work out how to handle this.
contentEncoding := r.Header.Get("Content-Encoding")
protoVer := r.Header.Get(RemoteWriteVersionHeader)
switch protoVer {
case "":
// No header provided, assume 0.1.0 as everything that relies on later.
protoVer = RemoteWriteVersion1HeaderValue
case RemoteWriteVersion1HeaderValue, RemoteWriteVersion20HeaderValue:
// We know this header, woo.
default:
// We have a version in the header but it is not one we recognise.
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", "Unknown remote write version in headers", "ver", protoVer)
// Return a 406 so that the client can choose a more appropriate protocol to use.
http.Error(w, "Unknown remote write version in headers", http.StatusNotAcceptable)
msg, err := h.parseProtoMsg(contentType)
if err != nil {
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err)
http.Error(w, err.Error(), http.StatusUnsupportedMediaType)
return
}
// Deal with 0.1.0 clients that forget to send Content-Encoding.
if protoVer == RemoteWriteVersion1HeaderValue && contentEncoding == "" {
contentEncoding = "snappy"
if _, ok := h.acceptedProtoMsgs[msg]; !ok {
err := fmt.Errorf("%v protobuf message is not accepted by this server; accepted %v", msg, func() (ret []string) {
for k := range h.acceptedProtoMsgs {
ret = append(ret, string(k))
}
return ret
}())
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err)
http.Error(w, err.Error(), http.StatusUnsupportedMediaType)
}
enc := r.Header.Get("Content-Encoding")
if enc == "" {
// Don't break yolo 1.0 clients if not needed. This is similar to what we did
// before 2.0: https://github.com/prometheus/prometheus/blob/d78253319daa62c8f28ed47e40bafcad2dd8b586/storage/remote/write_handler.go#L62
// We could give http.StatusUnsupportedMediaType, but let's assume snappy by default.
} else if enc != string(SnappyBlockCompression) {
err := fmt.Errorf("%v encoding (compression) is not accepted by this server; only %v is acceptable", enc, SnappyBlockCompression)
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err)
http.Error(w, err.Error(), http.StatusUnsupportedMediaType)
}
// Read the request body.
@ -174,44 +140,34 @@ func (h *writeHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
return
}
// Deal with contentEncoding first.
var decompressed []byte
switch contentEncoding {
case "snappy":
decompressed, err = snappy.Decode(nil, body)
if err != nil {
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err.Error())
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
default:
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", "Unsupported Content-Encoding", "contentEncoding", contentEncoding)
// Return a 406 so that the client can choose a more appropriate protocol to use.
http.Error(w, "Unsupported Content-Encoding", http.StatusNotAcceptable)
decompressed, err := snappy.Decode(nil, body)
if err != nil {
// TODO(bwplotka): Add more context to responded error?
level.Error(h.logger).Log("msg", "Error decompressing remote write request", "err", err.Error())
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
// Now we have a decompressed buffer we can unmarshal it.
// At this point we are happy with the version but need to check the encoding.
switch protoVer {
case RemoteWriteVersion1HeaderValue:
switch msg {
case config.RemoteWriteProtoMsgV1:
var req prompb.WriteRequest
if err := proto.Unmarshal(decompressed, &req); err != nil {
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err.Error())
// TODO(bwplotka): Add more context to responded error?
level.Error(h.logger).Log("msg", "Error decoding v1 remote write request", "protobuf_message", msg, "err", err.Error())
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
err = h.write(r.Context(), &req)
case RemoteWriteVersion20HeaderValue:
// 2.0 request.
var reqMinStr writev2.Request
if err := proto.Unmarshal(decompressed, &reqMinStr); err != nil {
level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err.Error())
case config.RemoteWriteProtoMsgV2:
var req writev2.Request
if err := proto.Unmarshal(decompressed, &req); err != nil {
// TODO(bwplotka): Add more context to responded error?
level.Error(h.logger).Log("msg", "Error decoding v2 remote write request", "protobuf_message", msg, "err", err.Error())
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
err = h.writeMinStr(r.Context(), &reqMinStr)
err = h.writeV2(r.Context(), &req)
}
switch {
@ -295,6 +251,49 @@ func (h *writeHandler) write(ctx context.Context, req *prompb.WriteRequest) (err
return nil
}
func (h *writeHandler) writeV2(ctx context.Context, req *writev2.Request) (err error) {
outOfOrderExemplarErrs := 0
app := h.appendable.Appender(ctx)
defer func() {
if err != nil {
_ = app.Rollback()
return
}
err = app.Commit()
}()
for _, ts := range req.Timeseries {
ls := writev2.DesymbolizeLabels(ts.LabelsRefs, req.Symbols)
err := h.appendSamplesV2(app, ts.Samples, ls)
if err != nil {
return err
}
for _, ep := range ts.Exemplars {
e := exemplarProtoV2ToExemplar(ep, req.Symbols)
h.appendExemplar(app, e, ls, &outOfOrderExemplarErrs)
}
err = h.appendHistogramsV2(app, ts.Histograms, ls)
if err != nil {
return err
}
m := metadataProtoV2ToMetadata(ts.Metadata, req.Symbols)
if _, err = app.UpdateMetadata(0, ls, m); err != nil {
level.Debug(h.logger).Log("msg", "error while updating metadata from remote write", "err", err)
}
}
if outOfOrderExemplarErrs > 0 {
_ = level.Warn(h.logger).Log("msg", "Error on ingesting out-of-order exemplars", "num_dropped", outOfOrderExemplarErrs)
}
return nil
}
func (h *writeHandler) appendExemplar(app storage.Appender, e exemplar.Exemplar, labels labels.Labels, outOfOrderExemplarErrs *int) {
_, err := app.AppendExemplar(0, labels, e)
err = h.checkAppendExemplarError(err, e, outOfOrderExemplarErrs)
@ -323,7 +322,7 @@ func (h *writeHandler) appendSamples(app storage.Appender, ss []prompb.Sample, l
return nil
}
func (h *writeHandler) appendMinSamples(app storage.Appender, ss []writev2.Sample, labels labels.Labels) error {
func (h *writeHandler) appendSamplesV2(app storage.Appender, ss []writev2.Sample, labels labels.Labels) error {
var ref storage.SeriesRef
var err error
for _, s := range ss {
@ -368,14 +367,14 @@ func (h *writeHandler) appendHistograms(app storage.Appender, hh []prompb.Histog
return nil
}
func (h *writeHandler) appendMinHistograms(app storage.Appender, hh []writev2.Histogram, labels labels.Labels) error {
func (h *writeHandler) appendHistogramsV2(app storage.Appender, hh []writev2.Histogram, labels labels.Labels) error {
var err error
for _, hp := range hh {
if hp.IsFloatHistogram() {
fhs := FloatMinHistogramProtoToFloatHistogram(hp)
fhs := FloatV2HistogramProtoToFloatHistogram(hp)
_, err = app.AppendHistogram(0, labels, hp.Timestamp, nil, fhs)
} else {
hs := MinHistogramProtoToHistogram(hp)
hs := V2HistogramProtoToHistogram(hp)
_, err = app.AppendHistogram(0, labels, hp.Timestamp, hs, nil)
}
if err != nil {
@ -452,46 +451,3 @@ func (h *otlpWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusOK)
}
func (h *writeHandler) writeMinStr(ctx context.Context, req *writev2.Request) (err error) {
outOfOrderExemplarErrs := 0
app := h.appendable.Appender(ctx)
defer func() {
if err != nil {
_ = app.Rollback()
return
}
err = app.Commit()
}()
for _, ts := range req.Timeseries {
ls := labelProtosV2ToLabels(ts.LabelsRefs, req.Symbols)
err := h.appendMinSamples(app, ts.Samples, ls)
if err != nil {
return err
}
for _, ep := range ts.Exemplars {
e := exemplarProtoV2ToExemplar(ep, req.Symbols)
h.appendExemplar(app, e, ls, &outOfOrderExemplarErrs)
}
err = h.appendMinHistograms(app, ts.Histograms, ls)
if err != nil {
return err
}
m := metadataProtoV2ToMetadata(ts.Metadata, req.Symbols)
if _, err = app.UpdateMetadata(0, ls, m); err != nil {
level.Debug(h.logger).Log("msg", "error while updating metadata from remote write", "err", err)
}
}
if outOfOrderExemplarErrs > 0 {
_ = level.Warn(h.logger).Log("msg", "Error on ingesting out-of-order exemplars", "num_dropped", outOfOrderExemplarErrs)
}
return nil
}

View file

@ -29,6 +29,9 @@ import (
"github.com/google/go-cmp/cmp"
"github.com/stretchr/testify/require"
"github.com/prometheus/prometheus/config"
writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2"
"github.com/prometheus/prometheus/model/exemplar"
"github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/labels"
@ -39,95 +42,218 @@ import (
"github.com/prometheus/prometheus/util/testutil"
)
func TestRemoteWriteHeadHandler(t *testing.T) {
handler := NewWriteHeadHandler(log.NewNopLogger(), nil, Version2)
req, err := http.NewRequest(http.MethodHead, "", nil)
func TestRemoteWriteHandlerHeadersHandling_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, writeRequestFixture.Timeseries, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
for _, tc := range []struct {
name string
reqHeaders map[string]string
expectedCode int
}{
// Generally Prometheus 1.0 Receiver never checked for existence of the headers, so
// we keep things permissive.
{
name: "correct PRW 1.0 headers",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV1],
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusNoContent,
},
{
name: "missing remote write version",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV1],
"Content-Encoding": string(SnappyBlockCompression),
},
expectedCode: http.StatusNoContent,
},
{
name: "no headers",
reqHeaders: map[string]string{},
expectedCode: http.StatusNoContent,
},
{
name: "missing content-type",
reqHeaders: map[string]string{
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusNoContent,
},
{
name: "missing content-encoding",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV1],
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusNoContent,
},
{
name: "wrong content-type",
reqHeaders: map[string]string{
"Content-Type": "yolo",
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "wrong content-type2",
reqHeaders: map[string]string{
"Content-Type": appProtoContentType + ";proto=yolo",
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "not supported content-encoding",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV1],
"Content-Encoding": "zstd",
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
} {
t.Run(tc.name, func(t *testing.T) {
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
for k, v := range tc.reqHeaders {
req.Header.Set(k, v)
}
resp := recorder.Result()
require.Equal(t, http.StatusOK, resp.StatusCode)
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
// Check header is expected value.
protHeader := resp.Header.Get(RemoteWriteVersionHeader)
require.Equal(t, "2.0;snappy,0.1.0", protHeader)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
out, err := io.ReadAll(resp.Body)
require.NoError(t, err)
_ = resp.Body.Close()
require.Equal(t, tc.expectedCode, resp.StatusCode, string(out))
})
}
}
func TestRemoteWriteHandlerMinimizedMissingContentEncoding(t *testing.T) {
// Send a v2 request without a "Content-Encoding:" header -> 406.
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeRequestMinimizedFixture.Timeseries, writeRequestMinimizedFixture.Symbols, nil, nil, nil, "snappy")
func TestRemoteWriteHandlerHeadersHandling_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeV2RequestFixture.Timeseries, writeV2RequestFixture.Symbols, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
// Do not provide "Content-Encoding: snappy" header.
// req.Header.Set("Content-Encoding", "snappy")
require.NoError(t, err)
for _, tc := range []struct {
name string
reqHeaders map[string]string
expectedCode int
}{
{
name: "correct PRW 2.0 headers",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2],
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusNoContent,
},
{
name: "missing remote write version",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2],
"Content-Encoding": string(SnappyBlockCompression),
},
expectedCode: http.StatusNoContent, // We don't check for now.
},
{
name: "no headers",
reqHeaders: map[string]string{},
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "missing content-type",
reqHeaders: map[string]string{
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
// This only gives 415, because we explicitly only support 2.0. If we supported both
// (default) it would be empty message parsed and ok response.
// This is perhaps better, than 415 for previously working 1.0 flow with
// no content-type.
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "missing content-encoding",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2],
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusNoContent, // Similar to 1.0 impl, we default to Snappy, so it works.
},
{
name: "wrong content-type",
reqHeaders: map[string]string{
"Content-Type": "yolo",
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "wrong content-type2",
reqHeaders: map[string]string{
"Content-Type": appProtoContentType + ";proto=yolo",
"Content-Encoding": string(SnappyBlockCompression),
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
{
name: "not supported content-encoding",
reqHeaders: map[string]string{
"Content-Type": remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2],
"Content-Encoding": "zstd",
RemoteWriteVersionHeader: RemoteWriteVersion20HeaderValue,
},
expectedCode: http.StatusUnsupportedMediaType,
},
} {
t.Run(tc.name, func(t *testing.T) {
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
for k, v := range tc.reqHeaders {
req.Header.Set(k, v)
}
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version2)
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
// Should give us a 406.
require.Equal(t, http.StatusNotAcceptable, resp.StatusCode)
resp := recorder.Result()
out, err := io.ReadAll(resp.Body)
require.NoError(t, err)
_ = resp.Body.Close()
require.Equal(t, tc.expectedCode, resp.StatusCode, string(out))
})
}
}
func TestRemoteWriteHandlerInvalidCompression(t *testing.T) {
// Send a v2 request without an unhandled compression scheme -> 406.
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeRequestMinimizedFixture.Timeseries, writeRequestMinimizedFixture.Symbols, nil, nil, nil, "snappy")
func TestRemoteWriteHandler_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, writeRequestFixture.Timeseries, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
req.Header.Set("Content-Encoding", "zstd")
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
// NOTE: Strictly speaking, even for 1.0 we require headers, but we never verified those
// in Prometheus, so keeping like this to not break existing 1.0 clients.
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version2)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
// Expect a 406.
require.Equal(t, http.StatusNotAcceptable, resp.StatusCode)
}
func TestRemoteWriteHandlerInvalidVersion(t *testing.T) {
// Send a protocol version number that isn't recognised/supported -> 406.
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeRequestMinimizedFixture.Timeseries, writeRequestMinimizedFixture.Symbols, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req.Header.Set(RemoteWriteVersionHeader, "3.0")
require.NoError(t, err)
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version2)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
// Expect a 406.
require.Equal(t, http.StatusNotAcceptable, resp.StatusCode)
}
func TestRemoteWriteHandler(t *testing.T) {
buf, _, _, err := buildWriteRequest(nil, writeRequestFixture.Timeseries, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
require.NoError(t, err)
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -135,10 +261,6 @@ func TestRemoteWriteHandler(t *testing.T) {
resp := recorder.Result()
require.Equal(t, http.StatusNoContent, resp.StatusCode)
// Check header is expected value.
protHeader := resp.Header.Get(RemoteWriteVersionHeader)
require.Equal(t, "0.1.0", protHeader)
b := labels.NewScratchBuilder(0)
i := 0
j := 0
@ -170,18 +292,19 @@ func TestRemoteWriteHandler(t *testing.T) {
}
}
func TestRemoteWriteHandlerMinimizedFormat(t *testing.T) {
buf, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeRequestMinimizedFixture.Timeseries, writeRequestMinimizedFixture.Symbols, nil, nil, nil, "snappy")
func TestRemoteWriteHandler_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeV2RequestFixture.Timeseries, writeV2RequestFixture.Symbols, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
// Must provide "Content-Encoding: snappy" header.
req.Header.Set("Content-Encoding", "snappy")
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
req.Header.Set("Content-Type", remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2])
req.Header.Set("Content-Encoding", string(SnappyBlockCompression))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
appendable := &mockAppendable{}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version2)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -189,24 +312,20 @@ func TestRemoteWriteHandlerMinimizedFormat(t *testing.T) {
resp := recorder.Result()
require.Equal(t, http.StatusNoContent, resp.StatusCode)
// Check header is expected value.
protHeader := resp.Header.Get(RemoteWriteVersionHeader)
require.Equal(t, "2.0;snappy,0.1.0", protHeader)
i := 0
j := 0
k := 0
// the reduced write request is equivalent to the write request fixture.
// we can use it for
for _, ts := range writeRequestMinimizedFixture.Timeseries {
ls := labelProtosV2ToLabels(ts.LabelsRefs, writeRequestMinimizedFixture.Symbols)
for _, ts := range writeV2RequestFixture.Timeseries {
ls := writev2.DesymbolizeLabels(ts.LabelsRefs, writeV2RequestFixture.Symbols)
for _, s := range ts.Samples {
require.Equal(t, mockSample{ls, s.Timestamp, s.Value}, appendable.samples[i])
i++
}
for _, e := range ts.Exemplars {
exemplarLabels := labelProtosV2ToLabels(e.LabelsRefs, writeRequestMinimizedFixture.Symbols)
exemplarLabels := writev2.DesymbolizeLabels(e.LabelsRefs, writeV2RequestFixture.Symbols)
require.Equal(t, mockExemplar{ls, exemplarLabels, e.Timestamp, e.Value}, appendable.exemplars[j])
j++
}
@ -223,25 +342,46 @@ func TestRemoteWriteHandlerMinimizedFormat(t *testing.T) {
k++
}
// todo: check for metadata
// TODO: check for metadata
}
}
func TestOutOfOrderSample(t *testing.T) {
buf, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
func TestOutOfOrderSample_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
Labels: []prompb.Label{{Name: "__name__", Value: "test_metric"}},
Samples: []prompb.Sample{{Value: 1, Timestamp: 0}},
}}, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
appendable := &mockAppendable{
latestSample: 100,
}
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
appendable := &mockAppendable{latestSample: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
require.Equal(t, http.StatusBadRequest, resp.StatusCode)
}
func TestOutOfOrderSample_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(nil, []writev2.TimeSeries{{
LabelsRefs: []uint32{0, 1},
Samples: []writev2.Sample{{Value: 1, Timestamp: 0}},
}}, []string{"__name__", "metric1"}, nil, nil, nil, "snappy") // TODO(bwplotka): No empty string!
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
req.Header.Set("Content-Type", remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2])
req.Header.Set("Content-Encoding", string(SnappyBlockCompression))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
appendable := &mockAppendable{latestSample: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -251,23 +391,20 @@ func TestOutOfOrderSample(t *testing.T) {
}
// This test case currently aims to verify that the WriteHandler endpoint
// don't fail on ingestion errors since the exemplar storage is
// don't fail on exemplar ingestion errors since the exemplar storage is
// still experimental.
func TestOutOfOrderExemplar(t *testing.T) {
buf, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
func TestOutOfOrderExemplar_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
Labels: []prompb.Label{{Name: "__name__", Value: "test_metric"}},
Exemplars: []prompb.Exemplar{{Labels: []prompb.Label{{Name: "foo", Value: "bar"}}, Value: 1, Timestamp: 0}},
}}, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
appendable := &mockAppendable{
latestExemplar: 100,
}
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
appendable := &mockAppendable{latestExemplar: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -277,21 +414,43 @@ func TestOutOfOrderExemplar(t *testing.T) {
require.Equal(t, http.StatusNoContent, resp.StatusCode)
}
func TestOutOfOrderHistogram(t *testing.T) {
buf, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
func TestOutOfOrderExemplar_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(nil, []writev2.TimeSeries{{
LabelsRefs: []uint32{0, 1},
Exemplars: []writev2.Exemplar{{LabelsRefs: []uint32{2, 3}, Value: 1, Timestamp: 0}},
}}, []string{"__name__", "metric1", "foo", "bar"}, nil, nil, nil, "snappy") // TODO(bwplotka): No empty string!
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
req.Header.Set("Content-Type", remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2])
req.Header.Set("Content-Encoding", string(SnappyBlockCompression))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
appendable := &mockAppendable{latestExemplar: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
// TODO: update to require.Equal(t, http.StatusConflict, resp.StatusCode) once exemplar storage is not experimental.
require.Equal(t, http.StatusNoContent, resp.StatusCode)
}
func TestOutOfOrderHistogram_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, []prompb.TimeSeries{{
Labels: []prompb.Label{{Name: "__name__", Value: "test_metric"}},
Histograms: []prompb.Histogram{HistogramToHistogramProto(0, &testHistogram), FloatHistogramToHistogramProto(1, testHistogram.ToFloat(nil))},
}}, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
appendable := &mockAppendable{
latestHistogram: 100,
}
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
appendable := &mockAppendable{latestHistogram: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -300,7 +459,31 @@ func TestOutOfOrderHistogram(t *testing.T) {
require.Equal(t, http.StatusBadRequest, resp.StatusCode)
}
func BenchmarkRemoteWritehandler(b *testing.B) {
func TestOutOfOrderHistogram_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(nil, []writev2.TimeSeries{{
LabelsRefs: []uint32{0, 1},
Histograms: []writev2.Histogram{HistogramToV2HistogramProto(0, &testHistogram), FloatHistogramToV2HistogramProto(1, testHistogram.ToFloat(nil))},
}}, []string{"__name__", "metric1"}, nil, nil, nil, "snappy") // TODO(bwplotka): No empty string!
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
req.Header.Set("Content-Type", remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2])
req.Header.Set("Content-Encoding", string(SnappyBlockCompression))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
appendable := &mockAppendable{latestHistogram: 100}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
require.Equal(t, http.StatusBadRequest, resp.StatusCode)
}
func BenchmarkRemoteWriteHandler(b *testing.B) {
const labelValue = "abcdefg'hijlmn234!@#$%^&*()_+~`\"{}[],./<>?hello0123hiOlá你好Dzieńdobry9Zd8ra765v4stvuyte"
var reqs []*http.Request
for i := 0; i < b.N; i++ {
@ -320,7 +503,7 @@ func BenchmarkRemoteWritehandler(b *testing.B) {
appendable := &mockAppendable{}
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
b.ResetTimer()
@ -329,18 +512,39 @@ func BenchmarkRemoteWritehandler(b *testing.B) {
}
}
func TestCommitErr(t *testing.T) {
buf, _, _, err := buildWriteRequest(nil, writeRequestFixture.Timeseries, nil, nil, nil, nil, "snappy")
func TestCommitErr_V1Message(t *testing.T) {
payload, _, _, err := buildWriteRequest(nil, writeRequestFixture.Timeseries, nil, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(buf))
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
appendable := &mockAppendable{
commitErr: fmt.Errorf("commit error"),
}
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, Version1)
appendable := &mockAppendable{commitErr: fmt.Errorf("commit error")}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
resp := recorder.Result()
body, err := io.ReadAll(resp.Body)
require.NoError(t, err)
require.Equal(t, http.StatusInternalServerError, resp.StatusCode)
require.Equal(t, "commit error\n", string(body))
}
func TestCommitErr_V2Message(t *testing.T) {
payload, _, _, err := buildV2WriteRequest(log.NewNopLogger(), writeV2RequestFixture.Timeseries, writeV2RequestFixture.Symbols, nil, nil, nil, "snappy")
require.NoError(t, err)
req, err := http.NewRequest("", "", bytes.NewReader(payload))
require.NoError(t, err)
req.Header.Set("Content-Type", remoteWriteContentTypeHeaders[config.RemoteWriteProtoMsgV2])
req.Header.Set("Content-Encoding", string(SnappyBlockCompression))
req.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue)
appendable := &mockAppendable{commitErr: fmt.Errorf("commit error")}
handler := NewWriteHandler(log.NewNopLogger(), nil, appendable, []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV2})
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -367,7 +571,7 @@ func BenchmarkRemoteWriteOOOSamples(b *testing.B) {
require.NoError(b, db.Close())
})
// TODO: test with other proto format(s)
handler := NewWriteHandler(log.NewNopLogger(), nil, db.Head(), Version1)
handler := NewWriteHandler(log.NewNopLogger(), nil, db.Head(), []config.RemoteWriteProtoMsg{config.RemoteWriteProtoMsgV1})
buf, _, _, err := buildWriteRequest(nil, genSeriesWithSample(1000, 200*time.Minute.Milliseconds()), nil, nil, nil, nil, "snappy")
require.NoError(b, err)

View file

@ -15,6 +15,7 @@ package remote
import (
"bytes"
"errors"
"net/http"
"net/http/httptest"
"net/url"
@ -43,11 +44,12 @@ func testRemoteWriteConfig() *config.RemoteWriteConfig {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
}
func TestNoDuplicateWriteConfigs(t *testing.T) {
func TestWriteStorageApplyConfig_NoDuplicateWriteConfigs(t *testing.T) {
dir := t.TempDir()
cfg1 := config.RemoteWriteConfig{
@ -58,7 +60,8 @@ func TestNoDuplicateWriteConfigs(t *testing.T) {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
cfg2 := config.RemoteWriteConfig{
Name: "write-2",
@ -68,7 +71,8 @@ func TestNoDuplicateWriteConfigs(t *testing.T) {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
cfg3 := config.RemoteWriteConfig{
URL: &common_config.URL{
@ -77,62 +81,49 @@ func TestNoDuplicateWriteConfigs(t *testing.T) {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
type testcase struct {
cfgs []*config.RemoteWriteConfig
err bool
}
cases := []testcase{
for _, tc := range []struct {
cfgs []*config.RemoteWriteConfig
expectedErr error
}{
{ // Two duplicates, we should get an error.
cfgs: []*config.RemoteWriteConfig{
&cfg1,
&cfg1,
},
err: true,
cfgs: []*config.RemoteWriteConfig{&cfg1, &cfg1},
expectedErr: errors.New("duplicate remote write configs are not allowed, found duplicate for URL: http://localhost"),
},
{ // Duplicates but with different names, we should not get an error.
cfgs: []*config.RemoteWriteConfig{
&cfg1,
&cfg2,
},
err: false,
cfgs: []*config.RemoteWriteConfig{&cfg1, &cfg2},
},
{ // Duplicates but one with no name, we should not get an error.
cfgs: []*config.RemoteWriteConfig{
&cfg1,
&cfg3,
},
err: false,
cfgs: []*config.RemoteWriteConfig{&cfg1, &cfg3},
},
{ // Duplicates both with no name, we should get an error.
cfgs: []*config.RemoteWriteConfig{
&cfg3,
&cfg3,
},
err: true,
cfgs: []*config.RemoteWriteConfig{&cfg3, &cfg3},
expectedErr: errors.New("duplicate remote write configs are not allowed, found duplicate for URL: http://localhost"),
},
}
} {
t.Run("", func(t *testing.T) {
s := NewWriteStorage(nil, nil, dir, time.Millisecond, nil, false)
conf := &config.Config{
GlobalConfig: config.DefaultGlobalConfig,
RemoteWriteConfigs: tc.cfgs,
}
err := s.ApplyConfig(conf)
if tc.expectedErr == nil {
require.NoError(t, err)
} else {
require.Error(t, err)
require.Equal(t, tc.expectedErr, err)
}
for _, tc := range cases {
// todo: test with new format type(s)
s := NewWriteStorage(nil, nil, dir, time.Millisecond, nil, false)
conf := &config.Config{
GlobalConfig: config.DefaultGlobalConfig,
RemoteWriteConfigs: tc.cfgs,
}
err := s.ApplyConfig(conf)
gotError := err != nil
require.Equal(t, tc.err, gotError)
err = s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
})
}
}
func TestRestartOnNameChange(t *testing.T) {
func TestWriteStorageApplyConfig_RestartOnNameChange(t *testing.T) {
dir := t.TempDir()
cfg := testRemoteWriteConfig()
@ -140,14 +131,11 @@ func TestRestartOnNameChange(t *testing.T) {
hash, err := toHash(cfg)
require.NoError(t, err)
// todo: test with new format type(s)
s := NewWriteStorage(nil, nil, dir, time.Millisecond, nil, false)
conf := &config.Config{
GlobalConfig: config.DefaultGlobalConfig,
RemoteWriteConfigs: []*config.RemoteWriteConfig{
cfg,
},
GlobalConfig: config.DefaultGlobalConfig,
RemoteWriteConfigs: []*config.RemoteWriteConfig{cfg},
}
require.NoError(t, s.ApplyConfig(conf))
require.Equal(t, s.queues[hash].client().Name(), cfg.Name)
@ -159,14 +147,12 @@ func TestRestartOnNameChange(t *testing.T) {
require.NoError(t, err)
require.Equal(t, s.queues[hash].client().Name(), conf.RemoteWriteConfigs[0].Name)
err = s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestUpdateWithRegisterer(t *testing.T) {
func TestWriteStorageApplyConfig_UpdateWithRegisterer(t *testing.T) {
dir := t.TempDir()
// todo: test with new format type(s)
s := NewWriteStorage(nil, prometheus.NewRegistry(), dir, time.Millisecond, nil, false)
c1 := &config.RemoteWriteConfig{
Name: "named",
@ -176,7 +162,8 @@ func TestUpdateWithRegisterer(t *testing.T) {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
c2 := &config.RemoteWriteConfig{
URL: &common_config.URL{
@ -185,7 +172,8 @@ func TestUpdateWithRegisterer(t *testing.T) {
Host: "localhost",
},
},
QueueConfig: config.DefaultQueueConfig,
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
conf := &config.Config{
GlobalConfig: config.DefaultGlobalConfig,
@ -200,14 +188,12 @@ func TestUpdateWithRegisterer(t *testing.T) {
require.Equal(t, 10, queue.cfg.MaxShards)
}
err := s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestWriteStorageLifecycle(t *testing.T) {
func TestWriteStorageApplyConfig_Lifecycle(t *testing.T) {
dir := t.TempDir()
// todo: test with new format type(s)
s := NewWriteStorage(nil, nil, dir, defaultFlushDeadline, nil, false)
conf := &config.Config{
GlobalConfig: config.DefaultGlobalConfig,
@ -218,14 +204,12 @@ func TestWriteStorageLifecycle(t *testing.T) {
require.NoError(t, s.ApplyConfig(conf))
require.Len(t, s.queues, 1)
err := s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestUpdateExternalLabels(t *testing.T) {
func TestWriteStorageApplyConfig_UpdateExternalLabels(t *testing.T) {
dir := t.TempDir()
// todo: test with new format type(s)
s := NewWriteStorage(nil, prometheus.NewRegistry(), dir, time.Second, nil, false)
externalLabels := labels.FromStrings("external", "true")
@ -248,14 +232,12 @@ func TestUpdateExternalLabels(t *testing.T) {
require.Len(t, s.queues, 1)
require.Equal(t, []labels.Label{{Name: "external", Value: "true"}}, s.queues[hash].externalLabels)
err = s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestWriteStorageApplyConfigsIdempotent(t *testing.T) {
func TestWriteStorageApplyConfig_Idempotent(t *testing.T) {
dir := t.TempDir()
// todo: test with new format type(s)
s := NewWriteStorage(nil, nil, dir, defaultFlushDeadline, nil, false)
conf := &config.Config{
GlobalConfig: config.GlobalConfig{},
@ -274,14 +256,12 @@ func TestWriteStorageApplyConfigsIdempotent(t *testing.T) {
_, hashExists := s.queues[hash]
require.True(t, hashExists, "Queue pointer should have remained the same")
err = s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestWriteStorageApplyConfigsPartialUpdate(t *testing.T) {
func TestWriteStorageApplyConfig_PartialUpdate(t *testing.T) {
dir := t.TempDir()
// todo: test with new format type(s)
s := NewWriteStorage(nil, nil, dir, defaultFlushDeadline, nil, false)
c0 := &config.RemoteWriteConfig{
@ -292,6 +272,7 @@ func TestWriteStorageApplyConfigsPartialUpdate(t *testing.T) {
Regex: relabel.MustNewRegexp(".+"),
},
},
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
c1 := &config.RemoteWriteConfig{
RemoteTimeout: model.Duration(20 * time.Second),
@ -299,10 +280,12 @@ func TestWriteStorageApplyConfigsPartialUpdate(t *testing.T) {
HTTPClientConfig: common_config.HTTPClientConfig{
BearerToken: "foo",
},
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
c2 := &config.RemoteWriteConfig{
RemoteTimeout: model.Duration(30 * time.Second),
QueueConfig: config.DefaultQueueConfig,
RemoteTimeout: model.Duration(30 * time.Second),
QueueConfig: config.DefaultQueueConfig,
ProtobufMessage: config.RemoteWriteProtoMsgV1,
}
conf := &config.Config{
@ -382,8 +365,7 @@ func TestWriteStorageApplyConfigsPartialUpdate(t *testing.T) {
_, hashExists = s.queues[hashes[2]]
require.True(t, hashExists, "Pointer of unchanged queue should have remained the same")
err = s.Close()
require.NoError(t, err)
require.NoError(t, s.Close())
}
func TestOTLPWriteHandler(t *testing.T) {

View file

@ -210,10 +210,9 @@ type API struct {
isAgent bool
statsRenderer StatsRenderer
remoteWriteHeadHandler http.Handler
remoteWriteHandler http.Handler
remoteReadHandler http.Handler
otlpWriteHandler http.Handler
remoteWriteHandler http.Handler
remoteReadHandler http.Handler
otlpWriteHandler http.Handler
codecs []Codec
}
@ -290,10 +289,7 @@ func NewAPI(
}
if rwEnabled {
// TODO(bwplotka): Use acceptRemoteWriteProtoMsgs in the next PR (split PR for review readability).
// and remove all head/negotiation.
a.remoteWriteHandler = remote.NewWriteHandler(logger, registerer, ap, 1)
a.remoteWriteHeadHandler = remote.NewWriteHeadHandler(logger, registerer, 1)
a.remoteWriteHandler = remote.NewWriteHandler(logger, registerer, ap, acceptRemoteWriteProtoMsgs)
}
if otlpEnabled {
a.otlpWriteHandler = remote.NewOTLPWriteHandler(logger, ap)
@ -390,7 +386,6 @@ func (api *API) Register(r *route.Router) {
r.Get("/status/walreplay", api.serveWALReplayStatus)
r.Post("/read", api.ready(api.remoteRead))
r.Post("/write", api.ready(api.remoteWrite))
r.Head("/write", api.remoteWriteHead)
r.Post("/otlp/v1/metrics", api.ready(api.otlpWrite))
r.Get("/alerts", wrapAgent(api.alerts))
@ -1659,14 +1654,6 @@ func (api *API) remoteWrite(w http.ResponseWriter, r *http.Request) {
}
}
func (api *API) remoteWriteHead(w http.ResponseWriter, r *http.Request) {
if api.remoteWriteHeadHandler != nil {
api.remoteWriteHeadHandler.ServeHTTP(w, r)
} else {
http.Error(w, "remote write receiver needs to be enabled with --web.enable-remote-write-receiver", http.StatusNotFound)
}
}
func (api *API) otlpWrite(w http.ResponseWriter, r *http.Request) {
if api.otlpWriteHandler != nil {
api.otlpWriteHandler.ServeHTTP(w, r)

View file

@ -337,48 +337,6 @@ var sampleFlagMap = map[string]string{
"flag2": "value2",
}
func TestHeadEndpoint(t *testing.T) {
for _, tc := range []struct {
name string
rwFormat config.RemoteWriteFormat
expectedStatusCode int
expectedHeaderValue string
}{
{
name: "HEAD Version 1",
rwFormat: remote.Version1,
expectedStatusCode: http.StatusOK,
expectedHeaderValue: "0.1.0",
},
{
name: "HEAD Version 2",
rwFormat: remote.Version2,
expectedStatusCode: http.StatusOK,
expectedHeaderValue: "2.0;snappy,0.1.0",
},
} {
r := route.New()
api := &API{
remoteWriteHeadHandler: remote.NewWriteHeadHandler(log.NewNopLogger(), nil, tc.rwFormat),
ready: func(f http.HandlerFunc) http.HandlerFunc { return f },
}
api.Register(r)
s := httptest.NewServer(r)
defer s.Close()
req, err := http.NewRequest(http.MethodHead, s.URL+"/write", nil)
require.NoError(t, err, "Error creating HEAD request")
client := &http.Client{}
resp, err := client.Do(req)
require.NoError(t, err, "Error executing HEAD request")
require.Equal(t, tc.expectedStatusCode, resp.StatusCode)
promHeader := resp.Header.Get(remote.RemoteWriteVersionHeader)
require.Equal(t, tc.expectedHeaderValue, promHeader)
}
}
func TestEndpoints(t *testing.T) {
storage := promql.LoadedStorage(t, `
load 1m