Merge branch 'public-single-node' into pmm-6401-read-prometheus-data-files

This commit is contained in:
Aliaksandr Valialkin 2023-02-20 20:00:43 -08:00
commit ff990ab0c5
No known key found for this signature in database
GPG key ID: A72BEC6CD3D0DED1
19 changed files with 281 additions and 132 deletions

View file

@ -997,7 +997,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
Optional `max_rows_per_line` arg may be added to the request for limiting the maximum number of rows exported per each JSON line.
@ -1046,7 +1046,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported CSV data can be imported to VictoriaMetrics via [/api/v1/import/csv](#how-to-import-csv-data).
@ -1074,7 +1074,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported data can be imported to VictoriaMetrics via [/api/v1/import/native](#how-to-import-data-in-native-format).
@ -1313,7 +1313,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
By default, the last point on the interval `[now - max_lookback ... now]` is scraped for each time series. The default value for `max_lookback` is `5m` (5 minutes), but it can be overridden with `max_lookback` query arg.
For instance, `/federate?match[]=up&max_lookback=1h` would return last points on the `[now - 1h ... now]` interval. This may be useful for time series federation

View file

@ -25,7 +25,9 @@ additionally to [discovering Prometheus-compatible targets and scraping metrics
* Can add, remove and modify labels (aka tags) via Prometheus relabeling. Can filter data before sending it to remote storage. See [these docs](#relabeling) for details.
* Can accept data via all the ingestion protocols supported by VictoriaMetrics - see [these docs](#how-to-push-data-to-vmagent).
* Can aggregate incoming samples by time and by labels before sending them to remote storage - see [these docs](https://docs.victoriametrics.com/stream-aggregation.html).
* Can replicate collected metrics simultaneously to multiple remote storage systems - see [these docs](#replication-and-high-availability).
* Can replicate collected metrics simultaneously to multiple Prometheus-compatible remote storage systems - see [these docs](#replication-and-high-availability).
* Can save egress network bandwidth usage costs when [VictoriaMetrics remote write protocol](#victoriametrics-remote-write-protocol)
is used for sending the data to VictoriaMetrics.
* Works smoothly in environments with unstable connections to remote storage. If the remote storage is unavailable, the collected metrics
are buffered at `-remoteWrite.tmpDataPath`. The buffered metrics are sent to remote storage as soon as the connection
to the remote storage is repaired. The maximum disk usage for the buffer can be limited with `-remoteWrite.maxDiskUsagePerURL`.
@ -45,14 +47,15 @@ additionally to [discovering Prometheus-compatible targets and scraping metrics
Please download `vmutils-*` archive from [releases page](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) (
`vmagent` is also available in [docker images](https://hub.docker.com/r/victoriametrics/vmagent/tags)),
unpack it and pass the following flags to the `vmagent` binary in order to start scraping Prometheus-compatible targets:
unpack it and pass the following flags to the `vmagent` binary in order to start scraping Prometheus-compatible targets
and sending the data to the Prometheus-compatible remote storage:
* `-promscrape.config` with the path to Prometheus config file (usually located at `/etc/prometheus/prometheus.yml`).
* `-promscrape.config` with the path to [Prometheus config file](https://docs.victoriametrics.com/sd_configs.html) (usually located at `/etc/prometheus/prometheus.yml`).
The path can point either to local file or to http url. `vmagent` doesn't support some sections of Prometheus config file,
so you may need either to delete these sections or to run `vmagent` with `-promscrape.config.strictParse=false` command-line flag.
In this case `vmagent` ignores unsupported sections. See [the list of unsupported sections](#unsupported-prometheus-config-sections).
* `-remoteWrite.url` with the remote storage endpoint such as VictoriaMetrics, the `-remoteWrite.url` argument can be specified
multiple times to replicate data concurrently to an arbitrary number of remote storage systems. See [various use cases](#use-cases).
* `-remoteWrite.url` with Prometheus-compatible remote storage endpoint such as VictoriaMetrics, the `-remoteWrite.url` argument can be specified
multiple times to replicate data concurrently to multiple remote storage systems. See [various use cases](#use-cases).
Example command line:
@ -60,8 +63,6 @@ Example command line:
/path/to/vmagent -promscrape.config=/path/to/prometheus.yml -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
Example of scrape configuration for `-promscrape.config` argument you can find [here](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/deployment/docker/prometheus.yml)
See [how to scrape Prometheus-compatible targets](#how-to-collect-metrics-in-prometheus-format) for more details.
If you use single-node VictoriaMetrics, then you can discover and scrape Prometheus-compatible targets directly from VictoriaMetrics
@ -75,6 +76,8 @@ and sending it to the provided `-remoteWrite.url`:
/path/to/vmagent -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
`vmagent` can save network bandwidth usage costs under high load when [VictoriaMetrics remote write protocol is enabled](#victoriametrics-remote-write-protocol).
See [troubleshooting docs](#troubleshooting) if you encounter common issues with `vmagent`.
Pass `-help` to `vmagent` in order to see [the full list of supported command-line flags with their descriptions](#advanced-usage).
@ -120,7 +123,8 @@ data to the remote storage. It re-tries sending the data to remote storage until
The maximum on-disk size for the buffered metrics can be limited with `-remoteWrite.maxDiskUsagePerURL`.
`vmagent` works on various architectures from the IoT world - 32-bit arm, 64-bit arm, ppc64, 386, amd64.
See [the corresponding Makefile rules](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/app/vmagent/Makefile) for details.
The `vmagent` can save network bandwidth usage costs by using [VictoriaMetrics remote write protocol](#victoriametrics-remote-write-protocol).
### Drop-in replacement for Prometheus
@ -175,6 +179,33 @@ the `-remoteWrite.url` command-line flag should be configured as `<schema>://<vm
according to [these docs](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format).
There is also support for multitenant writes. See [these docs](#multitenancy).
## VictoriaMetrics remote write protocol
By default `vmagent` uses Prometheus remote_write protocol for sending the data to the configured `-remoteWrite.url`.
This allows sending data to [any Prometheus-compatible remote storage](https://prometheus.io/docs/operating/integrations/#remote-endpoints-and-storage).
The Prometheus remote_write protocol may require big amounts of network bandwidth under high load.
This may result in high network egress costs when the configured remote storage is located in remote datacenter or availability zone.
This also may result in the increased disk IO at `vmagent` when it writes to disk the pending data, which must be sent to remote storage.
In this case the `vmagent` can be instructed to use VictoriaMetrics remote write protocol.
This allows reducing egress network bandwidth costs while reducing disk read/write IO at `vmagent` side under high load.
The `-remoteWrite.useVMProto=true` command-line flag instructs `vmagent` to send the data to the corresponding `-remoteWrite.url`
via VictoriaMetrics remote write protocol.
While all the [recently released](https://docs.victoriametrics.com/CHANGELOG.html) VictoriaMetrics components support
the VictoriaMetrics remote write protocol, third-party systems and old versions of VictoriaMetrics components may miss the support of this protocol.
The `-remoteWrite.useVMProto` command-line flag can be set independently per each configured `-remoteWrite.url`.
For example, the following command instructs `vmagent` to send the data to `https://victoriametrics/api/v1/write` via VictoriaMetrics remote write protocol,
while sending the data to `https://prom-compatible-storage/write` via Prometheus remote write protocol:
```
./vmagent -remoteWrite.url=https://victoriametrics/api/v1/write \
-remoteWrite.useVMProto=true \
-remoteWrite.url=https://prom-compatible-storage/write \
-remoteWrite.useVMProto=false
```
## Multitenancy
By default `vmagent` collects the data without tenant identifiers and routes it to the configured `-remoteWrite.url`.
@ -880,7 +911,7 @@ If you have suggestions for improvements or have found a bug - please open an is
The number of dropped blocks can be monitored via `vmagent_remotewrite_packets_dropped_total` metric exported at [/metrics page](#monitoring).
* Use `-remoteWrite.queues=1` when `-remoteWrite.url` points to remote storage, which doesn't accept out-of-order samples (aka data backfilling).
Such storage systems include Prometheus, Cortex and Thanos, which typically emit `out of order sample` errors.
Such storage systems include Prometheus, Mimir, Cortex and Thanos, which typically emit `out of order sample` errors.
The best solution is to use remote storage with [backfilling support](https://docs.victoriametrics.com/#backfilling) such as VictoriaMetrics.
* `vmagent` buffers scraped data at the `-remoteWrite.tmpDataPath` directory until it is sent to `-remoteWrite.url`.
@ -1212,7 +1243,7 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
-insert.maxQueueDuration duration
The maximum duration to wait in the queue when -maxConcurrentInserts concurrent insert requests are executed (default 1m0s)
-internStringMaxLen int
The maximum length for strings to intern. Lower limit may save memory at the cost of higher CPU usage. See https://en.wikipedia.org/wiki/String_interning (default 300)
The maximum length for strings to intern. Lower limit may save memory at the cost of higher CPU usage. See https://en.wikipedia.org/wiki/String_interning (default 500)
-kafka.consumer.topic array
Kafka topic names for data consumption. This flag is available only in VictoriaMetrics enterprise. See https://docs.victoriametrics.com/enterprise.html
Supports an array of values separated by comma or specified via multiple flags.
@ -1505,11 +1536,14 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
-remoteWrite.tmpDataPath string
Path to directory where temporary data for remote write component is stored. See also -remoteWrite.maxDiskUsagePerURL (default "vmagent-remotewrite-data")
-remoteWrite.url array
Remote storage URL to write data to. It must support Prometheus remote_write API. It is recommended using VictoriaMetrics as remote storage. Example url: http://<victoriametrics-host>:8428/api/v1/write . Pass multiple -remoteWrite.url flags in order to replicate data to multiple remote storage systems. See also -remoteWrite.multitenantURL
Remote storage URL to write data to. It must support Prometheus remote_write protocol. Example url: http://<victoriametrics-host>:8428/api/v1/write . It is recommended setting -remoteWrite.useVMProto command-line option when VictoriaMetrics is used as a remote storage in order to save network bandwidth. See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol . Pass multiple -remoteWrite.url options in order to replicate the collected data to multiple remote storage systems. See also -remoteWrite.multitenantURL
Supports an array of values separated by comma or specified via multiple flags.
-remoteWrite.urlRelabelConfig array
Optional path to relabel configs for the corresponding -remoteWrite.url. See also -remoteWrite.relabelConfig. The path can point either to local file or to http url. See https://docs.victoriametrics.com/vmagent.html#relabeling
Supports an array of values separated by comma or specified via multiple flags.
-remoteWrite.useVMProto array
Whether to use VictoriaMetrics protocol for sending the data to the given -remoteWrite.url in order to reduce network bandwidth usage and disk read/write IO under high load. See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol
Supports array of values separated by comma or specified via multiple flags.
-sortLabels
Whether to sort labels for incoming samples before writing them to all the configured remote storage systems. This may be needed for reducing memory usage at remote storage when the order of labels in incoming samples is random. For example, if m{k1="v1",k2="v2"} may be sent as m{k2="v2",k1="v1"}Enabled sorting for labels can slow down ingestion performance a bit
-tls

View file

@ -27,7 +27,8 @@ func InsertHandler(at *auth.Token, req *http.Request) error {
if err != nil {
return err
}
return stream.Parse(req.Body, func(tss []prompb.TimeSeries) error {
isVMRemoteWrite := req.Header.Get("Content-Encoding") == "zstd"
return stream.Parse(req.Body, isVMRemoteWrite, func(tss []prompb.TimeSeries) error {
return insertRows(at, tss, extraLabels)
})
}

View file

@ -67,10 +67,11 @@ var (
)
type client struct {
sanitizedURL string
remoteWriteURL string
fq *persistentqueue.FastQueue
hc *http.Client
sanitizedURL string
remoteWriteURL string
isVMRemoteWrite bool
fq *persistentqueue.FastQueue
hc *http.Client
sendBlock func(block []byte) bool
authCfg *promauth.Config
@ -92,7 +93,7 @@ type client struct {
stopCh chan struct{}
}
func newHTTPClient(argIdx int, remoteWriteURL, sanitizedURL string, fq *persistentqueue.FastQueue, concurrency int) *client {
func newHTTPClient(argIdx int, remoteWriteURL, sanitizedURL string, fq *persistentqueue.FastQueue, concurrency int, isVMRemoteWrite bool) *client {
authCfg, err := getAuthConfig(argIdx)
if err != nil {
logger.Panicf("FATAL: cannot initialize auth config for remoteWrite.url=%q: %s", remoteWriteURL, err)
@ -123,11 +124,12 @@ func newHTTPClient(argIdx int, remoteWriteURL, sanitizedURL string, fq *persiste
tr.Proxy = http.ProxyURL(pu)
}
c := &client{
sanitizedURL: sanitizedURL,
remoteWriteURL: remoteWriteURL,
authCfg: authCfg,
awsCfg: awsCfg,
fq: fq,
sanitizedURL: sanitizedURL,
remoteWriteURL: remoteWriteURL,
isVMRemoteWrite: isVMRemoteWrite,
authCfg: authCfg,
awsCfg: awsCfg,
fq: fq,
hc: &http.Client{
Transport: tr,
Timeout: sendTimeout.GetOptionalArgOrDefault(argIdx, time.Minute),
@ -291,7 +293,9 @@ func (c *client) runWorker() {
}
}
// sendBlockHTTP returns false only if c.stopCh is closed.
// sendBlockHTTP sends the given block to c.remoteWriteURL.
//
// The function returns false only if c.stopCh is closed.
// Otherwise it tries sending the block to remote storage indefinitely.
func (c *client) sendBlockHTTP(block []byte) bool {
c.rl.register(len(block), c.stopCh)
@ -313,8 +317,13 @@ again:
h := req.Header
h.Set("User-Agent", "vmagent")
h.Set("Content-Type", "application/x-protobuf")
h.Set("Content-Encoding", "snappy")
h.Set("X-Prometheus-Remote-Write-Version", "0.1.0")
if c.isVMRemoteWrite {
h.Set("Content-Encoding", "zstd")
h.Set("X-VictoriaMetrics-Remote-Write-Version", "1")
} else {
h.Set("Content-Encoding", "snappy")
h.Set("X-Prometheus-Remote-Write-Version", "0.1.0")
}
if c.awsCfg != nil {
if err := c.awsCfg.SignRequest(req, sigv4Hash); err != nil {
// there is no need in retry, request will be rejected by client.Do and retried by code below

View file

@ -8,6 +8,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/decimal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding/zstd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
@ -33,9 +34,10 @@ type pendingSeries struct {
periodicFlusherWG sync.WaitGroup
}
func newPendingSeries(pushBlock func(block []byte), significantFigures, roundDigits int) *pendingSeries {
func newPendingSeries(pushBlock func(block []byte), isVMRemoteWrite bool, significantFigures, roundDigits int) *pendingSeries {
var ps pendingSeries
ps.wr.pushBlock = pushBlock
ps.wr.isVMRemoteWrite = isVMRemoteWrite
ps.wr.significantFigures = significantFigures
ps.wr.roundDigits = roundDigits
ps.stopCh = make(chan struct{})
@ -88,6 +90,9 @@ type writeRequest struct {
// pushBlock is called when whe write request is ready to be sent.
pushBlock func(block []byte)
// Whether to encode the write request with VictoriaMetrics remote write protocol.
isVMRemoteWrite bool
// How many significant figures must be left before sending the writeRequest to pushBlock.
significantFigures int
@ -104,7 +109,7 @@ type writeRequest struct {
}
func (wr *writeRequest) reset() {
// Do not reset pushBlock, significantFigures and roundDigits, since they are re-used.
// Do not reset lastFlushTime, pushBlock, isVMRemoteWrite, significantFigures and roundDigits, since they are re-used.
wr.wr.Timeseries = nil
@ -126,7 +131,7 @@ func (wr *writeRequest) flush() {
wr.wr.Timeseries = wr.tss
wr.adjustSampleValues()
atomic.StoreUint64(&wr.lastFlushTime, fasttime.UnixTimestamp())
pushWriteRequest(&wr.wr, wr.pushBlock)
pushWriteRequest(&wr.wr, wr.pushBlock, wr.isVMRemoteWrite)
wr.reset()
}
@ -188,7 +193,7 @@ func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) {
wr.buf = buf
}
func pushWriteRequest(wr *prompbmarshal.WriteRequest, pushBlock func(block []byte)) {
func pushWriteRequest(wr *prompbmarshal.WriteRequest, pushBlock func(block []byte), isVMRemoteWrite bool) {
if len(wr.Timeseries) == 0 {
// Nothing to push
return
@ -197,7 +202,11 @@ func pushWriteRequest(wr *prompbmarshal.WriteRequest, pushBlock func(block []byt
bb.B = prompbmarshal.MarshalWriteRequest(bb.B[:0], wr)
if len(bb.B) <= maxUnpackedBlockSize.IntN() {
zb := snappyBufPool.Get()
zb.B = snappy.Encode(zb.B[:cap(zb.B)], bb.B)
if isVMRemoteWrite {
zb.B = zstd.CompressLevel(zb.B[:0], bb.B, 0)
} else {
zb.B = snappy.Encode(zb.B[:cap(zb.B)], bb.B)
}
writeRequestBufPool.Put(bb)
if len(zb.B) <= persistentqueue.MaxBlockSize {
pushBlock(zb.B)
@ -221,18 +230,18 @@ func pushWriteRequest(wr *prompbmarshal.WriteRequest, pushBlock func(block []byt
}
n := len(samples) / 2
wr.Timeseries[0].Samples = samples[:n]
pushWriteRequest(wr, pushBlock)
pushWriteRequest(wr, pushBlock, isVMRemoteWrite)
wr.Timeseries[0].Samples = samples[n:]
pushWriteRequest(wr, pushBlock)
pushWriteRequest(wr, pushBlock, isVMRemoteWrite)
wr.Timeseries[0].Samples = samples
return
}
timeseries := wr.Timeseries
n := len(timeseries) / 2
wr.Timeseries = timeseries[:n]
pushWriteRequest(wr, pushBlock)
pushWriteRequest(wr, pushBlock, isVMRemoteWrite)
wr.Timeseries = timeseries[n:]
pushWriteRequest(wr, pushBlock)
pushWriteRequest(wr, pushBlock, isVMRemoteWrite)
wr.Timeseries = timeseries
}

View file

@ -5,37 +5,44 @@ import (
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/golang/snappy"
)
func TestPushWriteRequest(t *testing.T) {
for _, rowsCount := range []int{1, 10, 100, 1e3, 1e4} {
rowsCounts := []int{1, 10, 100, 1e3, 1e4}
expectedBlockLensProm := []int{216, 1848, 16424, 169882, 1757876}
expectedBlockLensVM := []int{138, 492, 3927, 34995, 288476}
for i, rowsCount := range rowsCounts {
expectedBlockLenProm := expectedBlockLensProm[i]
expectedBlockLenVM := expectedBlockLensVM[i]
t.Run(fmt.Sprintf("%d", rowsCount), func(t *testing.T) {
testPushWriteRequest(t, rowsCount)
testPushWriteRequest(t, rowsCount, expectedBlockLenProm, expectedBlockLenVM)
})
}
}
func testPushWriteRequest(t *testing.T, rowsCount int) {
wr := newTestWriteRequest(rowsCount, 10)
pushBlockLen := 0
pushBlock := func(block []byte) {
if pushBlockLen > 0 {
panic(fmt.Errorf("BUG: pushBlock called multiple times; pushBlockLen=%d at first call, len(block)=%d at second call", pushBlockLen, len(block)))
func testPushWriteRequest(t *testing.T, rowsCount, expectedBlockLenProm, expectedBlockLenVM int) {
f := func(isVMRemoteWrite bool, expectedBlockLen int) {
t.Helper()
wr := newTestWriteRequest(rowsCount, 20)
pushBlockLen := 0
pushBlock := func(block []byte) {
if pushBlockLen > 0 {
panic(fmt.Errorf("BUG: pushBlock called multiple times; pushBlockLen=%d at first call, len(block)=%d at second call", pushBlockLen, len(block)))
}
pushBlockLen = len(block)
}
pushWriteRequest(wr, pushBlock, isVMRemoteWrite)
if pushBlockLen != expectedBlockLen {
t.Fatalf("unexpected block len for rowsCount=%d, isVMRemoteWrite=%v; got %d bytes; expecting %d bytes",
rowsCount, isVMRemoteWrite, pushBlockLen, expectedBlockLen)
}
pushBlockLen = len(block)
}
pushWriteRequest(wr, pushBlock)
b := prompbmarshal.MarshalWriteRequest(nil, wr)
zb := snappy.Encode(nil, b)
maxPushBlockLen := len(zb)
minPushBlockLen := maxPushBlockLen / 2
if pushBlockLen < minPushBlockLen {
t.Fatalf("unexpected block len after pushWriteRequest; got %d bytes; must be at least %d bytes", pushBlockLen, minPushBlockLen)
}
if pushBlockLen > maxPushBlockLen {
t.Fatalf("unexpected block len after pushWriteRequest; got %d bytes; must be smaller or equal to %d bytes", pushBlockLen, maxPushBlockLen)
}
// Check Prometheus remote write
f(false, expectedBlockLenProm)
// Check VictoriaMetrics remote write
f(true, expectedBlockLenVM)
}
func newTestWriteRequest(seriesCount, labelsCount int) *prompbmarshal.WriteRequest {

View file

@ -28,12 +28,17 @@ import (
)
var (
remoteWriteURLs = flagutil.NewArrayString("remoteWrite.url", "Remote storage URL to write data to. It must support Prometheus remote_write API. "+
"It is recommended using VictoriaMetrics as remote storage. Example url: http://<victoriametrics-host>:8428/api/v1/write . "+
"Pass multiple -remoteWrite.url flags in order to replicate data to multiple remote storage systems. See also -remoteWrite.multitenantURL")
remoteWriteURLs = flagutil.NewArrayString("remoteWrite.url", "Remote storage URL to write data to. It must support Prometheus remote_write protocol. "+
"Example url: http://<victoriametrics-host>:8428/api/v1/write . "+
"It is recommended setting -remoteWrite.useVMProto command-line option when VictoriaMetrics is used as a remote storage in order to save network bandwidth. "+
"See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol . "+
"Pass multiple -remoteWrite.url options in order to replicate the collected data to multiple remote storage systems. See also -remoteWrite.multitenantURL")
remoteWriteMultitenantURLs = flagutil.NewArrayString("remoteWrite.multitenantURL", "Base path for multitenant remote storage URL to write data to. "+
"See https://docs.victoriametrics.com/vmagent.html#multitenancy for details. Example url: http://<vminsert>:8480 . "+
"Pass multiple -remoteWrite.multitenantURL flags in order to replicate data to multiple remote storage systems. See also -remoteWrite.url")
useVMProto = flagutil.NewArrayBool("remoteWrite.useVMProto", "Whether to use VictoriaMetrics protocol for sending the data to the given -remoteWrite.url "+
"in order to reduce network bandwidth usage and disk read/write IO under high load. "+
"See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol")
tmpDataPath = flag.String("remoteWrite.tmpDataPath", "vmagent-remotewrite-data", "Path to directory where temporary data for remote write component is stored. "+
"See also -remoteWrite.maxDiskUsagePerURL")
queues = flag.Int("remoteWrite.queues", cgroup.AvailableCPUs()*2, "The number of concurrent queues to each -remoteWrite.url. Set more queues if default number of queues "+
@ -475,10 +480,11 @@ func newRemoteWriteCtx(argIdx int, at *auth.Token, remoteWriteURL *url.URL, maxI
_ = metrics.GetOrCreateGauge(fmt.Sprintf(`vmagent_remotewrite_pending_inmemory_blocks{path=%q, url=%q}`, queuePath, sanitizedURL), func() float64 {
return float64(fq.GetInmemoryQueueLen())
})
isVMRemoteWrite := useVMProto.GetOptionalArg(argIdx)
var c *client
switch remoteWriteURL.Scheme {
case "http", "https":
c = newHTTPClient(argIdx, remoteWriteURL.String(), sanitizedURL, fq, *queues)
c = newHTTPClient(argIdx, remoteWriteURL.String(), sanitizedURL, fq, *queues, isVMRemoteWrite)
default:
logger.Fatalf("unsupported scheme: %s for remoteWriteURL: %s, want `http`, `https`", remoteWriteURL.Scheme, sanitizedURL)
}
@ -495,7 +501,7 @@ func newRemoteWriteCtx(argIdx int, at *auth.Token, remoteWriteURL *url.URL, maxI
}
pss := make([]*pendingSeries, pssLen)
for i := range pss {
pss[i] = newPendingSeries(fq.MustWriteBlock, sf, rd)
pss[i] = newPendingSeries(fq.MustWriteBlock, isVMRemoteWrite, sf, rd)
}
rwctx := &remoteWriteCtx{

View file

@ -28,7 +28,8 @@ Use this feature for the following cases:
* Graphite datasource can be used for alerting and recording rules. See [these docs](#graphite);
* Recording and Alerting rules backfilling (aka `replay`). See [these docs](#rules-backfilling);
* Lightweight and without extra dependencies.
* Supports [reusable templates](#reusable-templates) for annotations.
* Supports [reusable templates](#reusable-templates) for annotations;
* Load of recording and alerting rules from local filesystem, GCS and S3.
## Limitations
@ -99,26 +100,6 @@ groups:
[ - <rule_group> ]
```
### Reading rules from object storage
[Enterprise version](https://docs.victoriametrics.com/enterprise.html) of `vmalert` may read alerting and recording rules
from object storage:
- `./bin/vmalert -rule=s3://bucket/dir/alert.rules` would read rules from the given path at S3 bucket
- `./bin/vmalert -rule=gs://bucket/bir/alert.rules` would read rules from the given path at GCS bucket
S3 and GCS paths support only matching by prefix, e.g. `s3://bucket/dir/rule_` matches
all files with prefix `rule_` in the folder `dir`.
The following [command-line flags](#flags) can be used for fine-tuning access to S3 and GCS:
- `-s3.credsFilePath` - path to file with GCS or S3 credentials. Credentials are loaded from default locations if not set.
- `-s3.configFilePath` - path to file with S3 configs. Configs are loaded from default location if not set.
- `-s3.configProfile` - profile name for S3 configs. If no set, the value of the environment variable will be loaded (`AWS_PROFILE` or `AWS_DEFAULT_PROFILE`).
- `-s3.customEndpoint` - custom S3 endpoint for use with S3-compatible storages (e.g. MinIO). S3 is used if not set.
- `-s3.forcePathStyle` - prefixing endpoint with bucket name when set false, true by default.
### Groups
Each group has the following attributes:
@ -424,6 +405,25 @@ The enterprise version of vmalert is available in `vmutils-*-enterprise.tar.gz`
at [release page](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) and in `*-enterprise`
tags at [Docker Hub](https://hub.docker.com/r/victoriametrics/vmalert/tags).
### Reading rules from object storage
[Enterprise version](https://docs.victoriametrics.com/enterprise.html) of `vmalert` may read alerting and recording rules
from object storage:
- `./bin/vmalert -rule=s3://bucket/dir/alert.rules` would read rules from the given path at S3 bucket
- `./bin/vmalert -rule=gs://bucket/bir/alert.rules` would read rules from the given path at GCS bucket
S3 and GCS paths support only matching by prefix, e.g. `s3://bucket/dir/rule_` matches
all files with prefix `rule_` in the folder `dir`.
The following [command-line flags](#flags) can be used for fine-tuning access to S3 and GCS:
- `-s3.credsFilePath` - path to file with GCS or S3 credentials. Credentials are loaded from default locations if not set.
- `-s3.configFilePath` - path to file with S3 configs. Configs are loaded from default location if not set.
- `-s3.configProfile` - profile name for S3 configs. If no set, the value of the environment variable will be loaded (`AWS_PROFILE` or `AWS_DEFAULT_PROFILE`).
- `-s3.customEndpoint` - custom S3 endpoint for use with S3-compatible storages (e.g. MinIO). S3 is used if not set.
- `-s3.forcePathStyle` - prefixing endpoint with bucket name when set false, true by default.
### Topology examples
The following sections are showing how `vmalert` may be used and configured

View file

@ -206,6 +206,22 @@ mwIDAQAB
```
This command will result in 3 keys loaded: 2 keys from files and 1 from command line.
### Using JWKS endpoint for JWT signature verification
`vmgateway` supports using JWKS endpoint for JWT signature verification.
In order to enable JWKS endpoint for JWT signature verification, you need to specify JWKS endpoint URL by using `auth.jwksEndpoints` flag.
When `auth.jwksEndpoints` is specified `vmageteway` will fetch public keys from the specified endpoint and use them for JWT signature verification.
Example usage for tokens issues by Azure Active Directory:
```console
/bin/vmgateway -eula \
-enable.auth \
-write.url=http://localhost:8480 \
-read.url=http://localhost:8481 \
-auth.jwksEndpoints=https://login.microsoftonline.com/common/discovery/v2.0/keys
```
## Configuration
The shortlist of configuration flags include the following:
@ -213,6 +229,9 @@ The shortlist of configuration flags include the following:
```console
-auth.httpHeader string
HTTP header name to look for JWT authorization token (default "Authorization")
-auth.jwksEndpoints array
JWKS endpoints to fetch keys for JWT tokens signature verification
Supports an array of values separated by comma or specified via multiple flags.
-auth.publicKeyFiles array
Path file with public key to verify JWT token signature
Supports an array of values separated by comma or specified via multiple flags.

View file

@ -23,7 +23,8 @@ func InsertHandler(req *http.Request) error {
if err != nil {
return err
}
return stream.Parse(req.Body, func(tss []prompb.TimeSeries) error {
isVMRemoteWrite := req.Header.Get("Content-Encoding") == "zstd"
return stream.Parse(req.Body, isVMRemoteWrite, func(tss []prompb.TimeSeries) error {
return insertRows(tss, extraLabels)
})
}

View file

@ -18,6 +18,8 @@ The following tip changes can be tested by building VictoriaMetrics components f
* SECURITY: upgrade base docker image (alpine) from 3.17.1 to 3.17.2. See [alpine 3.17.2 release notes](https://alpinelinux.org/posts/Alpine-3.17.2-released.html).
* SECURITY: upgrade Go builder from Go1.20.0 to Go1.20.1. See [the list of issues addressed in Go1.20.1](https://github.com/golang/go/issues?q=milestone%3AGo1.20.1+label%3ACherryPickApproved).
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add support for [VictoriaMetrics remote write protocol](https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol). This protocol allows saving egress network bandwidth costs when sending data from `vmagent` to VictoriaMetrics located in another datacenter or availability zone. This also allows reducing disk IO under high load when `vmagent` starts queuing the collected data to disk when the remote storage is temporarily unavailable or cannot keep up with the data ingestion rate. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1225).
* FEATURE: [vmgateway](https://docs.victoriametrics.com/vmgateway.html): add the ability to verify JWT signature via [JWKS endpoint](https://auth0.com/docs/secure/tokens/json-web-tokens/json-web-key-sets). See [these docs](https://docs.victoriametrics.com/vmgateway.html#using-jwks-endpoint-for-jwt-signature-verification).
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth.html): add the ability to limit the number of concurrent requests on a per-user basis via `-maxConcurrentPerUserRequests` command-line flag and via `max_concurrent_requests` config option. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3346) and [these docs](https://docs.victoriametrics.com/vmauth.html#concurrency-limiting).
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth.html): automatically retry failing `GET` requests on all [the configured backends](https://docs.victoriametrics.com/vmauth.html#load-balancing). Previously the backend error has been immediately returned to the client without retrying the request on the remaining backends.
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth.html): choose the backend with the minimum number of concurrently executed requests [among the configured backends](https://docs.victoriametrics.com/vmauth.html#load-balancing) in a round-robin manner for serving the incoming requests. This allows spreading the load among backends more evenly, while improving the response time.

View file

@ -998,7 +998,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
Optional `max_rows_per_line` arg may be added to the request for limiting the maximum number of rows exported per each JSON line.
@ -1047,7 +1047,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported CSV data can be imported to VictoriaMetrics via [/api/v1/import/csv](#how-to-import-csv-data).
@ -1075,7 +1075,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported data can be imported to VictoriaMetrics via [/api/v1/import/native](#how-to-import-data-in-native-format).
@ -1314,7 +1314,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
By default, the last point on the interval `[now - max_lookback ... now]` is scraped for each time series. The default value for `max_lookback` is `5m` (5 minutes), but it can be overridden with `max_lookback` query arg.
For instance, `/federate?match[]=up&max_lookback=1h` would return last points on the `[now - 1h ... now]` interval. This may be useful for time series federation

View file

@ -1001,7 +1001,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
Optional `max_rows_per_line` arg may be added to the request for limiting the maximum number of rows exported per each JSON line.
@ -1050,7 +1050,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/csv -d 'format=<format>' -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported CSV data can be imported to VictoriaMetrics via [/api/v1/import/csv](#how-to-import-csv-data).
@ -1078,7 +1078,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/api/v1/export/native -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
The exported data can be imported to VictoriaMetrics via [/api/v1/import/native](#how-to-import-data-in-native-format).
@ -1317,7 +1317,7 @@ See [allowed formats](#timestamp-formats) for these args.
For example:
```console
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=1654543486' -d 'end=1654543486'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48+00:00' -d 'end=2022-06-06T19:29:07+00:00'
curl http://<victoriametrics-addr>:8428/federate -d 'match[]=<timeseries_selector_for_export>' -d 'start=2022-06-06T19:25:48' -d 'end=2022-06-06T19:29:07'
```
By default, the last point on the interval `[now - max_lookback ... now]` is scraped for each time series. The default value for `max_lookback` is `5m` (5 minutes), but it can be overridden with `max_lookback` query arg.
For instance, `/federate?match[]=up&max_lookback=1h` would return last points on the `[now - 1h ... now]` interval. This may be useful for time series federation

View file

@ -267,7 +267,7 @@ consider [backfilling tips](https://docs.victoriametrics.com/Single-server-Victo
* How do I get raw data points with MetricsQL?
* _For getting raw data points specify the interval at which you want them in square brackets and send
as [instant query](https://docs.victoriametrics.com/keyConcepts.html#instant-query). For
example, `GET api/v1/query?query="my_metric[5m]"&time=<time>` will return raw samples for `my_metric` in interval
example, `GET api/v1/query?query=my_metric[5m]&time=<time>` will return raw samples for `my_metric` in interval
from `<time>` to `<time>-5m`._
* Can you have multiple aggregators in a MetricsQL query, e.g. `SELECT MAX(field), MIN(field) ...`?
* _Yes, try the following query `( alias(max(field), "max"), alias(min(field), "min") )`._

View file

@ -446,7 +446,7 @@ See also [useful tips for target relabeling](#useful-tips-for-target-relabeling)
## Useful tips for target relabeling
* Target relabelig can be debugged by clicking the `debug` link for the needed target on the `http://vmagent:8429/target`
* Target relabeling can be debugged by clicking the `debug` link for the needed target on the `http://vmagent:8429/target`
or on the `http://vmagent:8429/service-discovery` pages. See [these docs](https://docs.victoriametrics.com/vmagent.html#relabel-debug).
* Every discovered target contains a set of meta-labels, which start with `__meta_` prefix.
The specific sets of labels per each supported service discovery option are listed

View file

@ -29,7 +29,9 @@ additionally to [discovering Prometheus-compatible targets and scraping metrics
* Can add, remove and modify labels (aka tags) via Prometheus relabeling. Can filter data before sending it to remote storage. See [these docs](#relabeling) for details.
* Can accept data via all the ingestion protocols supported by VictoriaMetrics - see [these docs](#how-to-push-data-to-vmagent).
* Can aggregate incoming samples by time and by labels before sending them to remote storage - see [these docs](https://docs.victoriametrics.com/stream-aggregation.html).
* Can replicate collected metrics simultaneously to multiple remote storage systems - see [these docs](#replication-and-high-availability).
* Can replicate collected metrics simultaneously to multiple Prometheus-compatible remote storage systems - see [these docs](#replication-and-high-availability).
* Can save egress network bandwidth usage costs when [VictoriaMetrics remote write protocol](#victoriametrics-remote-write-protocol)
is used for sending the data to VictoriaMetrics.
* Works smoothly in environments with unstable connections to remote storage. If the remote storage is unavailable, the collected metrics
are buffered at `-remoteWrite.tmpDataPath`. The buffered metrics are sent to remote storage as soon as the connection
to the remote storage is repaired. The maximum disk usage for the buffer can be limited with `-remoteWrite.maxDiskUsagePerURL`.
@ -49,14 +51,15 @@ additionally to [discovering Prometheus-compatible targets and scraping metrics
Please download `vmutils-*` archive from [releases page](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) (
`vmagent` is also available in [docker images](https://hub.docker.com/r/victoriametrics/vmagent/tags)),
unpack it and pass the following flags to the `vmagent` binary in order to start scraping Prometheus-compatible targets:
unpack it and pass the following flags to the `vmagent` binary in order to start scraping Prometheus-compatible targets
and sending the data to the Prometheus-compatible remote storage:
* `-promscrape.config` with the path to Prometheus config file (usually located at `/etc/prometheus/prometheus.yml`).
* `-promscrape.config` with the path to [Prometheus config file](https://docs.victoriametrics.com/sd_configs.html) (usually located at `/etc/prometheus/prometheus.yml`).
The path can point either to local file or to http url. `vmagent` doesn't support some sections of Prometheus config file,
so you may need either to delete these sections or to run `vmagent` with `-promscrape.config.strictParse=false` command-line flag.
In this case `vmagent` ignores unsupported sections. See [the list of unsupported sections](#unsupported-prometheus-config-sections).
* `-remoteWrite.url` with the remote storage endpoint such as VictoriaMetrics, the `-remoteWrite.url` argument can be specified
multiple times to replicate data concurrently to an arbitrary number of remote storage systems. See [various use cases](#use-cases).
* `-remoteWrite.url` with Prometheus-compatible remote storage endpoint such as VictoriaMetrics, the `-remoteWrite.url` argument can be specified
multiple times to replicate data concurrently to multiple remote storage systems. See [various use cases](#use-cases).
Example command line:
@ -64,8 +67,6 @@ Example command line:
/path/to/vmagent -promscrape.config=/path/to/prometheus.yml -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
Example of scrape configuration for `-promscrape.config` argument you can find [here](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/deployment/docker/prometheus.yml)
See [how to scrape Prometheus-compatible targets](#how-to-collect-metrics-in-prometheus-format) for more details.
If you use single-node VictoriaMetrics, then you can discover and scrape Prometheus-compatible targets directly from VictoriaMetrics
@ -79,6 +80,8 @@ and sending it to the provided `-remoteWrite.url`:
/path/to/vmagent -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
`vmagent` can save network bandwidth usage costs under high load when [VictoriaMetrics remote write protocol is enabled](#victoriametrics-remote-write-protocol).
See [troubleshooting docs](#troubleshooting) if you encounter common issues with `vmagent`.
Pass `-help` to `vmagent` in order to see [the full list of supported command-line flags with their descriptions](#advanced-usage).
@ -124,7 +127,8 @@ data to the remote storage. It re-tries sending the data to remote storage until
The maximum on-disk size for the buffered metrics can be limited with `-remoteWrite.maxDiskUsagePerURL`.
`vmagent` works on various architectures from the IoT world - 32-bit arm, 64-bit arm, ppc64, 386, amd64.
See [the corresponding Makefile rules](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/app/vmagent/Makefile) for details.
The `vmagent` can save network bandwidth usage costs by using [VictoriaMetrics remote write protocol](#victoriametrics-remote-write-protocol).
### Drop-in replacement for Prometheus
@ -179,6 +183,33 @@ the `-remoteWrite.url` command-line flag should be configured as `<schema>://<vm
according to [these docs](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format).
There is also support for multitenant writes. See [these docs](#multitenancy).
## VictoriaMetrics remote write protocol
By default `vmagent` uses Prometheus remote_write protocol for sending the data to the configured `-remoteWrite.url`.
This allows sending data to [any Prometheus-compatible remote storage](https://prometheus.io/docs/operating/integrations/#remote-endpoints-and-storage).
The Prometheus remote_write protocol may require big amounts of network bandwidth under high load.
This may result in high network egress costs when the configured remote storage is located in remote datacenter or availability zone.
This also may result in the increased disk IO at `vmagent` when it writes to disk the pending data, which must be sent to remote storage.
In this case the `vmagent` can be instructed to use VictoriaMetrics remote write protocol.
This allows reducing egress network bandwidth costs while reducing disk read/write IO at `vmagent` side under high load.
The `-remoteWrite.useVMProto=true` command-line flag instructs `vmagent` to send the data to the corresponding `-remoteWrite.url`
via VictoriaMetrics remote write protocol.
While all the [recently released](https://docs.victoriametrics.com/CHANGELOG.html) VictoriaMetrics components support
the VictoriaMetrics remote write protocol, third-party systems and old versions of VictoriaMetrics components may miss the support of this protocol.
The `-remoteWrite.useVMProto` command-line flag can be set independently per each configured `-remoteWrite.url`.
For example, the following command instructs `vmagent` to send the data to `https://victoriametrics/api/v1/write` via VictoriaMetrics remote write protocol,
while sending the data to `https://prom-compatible-storage/write` via Prometheus remote write protocol:
```
./vmagent -remoteWrite.url=https://victoriametrics/api/v1/write \
-remoteWrite.useVMProto=true \
-remoteWrite.url=https://prom-compatible-storage/write \
-remoteWrite.useVMProto=false
```
## Multitenancy
By default `vmagent` collects the data without tenant identifiers and routes it to the configured `-remoteWrite.url`.
@ -884,7 +915,7 @@ If you have suggestions for improvements or have found a bug - please open an is
The number of dropped blocks can be monitored via `vmagent_remotewrite_packets_dropped_total` metric exported at [/metrics page](#monitoring).
* Use `-remoteWrite.queues=1` when `-remoteWrite.url` points to remote storage, which doesn't accept out-of-order samples (aka data backfilling).
Such storage systems include Prometheus, Cortex and Thanos, which typically emit `out of order sample` errors.
Such storage systems include Prometheus, Mimir, Cortex and Thanos, which typically emit `out of order sample` errors.
The best solution is to use remote storage with [backfilling support](https://docs.victoriametrics.com/#backfilling) such as VictoriaMetrics.
* `vmagent` buffers scraped data at the `-remoteWrite.tmpDataPath` directory until it is sent to `-remoteWrite.url`.
@ -1216,7 +1247,7 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
-insert.maxQueueDuration duration
The maximum duration to wait in the queue when -maxConcurrentInserts concurrent insert requests are executed (default 1m0s)
-internStringMaxLen int
The maximum length for strings to intern. Lower limit may save memory at the cost of higher CPU usage. See https://en.wikipedia.org/wiki/String_interning (default 300)
The maximum length for strings to intern. Lower limit may save memory at the cost of higher CPU usage. See https://en.wikipedia.org/wiki/String_interning (default 500)
-kafka.consumer.topic array
Kafka topic names for data consumption. This flag is available only in VictoriaMetrics enterprise. See https://docs.victoriametrics.com/enterprise.html
Supports an array of values separated by comma or specified via multiple flags.
@ -1509,11 +1540,14 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
-remoteWrite.tmpDataPath string
Path to directory where temporary data for remote write component is stored. See also -remoteWrite.maxDiskUsagePerURL (default "vmagent-remotewrite-data")
-remoteWrite.url array
Remote storage URL to write data to. It must support Prometheus remote_write API. It is recommended using VictoriaMetrics as remote storage. Example url: http://<victoriametrics-host>:8428/api/v1/write . Pass multiple -remoteWrite.url flags in order to replicate data to multiple remote storage systems. See also -remoteWrite.multitenantURL
Remote storage URL to write data to. It must support Prometheus remote_write protocol. Example url: http://<victoriametrics-host>:8428/api/v1/write . It is recommended setting -remoteWrite.useVMProto command-line option when VictoriaMetrics is used as a remote storage in order to save network bandwidth. See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol . Pass multiple -remoteWrite.url options in order to replicate the collected data to multiple remote storage systems. See also -remoteWrite.multitenantURL
Supports an array of values separated by comma or specified via multiple flags.
-remoteWrite.urlRelabelConfig array
Optional path to relabel configs for the corresponding -remoteWrite.url. See also -remoteWrite.relabelConfig. The path can point either to local file or to http url. See https://docs.victoriametrics.com/vmagent.html#relabeling
Supports an array of values separated by comma or specified via multiple flags.
-remoteWrite.useVMProto array
Whether to use VictoriaMetrics protocol for sending the data to the given -remoteWrite.url in order to reduce network bandwidth usage and disk read/write IO under high load. See https://docs.victoriametrics.com/vmagent.html#victoriametrics-remote-write-protocol
Supports array of values separated by comma or specified via multiple flags.
-sortLabels
Whether to sort labels for incoming samples before writing them to all the configured remote storage systems. This may be needed for reducing memory usage at remote storage when the order of labels in incoming samples is random. For example, if m{k1="v1",k2="v2"} may be sent as m{k2="v2",k1="v1"}Enabled sorting for labels can slow down ingestion performance a bit
-tls

View file

@ -32,7 +32,8 @@ Use this feature for the following cases:
* Graphite datasource can be used for alerting and recording rules. See [these docs](#graphite);
* Recording and Alerting rules backfilling (aka `replay`). See [these docs](#rules-backfilling);
* Lightweight and without extra dependencies.
* Supports [reusable templates](#reusable-templates) for annotations.
* Supports [reusable templates](#reusable-templates) for annotations;
* Load of recording and alerting rules from local filesystem, GCS and S3.
## Limitations
@ -103,26 +104,6 @@ groups:
[ - <rule_group> ]
```
### Reading rules from object storage
[Enterprise version](https://docs.victoriametrics.com/enterprise.html) of `vmalert` may read alerting and recording rules
from object storage:
- `./bin/vmalert -rule=s3://bucket/dir/alert.rules` would read rules from the given path at S3 bucket
- `./bin/vmalert -rule=gs://bucket/bir/alert.rules` would read rules from the given path at GCS bucket
S3 and GCS paths support only matching by prefix, e.g. `s3://bucket/dir/rule_` matches
all files with prefix `rule_` in the folder `dir`.
The following [command-line flags](#flags) can be used for fine-tuning access to S3 and GCS:
- `-s3.credsFilePath` - path to file with GCS or S3 credentials. Credentials are loaded from default locations if not set.
- `-s3.configFilePath` - path to file with S3 configs. Configs are loaded from default location if not set.
- `-s3.configProfile` - profile name for S3 configs. If no set, the value of the environment variable will be loaded (`AWS_PROFILE` or `AWS_DEFAULT_PROFILE`).
- `-s3.customEndpoint` - custom S3 endpoint for use with S3-compatible storages (e.g. MinIO). S3 is used if not set.
- `-s3.forcePathStyle` - prefixing endpoint with bucket name when set false, true by default.
### Groups
Each group has the following attributes:
@ -428,6 +409,25 @@ The enterprise version of vmalert is available in `vmutils-*-enterprise.tar.gz`
at [release page](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) and in `*-enterprise`
tags at [Docker Hub](https://hub.docker.com/r/victoriametrics/vmalert/tags).
### Reading rules from object storage
[Enterprise version](https://docs.victoriametrics.com/enterprise.html) of `vmalert` may read alerting and recording rules
from object storage:
- `./bin/vmalert -rule=s3://bucket/dir/alert.rules` would read rules from the given path at S3 bucket
- `./bin/vmalert -rule=gs://bucket/bir/alert.rules` would read rules from the given path at GCS bucket
S3 and GCS paths support only matching by prefix, e.g. `s3://bucket/dir/rule_` matches
all files with prefix `rule_` in the folder `dir`.
The following [command-line flags](#flags) can be used for fine-tuning access to S3 and GCS:
- `-s3.credsFilePath` - path to file with GCS or S3 credentials. Credentials are loaded from default locations if not set.
- `-s3.configFilePath` - path to file with S3 configs. Configs are loaded from default location if not set.
- `-s3.configProfile` - profile name for S3 configs. If no set, the value of the environment variable will be loaded (`AWS_PROFILE` or `AWS_DEFAULT_PROFILE`).
- `-s3.customEndpoint` - custom S3 endpoint for use with S3-compatible storages (e.g. MinIO). S3 is used if not set.
- `-s3.forcePathStyle` - prefixing endpoint with bucket name when set false, true by default.
### Topology examples
The following sections are showing how `vmalert` may be used and configured

View file

@ -210,6 +210,22 @@ mwIDAQAB
```
This command will result in 3 keys loaded: 2 keys from files and 1 from command line.
### Using JWKS endpoint for JWT signature verification
`vmgateway` supports using JWKS endpoint for JWT signature verification.
In order to enable JWKS endpoint for JWT signature verification, you need to specify JWKS endpoint URL by using `auth.jwksEndpoints` flag.
When `auth.jwksEndpoints` is specified `vmageteway` will fetch public keys from the specified endpoint and use them for JWT signature verification.
Example usage for tokens issues by Azure Active Directory:
```console
/bin/vmgateway -eula \
-enable.auth \
-write.url=http://localhost:8480 \
-read.url=http://localhost:8481 \
-auth.jwksEndpoints=https://login.microsoftonline.com/common/discovery/v2.0/keys
```
## Configuration
The shortlist of configuration flags include the following:
@ -217,6 +233,9 @@ The shortlist of configuration flags include the following:
```console
-auth.httpHeader string
HTTP header name to look for JWT authorization token (default "Authorization")
-auth.jwksEndpoints array
JWKS endpoints to fetch keys for JWT tokens signature verification
Supports an array of values separated by comma or specified via multiple flags.
-auth.publicKeyFiles array
Path file with public key to verify JWT token signature
Supports an array of values separated by comma or specified via multiple flags.

View file

@ -8,6 +8,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding/zstd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
@ -21,7 +22,7 @@ var maxInsertRequestSize = flagutil.NewBytes("maxInsertRequestSize", 32*1024*102
// Parse parses Prometheus remote_write message from reader and calls callback for the parsed timeseries.
//
// callback shouldn't hold tss after returning.
func Parse(r io.Reader, callback func(tss []prompb.TimeSeries) error) error {
func Parse(r io.Reader, isVMRemoteWrite bool, callback func(tss []prompb.TimeSeries) error) error {
wcr := writeconcurrencylimiter.GetReader(r)
defer writeconcurrencylimiter.PutReader(wcr)
r = wcr
@ -38,9 +39,16 @@ func Parse(r io.Reader, callback func(tss []prompb.TimeSeries) error) error {
bb := bodyBufferPool.Get()
defer bodyBufferPool.Put(bb)
var err error
bb.B, err = snappy.Decode(bb.B[:cap(bb.B)], ctx.reqBuf.B)
if err != nil {
return fmt.Errorf("cannot decompress request with length %d: %w", len(ctx.reqBuf.B), err)
if isVMRemoteWrite {
bb.B, err = zstd.Decompress(bb.B[:0], ctx.reqBuf.B)
if err != nil {
return fmt.Errorf("cannot decompress zstd-encoded request with length %d: %w", len(ctx.reqBuf.B), err)
}
} else {
bb.B, err = snappy.Decode(bb.B[:cap(bb.B)], ctx.reqBuf.B)
if err != nil {
return fmt.Errorf("cannot decompress snappy-encoded request with length %d: %w", len(ctx.reqBuf.B), err)
}
}
if int64(len(bb.B)) > maxInsertRequestSize.N {
return fmt.Errorf("too big unpacked request; mustn't exceed `-maxInsertRequestSize=%d` bytes; got %d bytes", maxInsertRequestSize.N, len(bb.B))