Merge branch 'public-single-node' into victorialogs-wip

This commit is contained in:
Aliaksandr Valialkin 2024-05-12 09:37:23 +02:00
commit f6ec23abcd
No known key found for this signature in database
GPG key ID: 52C003EE2BCDB9EB
116 changed files with 7017 additions and 2708 deletions

View file

@ -86,6 +86,7 @@ VictoriaMetrics has the following prominent features:
* [Prometheus exposition format](#how-to-import-data-in-prometheus-exposition-format).
* [InfluxDB line protocol](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) over HTTP, TCP and UDP.
* [Graphite plaintext protocol](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) with [tags](https://graphite.readthedocs.io/en/latest/tags.html#carbon).
* [Statsd plaintext protocol](#how-to-send-data-from-statsd-compatible-clients)
* [OpenTSDB put message](#sending-data-via-telnet-put-protocol).
* [HTTP OpenTSDB /api/put requests](#sending-opentsdb-data-via-http-apiput-requests).
* [JSON line format](#how-to-import-data-in-json-line-format).
@ -701,6 +702,45 @@ The `/api/v1/export` endpoint should return the following response:
{"metric":{"__name__":"measurement_field2","tag1":"value1","tag2":"value2"},"values":[1.23],"timestamps":[1695902762311]}
```
## How to send data from Statsd-compatible clients
VictoriaMetrics supports extended statsd protocol with tags. Also it does not support sampling and metric types(it will be ignored).
Enable Statsd receiver in VictoriaMetrics by setting `-statsdListenAddr` command line flag. For instance,
the following command will enable Statsd receiver in VictoriaMetrics on TCP and UDP port `8125`:
```console
/path/to/victoria-metrics-prod -statsdListenAddr=:8125
```
Example for writing data with Statsd plaintext protocol to local VictoriaMetrics using `nc`:
```console
echo "foo.bar:123|g|#foo:bar" | nc -N localhost 8125
```
Explicit setting of timestamps is not supported for statsd protocol. Timestamp is set to the current time when VictoriaMetrics or vmagent receives it.
An arbitrary number of lines delimited by `\n` (aka newline char) can be sent in one go.
After that the data may be read via [/api/v1/export](#how-to-export-data-in-json-line-format) endpoint:
<div class="with-copy" markdown="1">
```console
curl -G 'http://localhost:8428/api/v1/export' -d 'match=foo.bar.baz'
```
</div>
The `/api/v1/export` endpoint should return the following response:
```json
{"metric":{"__name__":"foo.bar.baz","tag1":"value1","tag2":"value2"},"values":[123],"timestamps":[1560277406000]}
```
Some examples of compatible statsd clients:
- [statsd-instrument](https://github.com/Shopify/statsd-instrument)
- [dogstatsd-ruby](https://github.com/DataDog/dogstatsd-ruby)
- [go-statsd-client](https://github.com/cactus/go-statsd-client)
## How to send data from Graphite-compatible agents such as [StatsD](https://github.com/etsy/statsd)
Enable Graphite receiver in VictoriaMetrics by setting `-graphiteListenAddr` command line flag. For instance,
@ -1356,6 +1396,7 @@ Additionally, VictoriaMetrics can accept metrics via the following popular data
* DataDog `submit metrics` API. See [these docs](#how-to-send-data-from-datadog-agent) for details.
* InfluxDB line protocol. See [these docs](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) for details.
* Graphite plaintext protocol. See [these docs](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) for details.
* Statsd plaintext protocol. See [these docs](#how-to-send-data-from-statsd-compatible-clients) for details.
* OpenTelemetry http API. See [these docs](#sending-data-via-opentelemetry) for details.
* OpenTSDB telnet put protocol. See [these docs](#sending-data-via-telnet-put-protocol) for details.
* OpenTSDB http `/api/put` protocol. See [these docs](#sending-opentsdb-data-via-http-apiput-requests) for details.
@ -3162,6 +3203,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li
An optional list of labels to drop from samples before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-streamAggr.ignoreFirstIntervals int
Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start
-streamAggr.ignoreOldSamples
Whether to ignore input samples with old timestamps outside the current aggregation interval. See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples
-streamAggr.keepInput

View file

@ -88,6 +88,9 @@ victoria-metrics-linux-ppc64le:
victoria-metrics-linux-s390x:
APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
victoria-metrics-linux-loong64:
APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
victoria-metrics-linux-386:
APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -88,6 +88,9 @@ vmagent-linux-ppc64le:
vmagent-linux-s390x:
APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmagent-linux-loong64:
APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmagent-linux-386:
APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -24,6 +24,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/prometheusimport"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/buildinfo"
@ -36,6 +37,7 @@ import (
influxserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/influx"
opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb"
opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp"
statsdserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape"
@ -61,6 +63,10 @@ var (
"See also -graphiteListenAddr.useProxyProtocol")
graphiteUseProxyProtocol = flag.Bool("graphiteListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -graphiteListenAddr . "+
"See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
statsdListenAddr = flag.String("statsdListenAddr", "", "TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. "+
"See also -statsdListenAddr.useProxyProtocol")
statsdUseProxyProtocol = flag.Bool("statsdListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -statsdListenAddr . "+
"See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
opentsdbListenAddr = flag.String("opentsdbListenAddr", "", "TCP and UDP address to listen for OpenTSDB metrics. "+
"Telnet put messages and HTTP /api/put messages are simultaneously served on TCP port. "+
"Usually :4242 must be set. Doesn't work if empty. See also -opentsdbListenAddr.useProxyProtocol")
@ -80,6 +86,7 @@ var (
var (
influxServer *influxserver.Server
graphiteServer *graphiteserver.Server
statsdServer *statsdserver.Server
opentsdbServer *opentsdbserver.Server
opentsdbhttpServer *opentsdbhttpserver.Server
)
@ -137,6 +144,9 @@ func main() {
if len(*graphiteListenAddr) > 0 {
graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, *graphiteUseProxyProtocol, graphite.InsertHandler)
}
if len(*statsdListenAddr) > 0 {
statsdServer = statsdserver.MustStart(*statsdListenAddr, *statsdUseProxyProtocol, statsd.InsertHandler)
}
if len(*opentsdbListenAddr) > 0 {
httpInsertHandler := getOpenTSDBHTTPInsertHandler()
opentsdbServer = opentsdbserver.MustStart(*opentsdbListenAddr, *opentsdbUseProxyProtocol, opentsdb.InsertHandler, httpInsertHandler)
@ -172,6 +182,9 @@ func main() {
if len(*graphiteListenAddr) > 0 {
graphiteServer.MustStop()
}
if len(*statsdListenAddr) > 0 {
statsdServer.MustStop()
}
if len(*opentsdbListenAddr) > 0 {
opentsdbServer.MustStop()
}

View file

@ -112,6 +112,7 @@ type writeRequest struct {
tss []prompbmarshal.TimeSeries
labels []prompbmarshal.Label
samples []prompbmarshal.Sample
exemplars []prompbmarshal.Exemplar
// buf holds labels data
buf []byte
@ -129,6 +130,7 @@ func (wr *writeRequest) reset() {
wr.labels = wr.labels[:0]
wr.samples = wr.samples[:0]
wr.exemplars = wr.exemplars[:0]
wr.buf = wr.buf[:0]
}
@ -200,6 +202,7 @@ func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) {
labelsDst := wr.labels
labelsLen := len(wr.labels)
samplesDst := wr.samples
exemplarsDst := wr.exemplars
buf := wr.buf
for i := range src.Labels {
labelsDst = append(labelsDst, prompbmarshal.Label{})
@ -216,8 +219,12 @@ func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) {
samplesDst = append(samplesDst, src.Samples...)
dst.Samples = samplesDst[len(samplesDst)-len(src.Samples):]
exemplarsDst = append(exemplarsDst, src.Exemplars...)
dst.Exemplars = exemplarsDst[len(exemplarsDst)-len(src.Exemplars):]
wr.samples = samplesDst
wr.labels = labelsDst
wr.exemplars = exemplarsDst
wr.buf = buf
}
@ -229,7 +236,6 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block
// Nothing to push
return true
}
marshalConcurrencyCh <- struct{}{}
bb := writeRequestBufPool.Get()
@ -266,6 +272,8 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block
if len(wr.Timeseries) == 1 {
// A single time series left. Recursively split its samples into smaller parts if possible.
samples := wr.Timeseries[0].Samples
exemplars := wr.Timeseries[0].Exemplars
if len(samples) == 1 {
logger.Warnf("dropping a sample for metric with too long labels exceeding -remoteWrite.maxBlockSize=%d bytes", maxUnpackedBlockSize.N)
return true
@ -277,11 +285,16 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block
return false
}
wr.Timeseries[0].Samples = samples[n:]
// We do not want to send exemplars twice
wr.Timeseries[0].Exemplars = nil
if !tryPushWriteRequest(wr, tryPushBlock, isVMRemoteWrite) {
wr.Timeseries[0].Samples = samples
wr.Timeseries[0].Exemplars = exemplars
return false
}
wr.Timeseries[0].Samples = samples
wr.Timeseries[0].Exemplars = exemplars
return true
}
timeseries := wr.Timeseries

View file

@ -10,8 +10,8 @@ import (
func TestPushWriteRequest(t *testing.T) {
rowsCounts := []int{1, 10, 100, 1e3, 1e4}
expectedBlockLensProm := []int{216, 1848, 16424, 169882, 1757876}
expectedBlockLensVM := []int{138, 492, 3927, 34995, 288476}
expectedBlockLensProm := []int{248, 1952, 17433, 180381, 1861994}
expectedBlockLensVM := []int{170, 575, 4748, 44936, 367096}
for i, rowsCount := range rowsCounts {
expectedBlockLenProm := expectedBlockLensProm[i]
expectedBlockLenVM := expectedBlockLensVM[i]
@ -59,6 +59,20 @@ func newTestWriteRequest(seriesCount, labelsCount int) *prompbmarshal.WriteReque
Value: fmt.Sprintf("value_%d_%d", i, j),
})
}
exemplar := prompbmarshal.Exemplar{
Labels: []prompbmarshal.Label{
{
Name: "trace_id",
Value: "123456",
},
{
Name: "log_id",
Value: "987654",
},
},
Value: float64(i),
Timestamp: 1000 * int64(i),
}
wr.Timeseries = append(wr.Timeseries, prompbmarshal.TimeSeries{
Labels: labels,
Samples: []prompbmarshal.Sample{
@ -67,6 +81,10 @@ func newTestWriteRequest(seriesCount, labelsCount int) *prompbmarshal.WriteReque
Timestamp: 1000 * int64(i),
},
},
Exemplars: []prompbmarshal.Exemplar{
exemplar,
},
})
}
return &wr

View file

@ -105,14 +105,16 @@ var (
"with -remoteWrite.streamAggr.config . See also -dedup.minScrapeInterval and https://docs.victoriametrics.com/stream-aggregation/#deduplication")
streamAggrIgnoreOldSamples = flagutil.NewArrayBool("remoteWrite.streamAggr.ignoreOldSamples", "Whether to ignore input samples with old timestamps outside the current aggregation interval "+
"for the corresponding -remoteWrite.streamAggr.config . See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples")
streamAggrIgnoreFirstIntervals = flag.Int("remoteWrite.streamAggr.ignoreFirstIntervals", 0, "Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after vmagent restarts. It could be caused by receiving unordered delayed data from clients pushing data into the vmagent. "+
"See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start")
streamAggrDropInputLabels = flagutil.NewArrayString("streamAggr.dropInputLabels", "An optional list of labels to drop from samples "+
"before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels")
disableOnDiskQueue = flag.Bool("remoteWrite.disableOnDiskQueue", false, "Whether to disable storing pending data to -remoteWrite.tmpDataPath "+
"when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence ."+
disableOnDiskQueue = flagutil.NewArrayBool("remoteWrite.disableOnDiskQueue", "Whether to disable storing pending data to -remoteWrite.tmpDataPath "+
"when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence ."+
"See also -remoteWrite.dropSamplesOnOverload")
dropSamplesOnOverload = flag.Bool("remoteWrite.dropSamplesOnOverload", false, "Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples "+
"cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence")
dropSamplesOnOverload = flagutil.NewArrayBool("remoteWrite.dropSamplesOnOverload", "Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples "+
"cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence")
)
var (
@ -133,6 +135,9 @@ var (
"see https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence"),
StatusCode: http.StatusTooManyRequests,
}
// disableOnDiskQueueAll is set to true if all remoteWrite.urls were configured to disable persistent queue via disableOnDiskQueue
disableOnDiskQueueAll bool
)
// MultitenancyEnabled returns true if -enableMultitenantHandlers or -remoteWrite.multitenantURL is specified.
@ -225,6 +230,15 @@ func Init() {
if len(*remoteWriteURLs) > 0 {
rwctxsDefault = newRemoteWriteCtxs(nil, *remoteWriteURLs)
}
disableOnDiskQueueAll = true
for _, v := range *disableOnDiskQueue {
if !v {
disableOnDiskQueueAll = false
break
}
}
dropDanglingQueues()
// Start config reloader.
@ -257,6 +271,9 @@ func dropDanglingQueues() {
// This is required for the case when the number of queues has been changed or URL have been changed.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4014
//
// In case if there were many persistent queues with identical *remoteWriteURLs
// the queue with the last index will be dropped.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6140
existingQueues := make(map[string]struct{}, len(rwctxsDefault))
for _, rwctx := range rwctxsDefault {
existingQueues[rwctx.fq.Dirname()] = struct{}{}
@ -435,10 +452,10 @@ func PushDropSamplesOnFailure(at *auth.Token, wr *prompbmarshal.WriteRequest) {
//
// The caller must return ErrQueueFullHTTPRetry to the client, which sends wr, if TryPush returns false.
func TryPush(at *auth.Token, wr *prompbmarshal.WriteRequest) bool {
return tryPush(at, wr, *dropSamplesOnOverload)
return tryPush(at, wr, false)
}
func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailure bool) bool {
func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, forceDropSamplesOnFailure bool) bool {
tss := wr.Timeseries
if at == nil && MultitenancyEnabled() {
@ -471,17 +488,18 @@ func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailur
rowsCount := getRowsCount(tss)
if *disableOnDiskQueue {
// Quick check whether writes to configured remote storage systems are blocked.
// This allows saving CPU time spent on relabeling and block compression
// if some of remote storage systems cannot keep up with the data ingestion rate.
// this shortcut is only applicable if all remote writes have disableOnDiskQueue = true
if disableOnDiskQueueAll {
for _, rwctx := range rwctxs {
if rwctx.fq.IsWriteBlocked() {
pushFailures.Inc()
if dropSamplesOnFailure {
rwctx.pushFailures.Inc()
if forceDropSamplesOnFailure || rwctx.dropSamplesOnOverload {
// Just drop samples
samplesDropped.Add(rowsCount)
return true
rwctx.rowsDroppedOnPushFailure.Add(rowsCount)
continue
}
return false
}
@ -534,27 +552,14 @@ func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailur
}
sortLabelsIfNeeded(tssBlock)
tssBlock = limitSeriesCardinality(tssBlock)
if !tryPushBlockToRemoteStorages(rwctxs, tssBlock) {
if !*disableOnDiskQueue {
logger.Panicf("BUG: tryPushBlockToRemoteStorages must return true if -remoteWrite.disableOnDiskQueue isn't set")
}
pushFailures.Inc()
if dropSamplesOnFailure {
samplesDropped.Add(rowsCount)
return true
}
if !tryPushBlockToRemoteStorages(rwctxs, tssBlock, forceDropSamplesOnFailure) {
return false
}
}
return true
}
var (
samplesDropped = metrics.NewCounter(`vmagent_remotewrite_samples_dropped_total`)
pushFailures = metrics.NewCounter(`vmagent_remotewrite_push_failures_total`)
)
func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries) bool {
func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, forceDropSamplesOnFailure bool) bool {
if len(tssBlock) == 0 {
// Nothing to push
return true
@ -562,7 +567,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar
if len(rwctxs) == 1 {
// Fast path - just push data to the configured single remote storage
return rwctxs[0].TryPush(tssBlock)
return rwctxs[0].TryPush(tssBlock, forceDropSamplesOnFailure)
}
// We need to push tssBlock to multiple remote storages.
@ -573,7 +578,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar
if replicas <= 0 {
replicas = 1
}
return tryShardingBlockAmongRemoteStorages(rwctxs, tssBlock, replicas)
return tryShardingBlockAmongRemoteStorages(rwctxs, tssBlock, replicas, forceDropSamplesOnFailure)
}
// Replicate tssBlock samples among rwctxs.
@ -585,7 +590,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar
for _, rwctx := range rwctxs {
go func(rwctx *remoteWriteCtx) {
defer wg.Done()
if !rwctx.TryPush(tssBlock) {
if !rwctx.TryPush(tssBlock, forceDropSamplesOnFailure) {
anyPushFailed.Store(true)
}
}(rwctx)
@ -594,7 +599,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar
return !anyPushFailed.Load()
}
func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, replicas int) bool {
func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, replicas int, forceDropSamplesOnFailure bool) bool {
x := getTSSShards(len(rwctxs))
defer putTSSShards(x)
@ -648,7 +653,7 @@ func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []pr
wg.Add(1)
go func(rwctx *remoteWriteCtx, tss []prompbmarshal.TimeSeries) {
defer wg.Done()
if !rwctx.TryPush(tss) {
if !rwctx.TryPush(tss, forceDropSamplesOnFailure) {
anyPushFailed.Store(true)
}
}(rwctx, shard)
@ -783,12 +788,17 @@ type remoteWriteCtx struct {
streamAggrKeepInput bool
streamAggrDropInput bool
disableOnDiskQueue bool
dropSamplesOnOverload bool
pss []*pendingSeries
pssNextIdx atomic.Uint64
rowsPushedAfterRelabel *metrics.Counter
rowsDroppedByRelabel *metrics.Counter
pushFailures *metrics.Counter
rowsDroppedOnPushFailure *metrics.Counter
}
func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks int, sanitizedURL string) *remoteWriteCtx {
@ -804,7 +814,8 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in
logger.Warnf("rounding the -remoteWrite.maxDiskUsagePerURL=%d to the minimum supported value: %d", maxPendingBytes, persistentqueue.DefaultChunkFileSize)
maxPendingBytes = persistentqueue.DefaultChunkFileSize
}
fq := persistentqueue.MustOpenFastQueue(queuePath, sanitizedURL, maxInmemoryBlocks, maxPendingBytes, *disableOnDiskQueue)
isPQDisabled := disableOnDiskQueue.GetOptionalArg(argIdx)
fq := persistentqueue.MustOpenFastQueue(queuePath, sanitizedURL, maxInmemoryBlocks, maxPendingBytes, isPQDisabled)
_ = metrics.GetOrCreateGauge(fmt.Sprintf(`vmagent_remotewrite_pending_data_bytes{path=%q, url=%q}`, queuePath, sanitizedURL), func() float64 {
return float64(fq.GetPendingBytes())
})
@ -847,8 +858,14 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in
c: c,
pss: pss,
dropSamplesOnOverload: dropSamplesOnOverload.GetOptionalArg(argIdx),
disableOnDiskQueue: isPQDisabled,
rowsPushedAfterRelabel: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_rows_pushed_after_relabel_total{path=%q, url=%q}`, queuePath, sanitizedURL)),
rowsDroppedByRelabel: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_relabel_metrics_dropped_total{path=%q, url=%q}`, queuePath, sanitizedURL)),
pushFailures: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_push_failures_total{path=%q, url=%q}`, queuePath, sanitizedURL)),
rowsDroppedOnPushFailure: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_samples_dropped_total{path=%q, url=%q}`, queuePath, sanitizedURL)),
}
// Initialize sas
@ -860,6 +877,7 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in
DedupInterval: dedupInterval,
DropInputLabels: *streamAggrDropInputLabels,
IgnoreOldSamples: ignoreOldSamples,
IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals,
}
sas, err := streamaggr.LoadFromFile(sasFile, rwctx.pushInternalTrackDropped, opts)
if err != nil {
@ -904,7 +922,11 @@ func (rwctx *remoteWriteCtx) MustStop() {
rwctx.rowsDroppedByRelabel = nil
}
func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool {
// TryPush sends tss series to the configured remote write endpoint
//
// TryPush can be called concurrently for multiple remoteWriteCtx,
// so it shouldn't modify tss entries.
func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries, forceDropSamplesOnFailure bool) bool {
// Apply relabeling
var rctx *relabelCtx
var v *[]prompbmarshal.TimeSeries
@ -943,7 +965,6 @@ func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool {
matchIdxsPool.Put(matchIdxs)
} else if rwctx.deduplicator != nil {
rwctx.deduplicator.Push(tss)
clear(tss)
tss = tss[:0]
}
@ -957,6 +978,14 @@ func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool {
putRelabelCtx(rctx)
}
if !ok {
rwctx.pushFailures.Inc()
if forceDropSamplesOnFailure || rwctx.dropSamplesOnOverload {
rwctx.rowsDroppedOnPushFailure.Add(len(tss))
return true
}
}
return ok
}
@ -981,13 +1010,13 @@ func (rwctx *remoteWriteCtx) pushInternalTrackDropped(tss []prompbmarshal.TimeSe
if rwctx.tryPushInternal(tss) {
return
}
if !*disableOnDiskQueue {
if !rwctx.disableOnDiskQueue {
logger.Panicf("BUG: tryPushInternal must return true if -remoteWrite.disableOnDiskQueue isn't set")
}
pushFailures.Inc()
if *dropSamplesOnOverload {
rwctx.pushFailures.Inc()
if dropSamplesOnOverload.GetOptionalArg(rwctx.idx) {
rowsCount := getRowsCount(tss)
samplesDropped.Add(rowsCount)
rwctx.rowsDroppedOnPushFailure.Add(rowsCount)
}
}

View file

@ -3,9 +3,16 @@ package remotewrite
import (
"fmt"
"math"
"os"
"reflect"
"testing"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/streamaggr"
"github.com/VictoriaMetrics/metrics"
)
func TestGetLabelsHash_Distribution(t *testing.T) {
@ -46,3 +53,163 @@ func TestGetLabelsHash_Distribution(t *testing.T) {
f(5)
f(10)
}
func TestRemoteWriteContext_TryPush_ImmutableTimeseries(t *testing.T) {
f := func(streamAggrConfig, relabelConfig string, dedupInterval time.Duration, keepInput, dropInput bool, input string) {
t.Helper()
perURLRelabel, err := promrelabel.ParseRelabelConfigsData([]byte(relabelConfig))
if err != nil {
t.Fatalf("cannot load relabel configs: %s", err)
}
rcs := &relabelConfigs{
perURL: []*promrelabel.ParsedConfigs{
perURLRelabel,
},
}
allRelabelConfigs.Store(rcs)
pss := make([]*pendingSeries, 1)
pss[0] = newPendingSeries(nil, true, 0, 100)
rwctx := &remoteWriteCtx{
idx: 0,
streamAggrKeepInput: keepInput,
streamAggrDropInput: dropInput,
pss: pss,
rowsPushedAfterRelabel: metrics.GetOrCreateCounter(`foo`),
rowsDroppedByRelabel: metrics.GetOrCreateCounter(`bar`),
}
if dedupInterval > 0 {
rwctx.deduplicator = streamaggr.NewDeduplicator(nil, dedupInterval, nil)
}
if len(streamAggrConfig) > 0 {
f := createFile(t, []byte(streamAggrConfig))
sas, err := streamaggr.LoadFromFile(f.Name(), nil, nil)
if err != nil {
t.Fatalf("cannot load streamaggr configs: %s", err)
}
rwctx.sas.Store(sas)
}
inputTss := mustParsePromMetrics(input)
expectedTss := make([]prompbmarshal.TimeSeries, len(inputTss))
// copy inputTss to make sure it is not mutated during TryPush call
copy(expectedTss, inputTss)
rwctx.TryPush(inputTss, false)
if !reflect.DeepEqual(expectedTss, inputTss) {
t.Fatalf("unexpected samples;\ngot\n%v\nwant\n%v", inputTss, expectedTss)
}
}
f(`
- interval: 1m
outputs: [sum_samples]
- interval: 2m
outputs: [count_series]
`, `
- action: keep
source_labels: [env]
regex: "dev"
`, 0, false, false, `
metric{env="dev"} 10
metric{env="bar"} 20
metric{env="dev"} 15
metric{env="bar"} 25
`)
f(``, ``, time.Hour, false, false, `
metric{env="dev"} 10
metric{env="foo"} 20
metric{env="dev"} 15
metric{env="foo"} 25
`)
f(``, `
- action: keep
source_labels: [env]
regex: "dev"
`, time.Hour, false, false, `
metric{env="dev"} 10
metric{env="bar"} 20
metric{env="dev"} 15
metric{env="bar"} 25
`)
f(``, `
- action: keep
source_labels: [env]
regex: "dev"
`, time.Hour, true, false, `
metric{env="test"} 10
metric{env="dev"} 20
metric{env="foo"} 15
metric{env="dev"} 25
`)
f(``, `
- action: keep
source_labels: [env]
regex: "dev"
`, time.Hour, false, true, `
metric{env="foo"} 10
metric{env="dev"} 20
metric{env="foo"} 15
metric{env="dev"} 25
`)
f(``, `
- action: keep
source_labels: [env]
regex: "dev"
`, time.Hour, true, true, `
metric{env="dev"} 10
metric{env="test"} 20
metric{env="dev"} 15
metric{env="bar"} 25
`)
}
func mustParsePromMetrics(s string) []prompbmarshal.TimeSeries {
var rows prometheus.Rows
errLogger := func(s string) {
panic(fmt.Errorf("unexpected error when parsing Prometheus metrics: %s", s))
}
rows.UnmarshalWithErrLogger(s, errLogger)
var tss []prompbmarshal.TimeSeries
samples := make([]prompbmarshal.Sample, 0, len(rows.Rows))
for _, row := range rows.Rows {
labels := make([]prompbmarshal.Label, 0, len(row.Tags)+1)
labels = append(labels, prompbmarshal.Label{
Name: "__name__",
Value: row.Metric,
})
for _, tag := range row.Tags {
labels = append(labels, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
samples = append(samples, prompbmarshal.Sample{
Value: row.Value,
Timestamp: row.Timestamp,
})
ts := prompbmarshal.TimeSeries{
Labels: labels,
Samples: samples[len(samples)-1:],
}
tss = append(tss, ts)
}
return tss
}
func createFile(t *testing.T, data []byte) *os.File {
t.Helper()
f, err := os.CreateTemp("", "")
if err != nil {
t.Fatal(err)
}
if err := os.WriteFile(f.Name(), data, 0644); err != nil {
t.Fatal(err)
}
if err := f.Sync(); err != nil {
t.Fatal(err)
}
return f
}

View file

@ -0,0 +1,68 @@
package statsd
import (
"io"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd/stream"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="statsd"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="statsd"}`)
)
// InsertHandler processes remote write for statsd plaintext protocol.
//
// See https://github.com/statsd/statsd/blob/master/docs/metric_types.md
func InsertHandler(r io.Reader) error {
return stream.Parse(r, false, func(rows []parser.Row) error {
return insertRows(nil, rows)
})
}
func insertRows(at *auth.Token, rows []parser.Row) error {
ctx := common.GetPushCtx()
defer common.PutPushCtx(ctx)
tssDst := ctx.WriteRequest.Timeseries[:0]
labels := ctx.Labels[:0]
samples := ctx.Samples[:0]
for i := range rows {
r := &rows[i]
labelsLen := len(labels)
labels = append(labels, prompbmarshal.Label{
Name: "__name__",
Value: r.Metric,
})
for j := range r.Tags {
tag := &r.Tags[j]
labels = append(labels, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
samples = append(samples, prompbmarshal.Sample{
Value: r.Value,
Timestamp: r.Timestamp,
})
tssDst = append(tssDst, prompbmarshal.TimeSeries{
Labels: labels[labelsLen:],
Samples: samples[len(samples)-1:],
})
}
ctx.WriteRequest.Timeseries = tssDst
ctx.Labels = labels
ctx.Samples = samples
if !remotewrite.TryPush(at, &ctx.WriteRequest) {
return remotewrite.ErrQueueFullHTTPRetry
}
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return nil
}

View file

@ -119,6 +119,9 @@ vmalert-linux-ppc64le:
vmalert-linux-s390x:
APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmalert-linux-loong64:
APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmalert-linux-386:
APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -32,7 +32,7 @@ type Rule interface {
close()
}
var errDuplicate = errors.New("result contains metrics with the same labelset after applying rule labels. See https://docs.victoriametrics.com/vmalert/#series-with-the-same-labelset for details")
var errDuplicate = errors.New("result contains metrics with the same labelset during evaluation. See https://docs.victoriametrics.com/vmalert/#series-with-the-same-labelset for details")
type ruleState struct {
sync.RWMutex

View file

@ -13,6 +13,20 @@ function collapseAll() {
$('.collapse').removeClass('show');
}
function showByID(id) {
if (!id) {
return
}
let parent = $("#" + id).parent();
if (!parent) {
return
}
let target = $("#" + parent.attr("data-bs-target"));
if (target.length > 0) {
target.addClass('show');
}
}
function toggleByID(id) {
if (id) {
let el = $("#" + id);
@ -61,7 +75,7 @@ function search() {
function setParamURL(key, value) {
let url = new URL(location.href)
url.searchParams.set(key, value);
window.history.replaceState(null, null, `?${url.searchParams.toString()}`);
window.history.replaceState(null, null, `?${url.searchParams.toString()}${url.hash}`);
}
function getParamURL(key) {
@ -141,7 +155,7 @@ $(document).ready(function () {
search()
let hash = window.location.hash.substr(1);
toggleByID(hash);
showByID(hash);
});
$(document).ready(function () {

View file

@ -87,6 +87,9 @@ vmauth-linux-ppc64le:
vmauth-linux-s390x:
APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmauth-linux-loong64:
APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmauth-linux-386:
APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -685,6 +685,70 @@ func isSetBool(boolP *bool, expectedValue bool) bool {
return *boolP == expectedValue
}
func TestGetLeastLoadedBackendURL(t *testing.T) {
up := mustParseURLs([]string{
"http://node1:343",
"http://node2:343",
"http://node3:343",
})
up.loadBalancingPolicy = "least_loaded"
fn := func(ns ...int) {
t.Helper()
bus := up.bus.Load()
pbus := *bus
for i, b := range pbus {
got := int(b.concurrentRequests.Load())
exp := ns[i]
if got != exp {
t.Fatalf("expected %q to have %d concurrent requests; got %d instead", b.url, exp, got)
}
}
}
up.getBackendURL()
fn(0, 1, 0)
up.getBackendURL()
fn(0, 1, 1)
up.getBackendURL()
fn(1, 1, 1)
up.getBackendURL()
up.getBackendURL()
fn(1, 2, 2)
bus := up.bus.Load()
pbus := *bus
pbus[0].concurrentRequests.Add(2)
pbus[2].concurrentRequests.Add(5)
fn(3, 2, 7)
up.getBackendURL()
fn(3, 3, 7)
up.getBackendURL()
fn(3, 4, 7)
up.getBackendURL()
fn(4, 4, 7)
up.getBackendURL()
fn(5, 4, 7)
up.getBackendURL()
fn(5, 5, 7)
up.getBackendURL()
fn(6, 5, 7)
up.getBackendURL()
fn(6, 6, 7)
up.getBackendURL()
up.getBackendURL()
fn(7, 7, 7)
}
func getRegexs(paths []string) []*Regex {
var sps []*Regex
for _, path := range paths {

View file

@ -81,6 +81,9 @@ vmbackup-linux-ppc64le:
vmbackup-linux-s390x:
APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmbackup-linux-loong64:
APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmbackup-linux-386:
APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -81,6 +81,9 @@ vmctl-linux-ppc64le:
vmctl-linux-s390x:
APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmctl-linux-loong64:
APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmctl-linux-386:
APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -32,6 +32,8 @@ var (
"See also -streamAggr.dropInputLabels and -dedup.minScrapeInterval and https://docs.victoriametrics.com/stream-aggregation/#deduplication")
streamAggrDropInputLabels = flagutil.NewArrayString("streamAggr.dropInputLabels", "An optional list of labels to drop from samples "+
"before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels")
streamAggrIgnoreFirstIntervals = flag.Int("streamAggr.ignoreFirstIntervals", 0, "Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. "+
"See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start")
streamAggrIgnoreOldSamples = flag.Bool("streamAggr.ignoreOldSamples", false, "Whether to ignore input samples with old timestamps outside the current aggregation interval. "+
"See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples")
)
@ -59,6 +61,7 @@ func CheckStreamAggrConfig() error {
DedupInterval: *streamAggrDedupInterval,
DropInputLabels: *streamAggrDropInputLabels,
IgnoreOldSamples: *streamAggrIgnoreOldSamples,
IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals,
}
sas, err := streamaggr.LoadFromFile(*streamAggrConfig, pushNoop, opts)
if err != nil {
@ -87,6 +90,7 @@ func InitStreamAggr() {
DedupInterval: *streamAggrDedupInterval,
DropInputLabels: *streamAggrDropInputLabels,
IgnoreOldSamples: *streamAggrIgnoreOldSamples,
IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals,
}
sas, err := streamaggr.LoadFromFile(*streamAggrConfig, pushAggregateSeries, opts)
if err != nil {
@ -120,6 +124,7 @@ func reloadStreamAggrConfig() {
DedupInterval: *streamAggrDedupInterval,
DropInputLabels: *streamAggrDropInputLabels,
IgnoreOldSamples: *streamAggrIgnoreOldSamples,
IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals,
}
sasNew, err := streamaggr.LoadFromFile(*streamAggrConfig, pushAggregateSeries, opts)
if err != nil {

View file

@ -26,6 +26,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/prompush"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/relabel"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
@ -36,6 +37,7 @@ import (
influxserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/influx"
opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb"
opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp"
statsdserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape"
@ -49,6 +51,10 @@ var (
"See also -graphiteListenAddr.useProxyProtocol")
graphiteUseProxyProtocol = flag.Bool("graphiteListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -graphiteListenAddr . "+
"See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
statsdListenAddr = flag.String("statsdListenAddr", "", "TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. "+
"See also -statsdListenAddr.useProxyProtocol")
statsdUseProxyProtocol = flag.Bool("statsdListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -statsdListenAddr . "+
"See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
influxListenAddr = flag.String("influxListenAddr", "", "TCP and UDP address to listen for InfluxDB line protocol data. Usually :8089 must be set. Doesn't work if empty. "+
"This flag isn't needed when ingesting data over HTTP - just send it to http://<victoriametrics>:8428/write . "+
"See also -influxListenAddr.useProxyProtocol")
@ -72,6 +78,7 @@ var (
var (
graphiteServer *graphiteserver.Server
statsdServer *statsdserver.Server
influxServer *influxserver.Server
opentsdbServer *opentsdbserver.Server
opentsdbhttpServer *opentsdbhttpserver.Server
@ -92,6 +99,9 @@ func Init() {
if len(*graphiteListenAddr) > 0 {
graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, *graphiteUseProxyProtocol, graphite.InsertHandler)
}
if len(*statsdListenAddr) > 0 {
statsdServer = statsdserver.MustStart(*statsdListenAddr, *statsdUseProxyProtocol, statsd.InsertHandler)
}
if len(*influxListenAddr) > 0 {
influxServer = influxserver.MustStart(*influxListenAddr, *influxUseProxyProtocol, influx.InsertHandlerForReader)
}
@ -112,6 +122,9 @@ func Stop() {
if len(*graphiteListenAddr) > 0 {
graphiteServer.MustStop()
}
if len(*statsdListenAddr) > 0 {
statsdServer.MustStop()
}
if len(*influxListenAddr) > 0 {
influxServer.MustStop()
}

View file

@ -0,0 +1,54 @@
package statsd
import (
"io"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/relabel"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd/stream"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="statsd"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="statsd"}`)
)
// InsertHandler processes remote write for statsd protocol with tags.
//
// https://github.com/statsd/statsd/blob/master/docs/metric_types.md
func InsertHandler(r io.Reader) error {
return stream.Parse(r, false, insertRows)
}
func insertRows(rows []parser.Row) error {
ctx := common.GetInsertCtx()
defer common.PutInsertCtx(ctx)
ctx.Reset(len(rows))
hasRelabeling := relabel.HasRelabeling()
for i := range rows {
r := &rows[i]
ctx.Labels = ctx.Labels[:0]
ctx.AddLabel("", r.Metric)
for j := range r.Tags {
tag := &r.Tags[j]
ctx.AddLabel(tag.Key, tag.Value)
}
if hasRelabeling {
ctx.ApplyRelabeling()
}
if len(ctx.Labels) == 0 {
// Skip metric without labels.
continue
}
ctx.SortLabelsIfNeeded()
if err := ctx.WriteDataPoint(nil, ctx.Labels, r.Timestamp, r.Value); err != nil {
return err
}
}
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return ctx.FlushBufs()
}

View file

@ -81,6 +81,9 @@ vmrestore-linux-ppc64le:
vmrestore-linux-s390x:
APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch
vmrestore-linux-loong64:
APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch
vmrestore-linux-386:
APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch

View file

@ -13,6 +13,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/netstorage"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/querystats"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/decimal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/querytracer"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
"github.com/VictoriaMetrics/metrics"
@ -26,6 +27,12 @@ var (
`For example, foo{bar=~"a.b.c"} will be automatically converted to foo{bar=~"a\\.b\\.c"}, i.e. all the dots in regexp filters will be automatically escaped `+
`in order to match only dot char instead of matching any char. Dots in ".+", ".*" and ".{n}" regexps aren't escaped. `+
`This option is DEPRECATED in favor of {__graphite__="a.*.c"} syntax for selecting metrics matching the given Graphite metrics filter`)
disableImplicitConversion = flag.Bool("search.disableImplicitConversion", false, "Whether to return an error for queries that rely on implicit subquery conversions, "+
"see https://docs.victoriametrics.com/metricsql/#subqueries for details. "+
"See also -search.logImplicitConversion.")
logImplicitConversion = flag.Bool("search.logImplicitConversion", false, "Whether to log queries with implicit subquery conversions, "+
"see https://docs.victoriametrics.com/metricsql/#subqueries for details. "+
"Such conversion can be disabled using -search.disableImplicitConversion.")
)
// UserReadableError is a type of error which supposed to be returned to the user without additional context.
@ -63,6 +70,16 @@ func Exec(qt *querytracer.Tracer, ec *EvalConfig, q string, isFirstPointOnly boo
return nil, err
}
if *disableImplicitConversion || *logImplicitConversion {
complete := isSubQueryComplete(e, false)
if !complete && *disableImplicitConversion {
return nil, fmt.Errorf("query contains subquery that requires implicit conversion and is rejected according to `-search.disableImplicitConversion=true` setting. See https://docs.victoriametrics.com/metricsql/#subqueries for details")
}
if !complete && *logImplicitConversion {
logger.Warnf("query=%q contains subquery that requires implicit conversion, see https://docs.victoriametrics.com/metricsql/#subqueries for details", e.AppendString(nil))
}
}
qid := activeQueriesV.Add(ec, q)
rv, err := evalExpr(qt, ec, e)
activeQueriesV.Remove(qid)
@ -404,3 +421,55 @@ func (pc *parseCache) Put(q string, pcv *parseCacheValue) {
pc.m[q] = pcv
pc.mu.Unlock()
}
// isSubQueryComplete checks if expr contains incomplete subquery
func isSubQueryComplete(e metricsql.Expr, isSubExpr bool) bool {
switch exp := e.(type) {
case *metricsql.FuncExpr:
if isSubExpr {
return false
}
fe := e.(*metricsql.FuncExpr)
for _, arg := range exp.Args {
if getRollupFunc(fe.Name) != nil {
isSubExpr = true
}
if !isSubQueryComplete(arg, isSubExpr) {
return false
}
}
case *metricsql.RollupExpr:
if _, ok := exp.Expr.(*metricsql.MetricExpr); ok {
return true
}
// exp.Step is optional in subqueries
if exp.Window == nil {
return false
}
return isSubQueryComplete(exp.Expr, false)
case *metricsql.AggrFuncExpr:
if isSubExpr {
return false
}
for _, arg := range exp.Args {
if !isSubQueryComplete(arg, false) {
return false
}
}
case *metricsql.BinaryOpExpr:
if isSubExpr {
return false
}
if !isSubQueryComplete(exp.Left, false) {
return false
}
if !isSubQueryComplete(exp.Right, false) {
return false
}
case *metricsql.MetricExpr:
return true
default:
return true
}
return true
}

View file

@ -9432,3 +9432,104 @@ func testAddLabels(t *testing.T, mn *storage.MetricName, labels ...string) {
})
}
}
func TestIsSubQueryCompleteTrue(t *testing.T) {
f := func(q string) {
t.Helper()
e, err := metricsql.Parse(q)
if err != nil {
t.Fatal(err)
}
if !isSubQueryComplete(e, false) {
t.Fatalf("query should be complete: %s", e.AppendString(nil))
}
}
f("rate(http_total)")
f("sum(http_total)")
f("absent(http_total)")
f("rate(http_total[1m])")
f("avg_over_time(up[1m])")
f("sum(http_total[1m])")
f("sum(rate(http_total))")
f("sum(sum(http_total))")
f(`sum(sum_over_time(http_total[1m] )) by (instance)`)
f("sum(up{cluster='a'}[1m] or up{cluster='b'}[1m])")
f("(avg_over_time(alarm_test1[1m]) - avg_over_time(alarm_test1[1m] offset 5m)) > 0.1")
f("http_total[1m] offset 1m")
// subquery
f("rate(http_total)[5m:1m]")
f("rate(sum(http_total)[5m:1m])")
f("rate(rate(http_total)[5m:1m])")
f("sum(rate(http_total[1m]))")
f("sum(rate(sum(http_total)[5m:1m]))")
f("rate(sum(rate(http_total))[5m:1m])")
f("rate(sum(sum(http_total))[5m:1m])")
f("rate(sum(rate(http_total))[5m:1m])")
f("rate(sum(sum(http_total))[5m:1m])")
f("avg_over_time(rate(http_total[5m])[5m:1m])")
f("delta(avg_over_time(up[1m])[5m:1m]) > 0.1")
f("avg_over_time(avg by (site) (metric)[2m:1m])")
f("sum(http_total)[5m:1m] offset 1m")
f("round(sum(sum_over_time(http_total[1m])) by (instance)) [5m:1m] offset 1m")
f("rate(sum(http_total)[5m:1m]) - rate(sum(http_total)[5m:1m])")
f("avg_over_time((rate(http_total)-rate(http_total))[5m:1m])")
f("sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])")
f("sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])")
f("sum(sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])) by (instance)")
// step (or resolution) is optional in subqueries
f("max_over_time(rate(my_counter_total[5m])[1h:])")
f("max_over_time(rate(my_counter_total[5m])[1h:1m])[5m:1m]")
f("max_over_time(rate(my_counter_total[5m])[1h:])[5m:]")
f(`
WITH (
cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"},
cpuIdle = rate(cpuSeconds{mode='idle'}[5m])
)
max_over_time(cpuIdle[1h:])`)
}
func TestIsSubQueryCompleteFalse(t *testing.T) {
f := func(q string) {
t.Helper()
e, err := metricsql.Parse(q)
if err != nil {
t.Fatal(err)
}
if isSubQueryComplete(e, false) {
t.Fatalf("expect to detect incomplete subquery: %s", e.AppendString(nil))
}
}
f("rate(sum(http_total))")
f("rate(rate(http_total))")
f("sum(rate(sum(http_total)))")
f("rate(sum(rate(http_total)))")
f("rate(sum(sum(http_total)))")
f("avg_over_time(rate(http_total[5m]))")
// https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3974
f("sum(http_total) offset 1m")
f(`round(sum(sum_over_time(http_total[1m])) by (instance)) offset 1m`)
f("rate(sum(http_total)) - rate(sum(http_total))")
f("avg_over_time(rate(http_total)-rate(http_total))")
// https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3996
f("sum_over_time(up{cluster='a'} or up{cluster='b'})")
f("sum_over_time(up{cluster='a'}[1m] or up{cluster='b'}[1m])")
f("sum(sum_over_time(up{cluster='a'}[1m] or up{cluster='b'}[1m])) by (instance)")
f(`
WITH (
cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"},
cpuIdle = rate(cpuSeconds{mode='idle'}[5m])
)
max_over_time(cpuIdle)`)
}

View file

@ -1,13 +1,13 @@
{
"files": {
"main.css": "./static/css/main.4ebf2874.css",
"main.js": "./static/js/main.202937c2.js",
"main.css": "./static/css/main.d0b400e3.css",
"main.js": "./static/js/main.59c17910.js",
"static/js/685.bebe1265.chunk.js": "./static/js/685.bebe1265.chunk.js",
"static/media/MetricsQL.md": "./static/media/MetricsQL.da86c2db4f0b05e286b0.md",
"static/media/MetricsQL.md": "./static/media/MetricsQL.df7574389d8f8bbcf0c7.md",
"index.html": "./index.html"
},
"entrypoints": [
"static/css/main.4ebf2874.css",
"static/js/main.202937c2.js"
"static/css/main.d0b400e3.css",
"static/js/main.59c17910.js"
]
}

View file

@ -1 +1 @@
<!doctype html><html lang="en"><head><meta charset="utf-8"/><link rel="icon" href="./favicon.ico"/><meta name="viewport" content="width=device-width,initial-scale=1,maximum-scale=5"/><meta name="theme-color" content="#000000"/><meta name="description" content="UI for VictoriaMetrics"/><link rel="apple-touch-icon" href="./apple-touch-icon.png"/><link rel="icon" type="image/png" sizes="32x32" href="./favicon-32x32.png"><link rel="manifest" href="./manifest.json"/><title>VM UI</title><script src="./dashboards/index.js" type="module"></script><meta name="twitter:card" content="summary_large_image"><meta name="twitter:image" content="./preview.jpg"><meta name="twitter:title" content="UI for VictoriaMetrics"><meta name="twitter:description" content="Explore and troubleshoot your VictoriaMetrics data"><meta name="twitter:site" content="@VictoriaMetrics"><meta property="og:title" content="Metric explorer for VictoriaMetrics"><meta property="og:description" content="Explore and troubleshoot your VictoriaMetrics data"><meta property="og:image" content="./preview.jpg"><meta property="og:type" content="website"><script defer="defer" src="./static/js/main.202937c2.js"></script><link href="./static/css/main.4ebf2874.css" rel="stylesheet"></head><body><noscript>You need to enable JavaScript to run this app.</noscript><div id="root"></div></body></html>
<!doctype html><html lang="en"><head><meta charset="utf-8"/><link rel="icon" href="./favicon.ico"/><meta name="viewport" content="width=device-width,initial-scale=1,maximum-scale=5"/><meta name="theme-color" content="#000000"/><meta name="description" content="UI for VictoriaMetrics"/><link rel="apple-touch-icon" href="./apple-touch-icon.png"/><link rel="icon" type="image/png" sizes="32x32" href="./favicon-32x32.png"><link rel="manifest" href="./manifest.json"/><title>VM UI</title><script src="./dashboards/index.js" type="module"></script><meta name="twitter:card" content="summary_large_image"><meta name="twitter:image" content="./preview.jpg"><meta name="twitter:title" content="UI for VictoriaMetrics"><meta name="twitter:description" content="Explore and troubleshoot your VictoriaMetrics data"><meta name="twitter:site" content="@VictoriaMetrics"><meta property="og:title" content="Metric explorer for VictoriaMetrics"><meta property="og:description" content="Explore and troubleshoot your VictoriaMetrics data"><meta property="og:image" content="./preview.jpg"><meta property="og:type" content="website"><script defer="defer" src="./static/js/main.59c17910.js"></script><link href="./static/css/main.d0b400e3.css" rel="stylesheet"></head><body><noscript>You need to enable JavaScript to run this app.</noscript><div id="root"></div></body></html>

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View file

@ -79,6 +79,8 @@ The list of MetricsQL features on top of PromQL:
* [Series selectors](https://docs.victoriametrics.com/keyconcepts/#filtering) accept multiple `or` filters. For example, `{env="prod",job="a" or env="dev",job="b"}`
selects series with `{env="prod",job="a"}` or `{env="dev",job="b"}` labels.
See [these docs](https://docs.victoriametrics.com/keyconcepts/#filtering-by-multiple-or-filters) for details.
* Support for matching against multiple numeric constants via `q == (C1, ..., CN)` and `q != (C1, ..., CN)` syntax. For example, `status_code == (300, 301, 304)`
returns `status_code` metrics with one of `300`, `301` or `304` values.
* Support for `group_left(*)` and `group_right(*)` for copying all the labels from time series on the `one` side
of [many-to-one operations](https://prometheus.io/docs/prometheus/latest/querying/operators/#many-to-one-and-one-to-many-vector-matches).
The copied label names may clash with the existing label names, so MetricsQL provides an ability to add prefix to the copied metric names
@ -152,9 +154,9 @@ MetricsQL provides the following functions:
### Rollup functions
**Rollup functions** (aka range functions or window functions) calculate rollups over **raw samples**
**Rollup functions** (aka range functions or window functions) calculate rollups over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window for the [selected time series](https://docs.victoriametrics.com/keyconcepts/#filtering).
For example, `avg_over_time(temperature[24h])` calculates the average temperature over raw samples for the last 24 hours.
For example, `avg_over_time(temperature[24h])` calculates the average temperature over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) for the last 24 hours.
Additional details:
@ -184,7 +186,7 @@ The list of supported rollup functions:
#### absent_over_time
`absent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1
if the given lookbehind window `d` doesn't contain raw samples. Otherwise, it returns an empty result.
if the given lookbehind window `d` doesn't contain [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). Otherwise, it returns an empty result.
This function is supported by PromQL.
@ -193,7 +195,7 @@ See also [present_over_time](#present_over_time).
#### aggr_over_time
`aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` is a [rollup function](#rollup-functions),
which calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`.
which calculates all the listed `rollup_func*` for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`.
The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -203,7 +205,7 @@ would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time)
#### ascent_over_time
`ascent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates
ascent of raw sample values on the given lookbehind window `d`. The calculations are performed individually
ascent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d`. The calculations are performed individually
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is useful for tracking height gains in GPS tracking. Metric names are stripped from the resulting rollups.
@ -215,7 +217,7 @@ See also [descent_over_time](#descent_over_time).
#### avg_over_time
`avg_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average value
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -225,7 +227,7 @@ See also [median_over_time](#median_over_time).
#### changes
`changes(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times
the raw samples changed on the given lookbehind window `d` per each time series returned
the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Unlike `changes()` in Prometheus it takes into account the change from the last sample before the given lookbehind window `d`.
@ -240,7 +242,7 @@ See also [changes_prometheus](#changes_prometheus).
#### changes_prometheus
`changes_prometheus(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times
the raw samples changed on the given lookbehind window `d` per each time series returned
the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
It doesn't take into account the change from the last sample before the given lookbehind window `d` in the same way as Prometheus does.
@ -254,7 +256,7 @@ See also [changes](#changes).
#### count_eq_over_time
`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -264,7 +266,7 @@ See also [count_over_time](#count_over_time), [share_eq_over_time](#share_eq_ove
#### count_gt_over_time
`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -274,7 +276,7 @@ See also [count_over_time](#count_over_time) and [share_gt_over_time](#share_gt_
#### count_le_over_time
`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which don't exceed `le`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -284,7 +286,7 @@ See also [count_over_time](#count_over_time) and [share_le_over_time](#share_le_
#### count_ne_over_time
`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which aren't equal to `ne`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -294,7 +296,7 @@ See also [count_over_time](#count_over_time).
#### count_over_time
`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -305,7 +307,7 @@ See also [count_le_over_time](#count_le_over_time), [count_gt_over_time](#count_
#### count_values_over_time
`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of raw samples
`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the same value over the given lookbehind window and stores the counts in a time series with an additional `label`, which contains each initial value.
The results are calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -315,8 +317,8 @@ See also [count_eq_over_time](#count_eq_over_time), [count_values](#count_values
#### decreases_over_time
`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value decreases
over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value decreases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -324,8 +326,8 @@ See also [increases_over_time](#increases_over_time).
#### default_rollup
`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
If the lookbehind window is skipped in square brackets, then it is automatically calculated as `max(step, scrape_interval)`, where `step` is the query arg value
passed to [/api/v1/query_range](https://docs.victoriametrics.com/keyconcepts/#range-query) or [/api/v1/query](https://docs.victoriametrics.com/keyconcepts/#instant-query),
@ -375,7 +377,7 @@ See also [deriv_fast](#deriv_fast) and [ideriv](#ideriv).
#### deriv_fast
`deriv_fast(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivative
using the first and the last raw samples on the given lookbehind window `d` per each time series returned
using the first and the last [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -384,8 +386,8 @@ See also [deriv](#deriv) and [ideriv](#ideriv).
#### descent_over_time
`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of raw sample values
on the given lookbehind window `d`. The calculations are performed individually per each time series returned
`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d`. The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is useful for tracking height loss in GPS tracking.
@ -396,8 +398,8 @@ See also [ascent_over_time](#ascent_over_time).
#### distinct_over_time
`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of distinct raw sample values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of unique [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -416,15 +418,15 @@ See also [lifetime](#lifetime) and [lag](#lag).
#### first_over_time
`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first raw sample value
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
See also [last_over_time](#last_over_time) and [tfirst_over_time](#tfirst_over_time).
#### geomean_over_time
`geomean_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [geometric mean](https://en.wikipedia.org/wiki/Geometric_mean)
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -432,8 +434,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### histogram_over_time
`histogram_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates
[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over raw samples on the given lookbehind window `d`.
It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The resulting histograms are useful to pass to [histogram_quantile](#histogram_quantile) for calculating quantiles
over multiple [gauges](https://docs.victoriametrics.com/keyconcepts/#gauge).
For example, the following query calculates median temperature by country over the last 24 hours:
@ -457,7 +459,7 @@ See also [hoeffding_bound_lower](#hoeffding_bound_lower).
#### holt_winters
`holt_winters(series_selector[d], sf, tf)` is a [rollup function](#rollup-functions), which calculates Holt-Winters value
(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for raw samples
(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
over the given lookbehind window `d` using the given smoothing factor `sf` and the given trend factor `tf`.
Both `sf` and `tf` must be in the range `[0...1]`. It is expected that the [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering)
returns time series of [gauge type](https://docs.victoriametrics.com/keyconcepts/#gauge).
@ -468,7 +470,7 @@ See also [range_linear_regression](#range_linear_regression).
#### idelta
`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two raw samples
`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -479,7 +481,8 @@ See also [delta](#delta).
#### ideriv
`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based on the last two raw samples
`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based
on the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
over the given lookbehind window `d`. The derivative is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -522,8 +525,8 @@ while [increase](#increase) ignores the first value in a series if it is too big
#### increases_over_time
`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value increases
over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value increases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -531,14 +534,15 @@ See also [decreases_over_time](#decreases_over_time).
#### integrate
`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over raw samples on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### irate
`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over the last two raw samples
`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over
the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
It is expected that the `series_selector` returns time series of [counter type](https://docs.victoriametrics.com/keyconcepts/#counter).
@ -560,8 +564,8 @@ See also [lifetime](#lifetime) and [duration_over_time](#duration_over_time).
#### last_over_time
`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -579,13 +583,14 @@ See also [duration_over_time](#duration_over_time) and [lag](#lag).
#### mad_over_time
`mad_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [median absolute deviation](https://en.wikipedia.org/wiki/Median_absolute_deviation)
over raw samples on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
See also [mad](#mad), [range_mad](#range_mad) and [outlier_iqr_over_time](#outlier_iqr_over_time).
#### max_over_time
`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over raw samples
`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -594,7 +599,7 @@ See also [tmax_over_time](#tmax_over_time).
#### median_over_time
`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over raw samples
`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -602,7 +607,7 @@ See also [avg_over_time](#avg_over_time).
#### min_over_time
`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over raw samples
`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -612,15 +617,16 @@ See also [tmin_over_time](#tmin_over_time).
#### mode_over_time
`mode_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [mode](https://en.wikipedia.org/wiki/Mode_(statistics))
for raw samples on the given lookbehind window `d`. It is calculated individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that raw sample values are discrete.
for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values are discrete.
#### outlier_iqr_over_time
`outlier_iqr_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last sample on the given lookbehind window `d`
if its value is either smaller than the `q25-1.5*iqr` or bigger than `q75+1.5*iqr` where:
- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over raw samples on the lookbehind window `d`
- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over raw samples on the lookbehind window `d`.
- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`
- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`.
The `outlier_iqr_over_time()` is useful for detecting anomalies in gauge values based on the previous history of values.
For example, `outlier_iqr_over_time(memory_usage_bytes[1h])` triggers when `memory_usage_bytes` suddenly goes outside the usual value range for the last hour.
@ -630,8 +636,8 @@ See also [outliers_iqr](#outliers_iqr).
#### predict_linear
`predict_linear(series_selector[d], t)` is a [rollup function](#rollup-functions), which calculates the value `t` seconds in the future using
linear interpolation over raw samples on the given lookbehind window `d`. The predicted value is calculated individually per each time series
returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
linear interpolation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`.
The predicted value is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -639,7 +645,7 @@ See also [range_linear_regression](#range_linear_regression).
#### present_over_time
`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single raw sample
`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. Otherwise, an empty result is returned.
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -648,7 +654,7 @@ This function is supported by PromQL.
#### quantile_over_time
`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over raw samples
`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The `phi` value must be in the range `[0...1]`.
@ -659,7 +665,7 @@ See also [quantiles_over_time](#quantiles_over_time).
#### quantiles_over_time
`quantiles_over_time("phiLabel", phi1, ..., phiN, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi*`-quantiles
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The function returns individual series per each `phi*` with `{phiLabel="phi*"}` label. `phi*` values must be in the range `[0...1]`.
@ -667,7 +673,7 @@ See also [quantile_over_time](#quantile_over_time).
#### range_over_time
`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over raw samples
`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
E.g. it calculates `max_over_time(series_selector[d]) - min_over_time(series_selector[d])`.
@ -692,7 +698,7 @@ See also [irate](#irate) and [rollup_rate](#rollup_rate).
#### rate_over_sum
`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of raw samples
`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -711,7 +717,7 @@ This function is supported by PromQL.
#### rollup
`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for raw samples
`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
These values are calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -721,7 +727,8 @@ See also [label_match](#label_match).
#### rollup_candlestick
`rollup_candlestick(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `open`, `high`, `low` and `close` values (aka OHLC)
over raw samples on the given lookbehind window `d` and returns them in time series with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels.
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series
with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels.
The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for financial applications.
@ -730,7 +737,7 @@ See also [label_match](#label_match).
#### rollup_delta
`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent raw samples
`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -745,8 +752,8 @@ See also [rollup_increase](#rollup_increase).
#### rollup_deriv
`rollup_deriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivatives
for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values
for the calculated per-second derivatives and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Optional 2nd argument `"min"`, `"max"` or `"avg"` can be passed to keep only one calculation result and without adding a label.
@ -756,7 +763,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_increase
`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent raw samples
`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -768,7 +775,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_rate
`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates for adjacent raw samples
`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates
for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
@ -785,7 +793,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_scrape_interval
`rollup_scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the interval in seconds between
adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval
adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -796,7 +804,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### scrape_interval
`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds between raw samples
`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds
between [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -805,7 +814,8 @@ See also [rollup_scrape_interval](#rollup_scrape_interval).
#### share_gt_over_time
`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -817,7 +827,8 @@ See also [share_le_over_time](#share_le_over_time) and [count_gt_over_time](#cou
#### share_le_over_time
`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are smaller or equal to `le`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -830,7 +841,8 @@ See also [share_gt_over_time](#share_gt_over_time) and [count_le_over_time](#cou
#### share_eq_over_time
`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -848,7 +860,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### stddev_over_time
`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over raw samples
`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -859,7 +871,7 @@ See also [stdvar_over_time](#stdvar_over_time).
#### stdvar_over_time
`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over raw samples
`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -870,8 +882,8 @@ See also [stddev_over_time](#stddev_over_time).
#### sum_eq_over_time
`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values equal to `eq`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values equal to `eq` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -879,8 +891,8 @@ See also [sum_over_time](#sum_over_time) and [count_eq_over_time](#count_eq_over
#### sum_gt_over_time
`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values bigger than `gt`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values bigger than `gt` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -888,8 +900,8 @@ See also [sum_over_time](#sum_over_time) and [count_gt_over_time](#count_gt_over
#### sum_le_over_time
`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values smaller or equal to `le`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values smaller or equal to `le` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -897,7 +909,7 @@ See also [sum_over_time](#sum_over_time) and [count_le_over_time](#count_le_over
#### sum_over_time
`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of raw sample values
`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -906,14 +918,15 @@ This function is supported by PromQL.
#### sum2_over_time
`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for raw sample values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### timestamp
`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample
`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -924,7 +937,8 @@ See also [time](#time) and [now](#now).
#### timestamp_with_name
`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample
`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are preserved in the resulting rollups.
@ -933,7 +947,8 @@ See also [timestamp](#timestamp) and [keep_metric_names](#keep_metric_names) mod
#### tfirst_over_time
`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the first raw sample
`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -957,7 +972,8 @@ See also [tlast_change_over_time](#tlast_change_over_time).
#### tmax_over_time
`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample
`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the maximum value on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -967,7 +983,8 @@ See also [max_over_time](#max_over_time).
#### tmin_over_time
`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample
`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the minimum value on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -978,7 +995,7 @@ See also [min_over_time](#min_over_time).
#### zscore_over_time
`zscore_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns [z-score](https://en.wikipedia.org/wiki/Standard_score)
for raw samples on the given lookbehind window `d`. It is calculated independently per each time series returned
for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -2212,6 +2229,7 @@ Any [rollup function](#rollup-functions) for something other than [series select
Nested rollup functions can be implicit thanks to the [implicit query conversions](#implicit-query-conversions).
For example, `delta(sum(m))` is implicitly converted to `delta(sum(default_rollup(m))[1i:1i])`, so it becomes a subquery,
since it contains [default_rollup](#default_rollup) nested into [delta](#delta).
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.
VictoriaMetrics performs subqueries in the following way:
@ -2245,3 +2263,4 @@ VictoriaMetrics performs the following implicit conversions for incoming queries
* If something other than [series selector](https://docs.victoriametrics.com/keyconcepts/#filtering)
is passed to [rollup function](#rollup-functions), then a [subquery](#subqueries) with `1i` lookbehind window and `1i` step is automatically formed.
For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`.
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.

View file

@ -1,4 +1,4 @@
FROM golang:1.22.2 as build-web-stage
FROM golang:1.22.3 as build-web-stage
COPY build /build
WORKDIR /build

View file

@ -79,6 +79,8 @@ The list of MetricsQL features on top of PromQL:
* [Series selectors](https://docs.victoriametrics.com/keyconcepts/#filtering) accept multiple `or` filters. For example, `{env="prod",job="a" or env="dev",job="b"}`
selects series with `{env="prod",job="a"}` or `{env="dev",job="b"}` labels.
See [these docs](https://docs.victoriametrics.com/keyconcepts/#filtering-by-multiple-or-filters) for details.
* Support for matching against multiple numeric constants via `q == (C1, ..., CN)` and `q != (C1, ..., CN)` syntax. For example, `status_code == (300, 301, 304)`
returns `status_code` metrics with one of `300`, `301` or `304` values.
* Support for `group_left(*)` and `group_right(*)` for copying all the labels from time series on the `one` side
of [many-to-one operations](https://prometheus.io/docs/prometheus/latest/querying/operators/#many-to-one-and-one-to-many-vector-matches).
The copied label names may clash with the existing label names, so MetricsQL provides an ability to add prefix to the copied metric names
@ -152,9 +154,9 @@ MetricsQL provides the following functions:
### Rollup functions
**Rollup functions** (aka range functions or window functions) calculate rollups over **raw samples**
**Rollup functions** (aka range functions or window functions) calculate rollups over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window for the [selected time series](https://docs.victoriametrics.com/keyconcepts/#filtering).
For example, `avg_over_time(temperature[24h])` calculates the average temperature over raw samples for the last 24 hours.
For example, `avg_over_time(temperature[24h])` calculates the average temperature over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) for the last 24 hours.
Additional details:
@ -184,7 +186,7 @@ The list of supported rollup functions:
#### absent_over_time
`absent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1
if the given lookbehind window `d` doesn't contain raw samples. Otherwise, it returns an empty result.
if the given lookbehind window `d` doesn't contain [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). Otherwise, it returns an empty result.
This function is supported by PromQL.
@ -193,7 +195,7 @@ See also [present_over_time](#present_over_time).
#### aggr_over_time
`aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` is a [rollup function](#rollup-functions),
which calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`.
which calculates all the listed `rollup_func*` for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`.
The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -203,7 +205,7 @@ would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time)
#### ascent_over_time
`ascent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates
ascent of raw sample values on the given lookbehind window `d`. The calculations are performed individually
ascent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d`. The calculations are performed individually
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is useful for tracking height gains in GPS tracking. Metric names are stripped from the resulting rollups.
@ -215,7 +217,7 @@ See also [descent_over_time](#descent_over_time).
#### avg_over_time
`avg_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average value
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -225,7 +227,7 @@ See also [median_over_time](#median_over_time).
#### changes
`changes(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times
the raw samples changed on the given lookbehind window `d` per each time series returned
the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Unlike `changes()` in Prometheus it takes into account the change from the last sample before the given lookbehind window `d`.
@ -240,7 +242,7 @@ See also [changes_prometheus](#changes_prometheus).
#### changes_prometheus
`changes_prometheus(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times
the raw samples changed on the given lookbehind window `d` per each time series returned
the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
It doesn't take into account the change from the last sample before the given lookbehind window `d` in the same way as Prometheus does.
@ -254,7 +256,7 @@ See also [changes](#changes).
#### count_eq_over_time
`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -264,7 +266,7 @@ See also [count_over_time](#count_over_time), [share_eq_over_time](#share_eq_ove
#### count_gt_over_time
`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -274,7 +276,7 @@ See also [count_over_time](#count_over_time) and [share_gt_over_time](#share_gt_
#### count_le_over_time
`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which don't exceed `le`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -284,7 +286,7 @@ See also [count_over_time](#count_over_time) and [share_le_over_time](#share_le_
#### count_ne_over_time
`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which aren't equal to `ne`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -294,7 +296,7 @@ See also [count_over_time](#count_over_time).
#### count_over_time
`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw samples
`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -305,7 +307,7 @@ See also [count_le_over_time](#count_le_over_time), [count_gt_over_time](#count_
#### count_values_over_time
`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of raw samples
`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the same value over the given lookbehind window and stores the counts in a time series with an additional `label`, which contains each initial value.
The results are calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -315,8 +317,8 @@ See also [count_eq_over_time](#count_eq_over_time), [count_values](#count_values
#### decreases_over_time
`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value decreases
over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value decreases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -324,8 +326,8 @@ See also [increases_over_time](#increases_over_time).
#### default_rollup
`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
If the lookbehind window is skipped in square brackets, then it is automatically calculated as `max(step, scrape_interval)`, where `step` is the query arg value
passed to [/api/v1/query_range](https://docs.victoriametrics.com/keyconcepts/#range-query) or [/api/v1/query](https://docs.victoriametrics.com/keyconcepts/#instant-query),
@ -375,7 +377,7 @@ See also [deriv_fast](#deriv_fast) and [ideriv](#ideriv).
#### deriv_fast
`deriv_fast(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivative
using the first and the last raw samples on the given lookbehind window `d` per each time series returned
using the first and the last [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -384,8 +386,8 @@ See also [deriv](#deriv) and [ideriv](#ideriv).
#### descent_over_time
`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of raw sample values
on the given lookbehind window `d`. The calculations are performed individually per each time series returned
`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d`. The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is useful for tracking height loss in GPS tracking.
@ -396,8 +398,8 @@ See also [ascent_over_time](#ascent_over_time).
#### distinct_over_time
`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of distinct raw sample values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of unique [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -416,15 +418,15 @@ See also [lifetime](#lifetime) and [lag](#lag).
#### first_over_time
`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first raw sample value
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
See also [last_over_time](#last_over_time) and [tfirst_over_time](#tfirst_over_time).
#### geomean_over_time
`geomean_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [geometric mean](https://en.wikipedia.org/wiki/Geometric_mean)
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -432,8 +434,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### histogram_over_time
`histogram_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates
[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over raw samples on the given lookbehind window `d`.
It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The resulting histograms are useful to pass to [histogram_quantile](#histogram_quantile) for calculating quantiles
over multiple [gauges](https://docs.victoriametrics.com/keyconcepts/#gauge).
For example, the following query calculates median temperature by country over the last 24 hours:
@ -457,7 +459,7 @@ See also [hoeffding_bound_lower](#hoeffding_bound_lower).
#### holt_winters
`holt_winters(series_selector[d], sf, tf)` is a [rollup function](#rollup-functions), which calculates Holt-Winters value
(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for raw samples
(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
over the given lookbehind window `d` using the given smoothing factor `sf` and the given trend factor `tf`.
Both `sf` and `tf` must be in the range `[0...1]`. It is expected that the [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering)
returns time series of [gauge type](https://docs.victoriametrics.com/keyconcepts/#gauge).
@ -468,7 +470,7 @@ See also [range_linear_regression](#range_linear_regression).
#### idelta
`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two raw samples
`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -479,7 +481,8 @@ See also [delta](#delta).
#### ideriv
`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based on the last two raw samples
`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based
on the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
over the given lookbehind window `d`. The derivative is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -522,8 +525,8 @@ while [increase](#increase) ignores the first value in a series if it is too big
#### increases_over_time
`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value increases
over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value increases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -531,14 +534,15 @@ See also [decreases_over_time](#decreases_over_time).
#### integrate
`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over raw samples on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### irate
`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over the last two raw samples
`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over
the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
It is expected that the `series_selector` returns time series of [counter type](https://docs.victoriametrics.com/keyconcepts/#counter).
@ -560,8 +564,8 @@ See also [lifetime](#lifetime) and [duration_over_time](#duration_over_time).
#### last_over_time
`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d`
per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -579,13 +583,14 @@ See also [duration_over_time](#duration_over_time) and [lag](#lag).
#### mad_over_time
`mad_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [median absolute deviation](https://en.wikipedia.org/wiki/Median_absolute_deviation)
over raw samples on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
See also [mad](#mad), [range_mad](#range_mad) and [outlier_iqr_over_time](#outlier_iqr_over_time).
#### max_over_time
`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over raw samples
`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -594,7 +599,7 @@ See also [tmax_over_time](#tmax_over_time).
#### median_over_time
`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over raw samples
`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -602,7 +607,7 @@ See also [avg_over_time](#avg_over_time).
#### min_over_time
`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over raw samples
`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -612,15 +617,16 @@ See also [tmin_over_time](#tmin_over_time).
#### mode_over_time
`mode_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [mode](https://en.wikipedia.org/wiki/Mode_(statistics))
for raw samples on the given lookbehind window `d`. It is calculated individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that raw sample values are discrete.
for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values are discrete.
#### outlier_iqr_over_time
`outlier_iqr_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last sample on the given lookbehind window `d`
if its value is either smaller than the `q25-1.5*iqr` or bigger than `q75+1.5*iqr` where:
- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over raw samples on the lookbehind window `d`
- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over raw samples on the lookbehind window `d`.
- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`
- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`.
The `outlier_iqr_over_time()` is useful for detecting anomalies in gauge values based on the previous history of values.
For example, `outlier_iqr_over_time(memory_usage_bytes[1h])` triggers when `memory_usage_bytes` suddenly goes outside the usual value range for the last hour.
@ -630,8 +636,8 @@ See also [outliers_iqr](#outliers_iqr).
#### predict_linear
`predict_linear(series_selector[d], t)` is a [rollup function](#rollup-functions), which calculates the value `t` seconds in the future using
linear interpolation over raw samples on the given lookbehind window `d`. The predicted value is calculated individually per each time series
returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
linear interpolation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`.
The predicted value is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
This function is supported by PromQL.
@ -639,7 +645,7 @@ See also [range_linear_regression](#range_linear_regression).
#### present_over_time
`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single raw sample
`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. Otherwise, an empty result is returned.
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -648,7 +654,7 @@ This function is supported by PromQL.
#### quantile_over_time
`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over raw samples
`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The `phi` value must be in the range `[0...1]`.
@ -659,7 +665,7 @@ See also [quantiles_over_time](#quantiles_over_time).
#### quantiles_over_time
`quantiles_over_time("phiLabel", phi1, ..., phiN, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi*`-quantiles
over raw samples on the given lookbehind window `d` per each time series returned
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
The function returns individual series per each `phi*` with `{phiLabel="phi*"}` label. `phi*` values must be in the range `[0...1]`.
@ -667,7 +673,7 @@ See also [quantile_over_time](#quantile_over_time).
#### range_over_time
`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over raw samples
`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
E.g. it calculates `max_over_time(series_selector[d]) - min_over_time(series_selector[d])`.
@ -692,7 +698,7 @@ See also [irate](#irate) and [rollup_rate](#rollup_rate).
#### rate_over_sum
`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of raw samples
`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`. The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -711,7 +717,7 @@ This function is supported by PromQL.
#### rollup
`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for raw samples
`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
These values are calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -721,7 +727,8 @@ See also [label_match](#label_match).
#### rollup_candlestick
`rollup_candlestick(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `open`, `high`, `low` and `close` values (aka OHLC)
over raw samples on the given lookbehind window `d` and returns them in time series with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels.
over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series
with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels.
The calculations are performed individually per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for financial applications.
@ -730,7 +737,7 @@ See also [label_match](#label_match).
#### rollup_delta
`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent raw samples
`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -745,8 +752,8 @@ See also [rollup_increase](#rollup_increase).
#### rollup_deriv
`rollup_deriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivatives
for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values
for the calculated per-second derivatives and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Optional 2nd argument `"min"`, `"max"` or `"avg"` can be passed to keep only one calculation result and without adding a label.
@ -756,7 +763,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_increase
`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent raw samples
`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -768,7 +775,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_rate
`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates for adjacent raw samples
`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates
for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
@ -785,7 +793,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### rollup_scrape_interval
`rollup_scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the interval in seconds between
adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval
adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval
and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels.
The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -796,7 +804,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### scrape_interval
`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds between raw samples
`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds
between [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -805,7 +814,8 @@ See also [rollup_scrape_interval](#rollup_scrape_interval).
#### share_gt_over_time
`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -817,7 +827,8 @@ See also [share_le_over_time](#share_le_over_time) and [count_gt_over_time](#cou
#### share_le_over_time
`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are smaller or equal to `le`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -830,7 +841,8 @@ See also [share_gt_over_time](#share_gt_over_time) and [count_le_over_time](#cou
#### share_eq_over_time
`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples
`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`)
of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -848,7 +860,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k
#### stddev_over_time
`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over raw samples
`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -859,7 +871,7 @@ See also [stdvar_over_time](#stdvar_over_time).
#### stdvar_over_time
`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over raw samples
`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -870,8 +882,8 @@ See also [stddev_over_time](#stddev_over_time).
#### sum_eq_over_time
`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values equal to `eq`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values equal to `eq` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -879,8 +891,8 @@ See also [sum_over_time](#sum_over_time) and [count_eq_over_time](#count_eq_over
#### sum_gt_over_time
`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values bigger than `gt`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values bigger than `gt` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -888,8 +900,8 @@ See also [sum_over_time](#sum_over_time) and [count_gt_over_time](#count_gt_over
#### sum_le_over_time
`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values smaller or equal to `le`
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values smaller or equal to `le` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -897,7 +909,7 @@ See also [sum_over_time](#sum_over_time) and [count_le_over_time](#count_le_over
#### sum_over_time
`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of raw sample values
`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -906,14 +918,15 @@ This function is supported by PromQL.
#### sum2_over_time
`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for raw sample values
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### timestamp
`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample
`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -924,7 +937,8 @@ See also [time](#time) and [now](#now).
#### timestamp_with_name
`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample
`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are preserved in the resulting rollups.
@ -933,7 +947,8 @@ See also [timestamp](#timestamp) and [keep_metric_names](#keep_metric_names) mod
#### tfirst_over_time
`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the first raw sample
`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -957,7 +972,8 @@ See also [tlast_change_over_time](#tlast_change_over_time).
#### tmax_over_time
`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample
`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the maximum value on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -967,7 +983,8 @@ See also [max_over_time](#max_over_time).
#### tmin_over_time
`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample
`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision
for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples)
with the minimum value on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
@ -978,7 +995,7 @@ See also [min_over_time](#min_over_time).
#### zscore_over_time
`zscore_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns [z-score](https://en.wikipedia.org/wiki/Standard_score)
for raw samples on the given lookbehind window `d`. It is calculated independently per each time series returned
for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated independently per each time series returned
from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering).
Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
@ -2212,6 +2229,7 @@ Any [rollup function](#rollup-functions) for something other than [series select
Nested rollup functions can be implicit thanks to the [implicit query conversions](#implicit-query-conversions).
For example, `delta(sum(m))` is implicitly converted to `delta(sum(default_rollup(m))[1i:1i])`, so it becomes a subquery,
since it contains [default_rollup](#default_rollup) nested into [delta](#delta).
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.
VictoriaMetrics performs subqueries in the following way:
@ -2245,3 +2263,4 @@ VictoriaMetrics performs the following implicit conversions for incoming queries
* If something other than [series selector](https://docs.victoriametrics.com/keyconcepts/#filtering)
is passed to [rollup function](#rollup-functions), then a [subquery](#subqueries) with `1i` lookbehind window and `1i` step is automatically formed.
For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`.
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.

View file

@ -9,6 +9,7 @@ import { partialWarning, seriesFetchedWarning } from "./warningText";
import { AutocompleteOptions } from "../../Main/Autocomplete/Autocomplete";
import useDeviceDetect from "../../../hooks/useDeviceDetect";
import { useQueryState } from "../../../state/query/QueryStateContext";
import debounce from "lodash.debounce";
export interface QueryEditorProps {
onChange: (query: string) => void;
@ -40,9 +41,12 @@ const QueryEditor: FC<QueryEditorProps> = ({
const { isMobile } = useDeviceDetect();
const [openAutocomplete, setOpenAutocomplete] = useState(false);
const [caretPosition, setCaretPosition] = useState([0, 0]);
const [caretPosition, setCaretPosition] = useState<[number, number]>([0, 0]);
const autocompleteAnchorEl = useRef<HTMLInputElement>(null);
const [showAutocomplete, setShowAutocomplete] = useState(autocomplete);
const debouncedSetShowAutocomplete = useRef(debounce(setShowAutocomplete, 500)).current;
const warning = [
{
show: stats?.seriesFetched === "0" && !stats.resultLength,
@ -58,8 +62,9 @@ const QueryEditor: FC<QueryEditorProps> = ({
label = `${label} (${stats.executionTimeMsec || 0}ms)`;
}
const handleSelect = (val: string) => {
const handleSelect = (val: string, caretPosition: number) => {
onChange(val);
setCaretPosition([caretPosition, caretPosition]);
};
const handleKeyDown = (e: KeyboardEvent) => {
@ -100,14 +105,19 @@ const QueryEditor: FC<QueryEditorProps> = ({
setOpenAutocomplete(!!val.length);
};
const handleChangeCaret = (val: number[]) => {
setCaretPosition(val);
const handleChangeCaret = (val: [number, number]) => {
setCaretPosition(prev => prev[0] === val[0] && prev[1] === val[1] ? prev : val);
};
useEffect(() => {
setOpenAutocomplete(autocomplete);
}, [autocompleteQuick]);
useEffect(() => {
setShowAutocomplete(false);
debouncedSetShowAutocomplete(true);
}, [caretPosition]);
return (
<div
className="vm-query-editor"
@ -125,12 +135,14 @@ const QueryEditor: FC<QueryEditorProps> = ({
onChangeCaret={handleChangeCaret}
disabled={disabled}
inputmode={"search"}
caretPosition={caretPosition}
/>
{autocomplete && (
{showAutocomplete && autocomplete && (
<QueryEditorAutocomplete
value={value}
anchorEl={autocompleteAnchorEl}
caretPosition={caretPosition}
hasHelperText={Boolean(warning || error)}
onSelect={handleSelect}
onFoundOptions={handleChangeFoundOptions}
/>

View file

@ -1,7 +1,6 @@
import React, { FC, Ref, useState, useEffect, useMemo } from "preact/compat";
import React, { FC, Ref, useState, useEffect, useMemo, useCallback } from "preact/compat";
import Autocomplete, { AutocompleteOptions } from "../../Main/Autocomplete/Autocomplete";
import { useFetchQueryOptions } from "../../../hooks/useFetchQueryOptions";
import { getTextWidth } from "../../../utils/uplot";
import { escapeRegexp, hasUnclosedQuotes } from "../../../utils/regexp";
import useGetMetricsQL from "../../../hooks/useGetMetricsQL";
import { QueryContextType } from "../../../types";
@ -10,8 +9,9 @@ import { AUTOCOMPLETE_LIMITS } from "../../../constants/queryAutocomplete";
interface QueryEditorAutocompleteProps {
value: string;
anchorEl: Ref<HTMLInputElement>;
caretPosition: number[];
onSelect: (val: string) => void;
caretPosition: [number, number]; // [start, end]
hasHelperText: boolean;
onSelect: (val: string, caretPosition: number) => void;
onFoundOptions: (val: AutocompleteOptions[]) => void;
}
@ -19,16 +19,24 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
value,
anchorEl,
caretPosition,
hasHelperText,
onSelect,
onFoundOptions
}) => {
const [leftOffset, setLeftOffset] = useState(0);
const [offsetPos, setOffsetPos] = useState({ top: 0, left: 0 });
const metricsqlFunctions = useGetMetricsQL();
const values = useMemo(() => {
if (caretPosition[0] !== caretPosition[1]) return { beforeCursor: value, afterCursor: "" };
const beforeCursor = value.substring(0, caretPosition[0]);
const afterCursor = value.substring(caretPosition[1]);
return { beforeCursor, afterCursor };
}, [value, caretPosition]);
const exprLastPart = useMemo(() => {
const parts = value.split("}");
const parts = values.beforeCursor.split("}");
return parts[parts.length - 1];
}, [value]);
}, [values]);
const metric = useMemo(() => {
const regexp = /\b[^{}(),\s]+(?={|$)/g;
@ -43,7 +51,7 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
}, [exprLastPart]);
const shouldSuppressAutoSuggestion = (value: string) => {
const pattern = /([(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/;
const pattern = /([{(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/;
const parts = value.split(/\s+/);
const partsCount = parts.length;
const lastPart = parts[partsCount - 1];
@ -55,7 +63,7 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
};
const context = useMemo(() => {
if (!value || value.endsWith("}") || shouldSuppressAutoSuggestion(value)) {
if (!values.beforeCursor || values.beforeCursor.endsWith("}") || shouldSuppressAutoSuggestion(values.beforeCursor)) {
return QueryContextType.empty;
}
@ -63,19 +71,19 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
const labelValueRegexp = new RegExp(`(${escapeRegexp(metric)})?{?.+${escapeRegexp(label)}(=|!=|=~|!~)"?([^"]*)$`, "g");
switch (true) {
case labelValueRegexp.test(value):
case labelValueRegexp.test(values.beforeCursor):
return QueryContextType.labelValue;
case labelRegexp.test(value):
case labelRegexp.test(values.beforeCursor):
return QueryContextType.label;
default:
return QueryContextType.metricsql;
}
}, [value, metric, label]);
}, [values, metric, label]);
const valueByContext = useMemo(() => {
const wordMatch = value.match(/([\w_\-.:/]+(?![},]))$/);
const wordMatch = values.beforeCursor.match(/([\w_\-.:/]+(?![},]))$/);
return wordMatch ? wordMatch[0] : "";
}, [value]);
}, [values.beforeCursor]);
const { metrics, labels, labelValues, loading } = useFetchQueryOptions({
valueByContext,
@ -97,8 +105,10 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
}
}, [context, metrics, labels, labelValues]);
const handleSelect = (insert: string) => {
const handleSelect = useCallback((insert: string) => {
// Find the start and end of valueByContext in the query string
const value = values.beforeCursor;
let valueAfterCursor = values.afterCursor;
const startIndexOfValueByContext = value.lastIndexOf(valueByContext, caretPosition[0]);
const endIndexOfValueByContext = startIndexOfValueByContext + valueByContext.length;
@ -110,26 +120,59 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
if (context === QueryContextType.labelValue) {
const quote = "\"";
const needsQuote = /(?:=|!=|=~|!~)$/.test(beforeValueByContext);
valueAfterCursor = valueAfterCursor.replace(/^[^\s"|},]*/, "");
insert = `${needsQuote ? quote : ""}${insert}`;
}
if (context === QueryContextType.label) {
valueAfterCursor = valueAfterCursor.replace(/^[^\s=!,{}()"|+\-/*^]*/, "");
}
if (context === QueryContextType.metricsql) {
valueAfterCursor = valueAfterCursor.replace(/^[^\s[\]{}()"|+\-/*^]*/, "");
}
// Assemble the new value with the inserted text
const newVal = `${beforeValueByContext}${insert}${afterValueByContext}`;
onSelect(newVal);
};
const newVal = `${beforeValueByContext}${insert}${afterValueByContext}${valueAfterCursor}`;
onSelect(newVal, beforeValueByContext.length + insert.length);
}, [values]);
useEffect(() => {
if (!anchorEl.current) {
setLeftOffset(0);
setOffsetPos({ top: 0, left: 0 });
return;
}
const style = window.getComputedStyle(anchorEl.current);
const element = anchorEl.current.querySelector("textarea") || anchorEl.current;
const style = window.getComputedStyle(element);
const fontSize = `${style.getPropertyValue("font-size")}`;
const fontFamily = `${style.getPropertyValue("font-family")}`;
const offset = getTextWidth(value, `${fontSize} ${fontFamily}`);
setLeftOffset(offset);
}, [anchorEl, caretPosition]);
const lineHeight = parseInt(`${style.getPropertyValue("line-height")}`);
const span = document.createElement("div");
span.style.font = `${fontSize} ${fontFamily}`;
span.style.padding = style.getPropertyValue("padding");
span.style.lineHeight = `${lineHeight}px`;
span.style.width = `${element.offsetWidth}px`;
span.style.maxWidth = `${element.offsetWidth}px`;
span.style.whiteSpace = style.getPropertyValue("white-space");
span.style.overflowWrap = style.getPropertyValue("overflow-wrap");
const marker = document.createElement("span");
span.appendChild(document.createTextNode(values.beforeCursor));
span.appendChild(marker);
span.appendChild(document.createTextNode(values.afterCursor));
document.body.appendChild(span);
const spanRect = span.getBoundingClientRect();
const markerRect = marker.getBoundingClientRect();
const leftOffset = markerRect.left - spanRect.left;
const topOffset = markerRect.bottom - spanRect.bottom - (hasHelperText ? lineHeight : 0);
setOffsetPos({ top: topOffset, left: leftOffset });
span.remove();
marker.remove();
}, [anchorEl, caretPosition, hasHelperText]);
return (
<>
@ -140,7 +183,7 @@ const QueryEditorAutocomplete: FC<QueryEditorAutocompleteProps> = ({
options={options}
anchor={anchorEl}
minLength={0}
offset={{ top: 0, left: leftOffset }}
offset={offsetPos}
onSelect={handleSelect}
onFoundOptions={onFoundOptions}
maxDisplayResults={{

View file

@ -2,4 +2,13 @@
.vm-query-editor {
position: relative;
.marker-detection {
position: absolute;
top: 0;
left: 0;
pointer-events: none;
z-index: -9999;
visibility: hidden;
}
}

View file

@ -120,7 +120,7 @@ const Autocomplete: FC<AutocompleteProps> = ({
const handleKeyDown = useCallback((e: KeyboardEvent) => {
const { key, ctrlKey, metaKey, shiftKey } = e;
const modifiers = ctrlKey || metaKey || shiftKey;
const hasOptions = foundOptions.length;
const hasOptions = foundOptions.length && !hideFoundedOptions;
if (key === "ArrowUp" && !modifiers && hasOptions) {
e.preventDefault();
@ -148,7 +148,7 @@ const Autocomplete: FC<AutocompleteProps> = ({
if (key === "Escape") {
handleCloseAutocomplete();
}
}, [focusOption, foundOptions, handleCloseAutocomplete, onSelect, selected]);
}, [focusOption, foundOptions, hideFoundedOptions, handleCloseAutocomplete, onSelect, selected]);
useEffect(() => {
setOpenAutocomplete(value.length >= minLength);

View file

@ -1,6 +1,7 @@
import React, {
FC,
useEffect,
useState,
useRef,
useMemo,
FormEvent,
@ -28,12 +29,13 @@ interface TextFieldProps {
autofocus?: boolean
helperText?: string
inputmode?: "search" | "text" | "email" | "tel" | "url" | "none" | "numeric" | "decimal"
caretPosition?: [number, number]
onChange?: (value: string) => void
onEnter?: () => void
onKeyDown?: (e: KeyboardEvent) => void
onFocus?: () => void
onBlur?: () => void
onChangeCaret?: (position: number[]) => void
onChangeCaret?: (position: [number, number]) => void
}
const TextField: FC<TextFieldProps> = ({
@ -49,6 +51,7 @@ const TextField: FC<TextFieldProps> = ({
disabled = false,
autofocus = false,
inputmode = "text",
caretPosition,
onChange,
onEnter,
onKeyDown,
@ -62,6 +65,7 @@ const TextField: FC<TextFieldProps> = ({
const inputRef = useRef<HTMLInputElement>(null);
const textareaRef = useRef<HTMLTextAreaElement>(null);
const fieldRef = useMemo(() => type === "textarea" ? textareaRef : inputRef, [type]);
const [selectionPos, setSelectionPos] = useState<[start: number, end: number]>([0, 0]);
const inputClasses = classNames({
"vm-text-field__input": true,
@ -74,7 +78,7 @@ const TextField: FC<TextFieldProps> = ({
const updateCaretPosition = (target: HTMLInputElement | HTMLTextAreaElement) => {
const { selectionStart, selectionEnd } = target;
onChangeCaret && onChangeCaret([selectionStart || 0, selectionEnd || 0]);
setSelectionPos([selectionStart || 0, selectionEnd || 0]);
};
const handleMouseUp = (e: MouseEvent<HTMLInputElement | HTMLTextAreaElement>) => {
@ -102,11 +106,6 @@ const TextField: FC<TextFieldProps> = ({
updateCaretPosition(e.currentTarget);
};
useEffect(() => {
if (!autofocus || isMobile) return;
fieldRef?.current?.focus && fieldRef.current.focus();
}, [fieldRef, autofocus]);
const handleFocus = () => {
onFocus && onFocus();
};
@ -115,6 +114,31 @@ const TextField: FC<TextFieldProps> = ({
onBlur && onBlur();
};
const setSelectionRange = (range: [number, number]) => {
try {
fieldRef.current && fieldRef.current.setSelectionRange(range[0], range[1]);
} catch (e) {
return e;
}
};
useEffect(() => {
if (!autofocus || isMobile) return;
fieldRef?.current?.focus && fieldRef.current.focus();
}, [fieldRef, autofocus]);
useEffect(() => {
onChangeCaret && onChangeCaret(selectionPos);
}, [selectionPos]);
useEffect(() => {
setSelectionRange(selectionPos);
}, [value]);
useEffect(() => {
caretPosition && setSelectionRange(caretPosition);
}, [caretPosition]);
return <label
className={classNames({
"vm-text-field": true,

View file

@ -9,7 +9,7 @@
&_textarea:after {
content: attr(data-replicated-value) " ";
white-space: pre-wrap;
word-wrap: break-word;
overflow-wrap: break-word;
visibility: hidden;
}

View file

@ -1,5 +1,4 @@
import { useCallback, useState } from "react";
import useIsomorphicLayoutEffect from "./useIsomorphicLayoutEffect";
import { useCallback, useEffect, useState } from "react";
import useEventListener from "./useEventListener";
export interface ElementSize {
@ -28,7 +27,7 @@ const useElementSize = <T extends HTMLElement = HTMLDivElement>(): [(node: T | n
useEventListener("resize", handleSize);
useIsomorphicLayoutEffect(handleSize, [ref?.offsetHeight, ref?.offsetWidth]);
useEffect(handleSize, [ref?.offsetHeight, ref?.offsetWidth]);
return [setRef, size];
};

View file

@ -1,5 +1,4 @@
import { RefObject, useEffect, useRef } from "react";
import useIsomorphicLayoutEffect from "./useIsomorphicLayoutEffect";
// MediaQueryList Event based useEventListener interface
function useEventListener<K extends keyof MediaQueryListEventMap>(
@ -56,7 +55,7 @@ function useEventListener<
// Create a ref that stores handler
const savedHandler = useRef(handler);
useIsomorphicLayoutEffect(() => {
useEffect(() => {
savedHandler.current = handler;
}, [handler]);

View file

@ -153,9 +153,17 @@ export const useFetchQuery = ({
setTraces(tempTraces);
setIsHistogram(prev => totalLength ? isHistogramResult : prev);
} catch (e) {
if (e instanceof Error && e.name !== "AbortError") {
setError(`${e.name}: ${e.message}`);
const error = e as Error;
if (error.name === "AbortError") {
// Aborts are expected, don't show an error for them.
return;
}
const helperText = "Please check your serverURL settings and confirm server availability.";
let text = `Error executing query: ${error.message}. ${helperText}`;
if (error.message === "Unexpected end of JSON input") {
text += "\nAdditionally, this error can occur if the server response is too large to process. Apply more specific filters to reduce the data volume.";
}
setError(text);
}
setIsLoading(false);
};

View file

@ -96,6 +96,7 @@ export const useFetchQueryOptions = ({ valueByContext, metric, label, context }:
const cachedData = autocompleteCache.get(key);
if (cachedData) {
setter(processData(cachedData, type));
setLoading(false);
return;
}
const response = await fetch(`${serverUrl}/api/v1/${urlSuffix}?${params}`, { signal });
@ -104,13 +105,13 @@ export const useFetchQueryOptions = ({ valueByContext, metric, label, context }:
setter(processData(data, type));
queryDispatch({ type: "SET_AUTOCOMPLETE_CACHE", payload: { key, value: data } });
}
setLoading(false);
} catch (e) {
if (e instanceof Error && e.name !== "AbortError") {
queryDispatch({ type: "SET_AUTOCOMPLETE_CACHE", payload: { key, value: [] } });
setLoading(false);
console.error(e);
}
} finally {
setLoading(false);
}
};

View file

@ -1,5 +0,0 @@
import { useEffect, useLayoutEffect } from "react";
const useIsomorphicLayoutEffect = typeof window !== "undefined" ? useLayoutEffect : useEffect;
export default useIsomorphicLayoutEffect;

View file

@ -1,6 +1,6 @@
import { useState } from "react";
import useIsomorphicLayoutEffect from "./useIsomorphicLayoutEffect";
import useEventListener from "./useEventListener";
import { useEffect } from "preact/compat";
interface WindowSize {
width: number
@ -23,7 +23,7 @@ const useWindowSize = (): WindowSize => {
useEventListener("resize", handleSize);
// Set size at the first client-side load
useIsomorphicLayoutEffect(handleSize, []);
useEffect(handleSize, []);
return windowSize;
};

View file

@ -12,7 +12,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "9.0.4"
"version": "10.4.0"
},
{
"type": "datasource",
@ -124,9 +124,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -198,9 +200,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -260,9 +264,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -323,9 +329,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -399,9 +407,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -471,9 +481,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -546,9 +558,11 @@
"fields": "",
"values": false
},
"textMode": "auto"
"showPercentChange": false,
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -577,7 +591,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -630,7 +646,9 @@
},
"id": 22,
"options": {
"cellHeight": "sm",
"footer": {
"countRows": false,
"fields": "",
"reducer": [
"sum"
@ -645,7 +663,7 @@
}
]
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -700,7 +718,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -753,7 +773,9 @@
},
"id": 21,
"options": {
"cellHeight": "sm",
"footer": {
"countRows": false,
"fields": "",
"reducer": [
"sum"
@ -768,7 +790,7 @@
}
]
},
"pluginVersion": "9.0.4",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -885,7 +907,7 @@
"min",
"mean"
],
"displayMode": "table",
"displayMode": "list",
"placement": "bottom",
"showLegend": false
},
@ -1106,7 +1128,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1251,7 +1275,8 @@
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom"
"placement": "bottom",
"showLegend": true
},
"orientation": "auto",
"showValue": "auto",
@ -1343,7 +1368,7 @@
"max",
"mean"
],
"displayMode": "table",
"displayMode": "list",
"placement": "right",
"showLegend": false
},
@ -1436,7 +1461,8 @@
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom"
"placement": "bottom",
"showLegend": true
},
"orientation": "auto",
"showValue": "auto",
@ -1657,8 +1683,7 @@
}
],
"refresh": "1m",
"schemaVersion": 36,
"style": "dark",
"schemaVersion": 39,
"tags": [],
"templating": {
"list": [

View file

@ -1,4 +1,38 @@
{
"__inputs": [],
"__elements": {},
"__requires": [
{
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.4.0"
},
{
"type": "panel",
"id": "piechart",
"name": "Pie chart",
"version": ""
},
{
"type": "datasource",
"id": "prometheus",
"name": "Prometheus",
"version": "1.0.0"
},
{
"type": "panel",
"id": "table",
"name": "Table",
"version": ""
},
{
"type": "panel",
"id": "timeseries",
"name": "Time series",
"version": ""
}
],
"annotations": {
"list": [
{
@ -26,7 +60,7 @@
"fiscalYearStartMonth": 0,
"gnetId": 16399,
"graphTooltip": 1,
"id": 80,
"id": null,
"links": [],
"liveNow": false,
"panels": [
@ -55,6 +89,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -158,6 +193,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -214,7 +250,6 @@
"y": 1
},
"id": 4,
"links": [],
"options": {
"legend": {
"calcs": [
@ -264,6 +299,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -376,6 +412,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -432,7 +469,6 @@
"y": 9
},
"id": 31,
"links": [],
"options": {
"legend": {
"calcs": [
@ -482,6 +518,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -538,7 +575,6 @@
"y": 17
},
"id": 10,
"links": [],
"options": {
"legend": {
"calcs": [
@ -587,6 +623,7 @@
"mode": "palette-classic"
},
"custom": {
"axisBorderShow": false,
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
@ -1114,7 +1151,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1198,7 +1237,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1230,7 +1269,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1314,7 +1355,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1346,7 +1387,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1430,7 +1473,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1462,7 +1505,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1546,7 +1591,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1578,7 +1623,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1662,7 +1709,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1694,7 +1741,9 @@
},
"custom": {
"align": "auto",
"displayMode": "auto",
"cellOptions": {
"type": "auto"
},
"inspect": false
},
"mappings": [],
@ -1778,7 +1827,7 @@
}
]
},
"pluginVersion": "10.1.5",
"pluginVersion": "10.4.0",
"targets": [
{
"datasource": {
@ -1800,8 +1849,7 @@
}
],
"refresh": false,
"schemaVersion": 37,
"style": "dark",
"schemaVersion": 39,
"tags": [
"VictoriaMetrics",
"monitoring"
@ -1828,15 +1876,7 @@
},
{
"allValue": ".*",
"current": {
"selected": true,
"text": [
"All"
],
"value": [
"$__all"
]
},
"current": {},
"datasource": {
"type": "prometheus",
"uid": "$ds"
@ -1862,15 +1902,7 @@
},
{
"allValue": ".*",
"current": {
"selected": true,
"text": [
"All"
],
"value": [
"$__all"
]
},
"current": {},
"datasource": {
"type": "prometheus",
"uid": "$ds"
@ -1915,6 +1947,6 @@
"timezone": "",
"title": "VictoriaMetrics Cluster Per Tenant Statistic",
"uid": "IZFqd3lMz",
"version": 15,
"version": 1,
"weekStart": ""
}

File diff suppressed because it is too large Load diff

View file

@ -6,7 +6,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.3.1"
"version": "10.4.2"
},
{
"type": "datasource",
@ -115,8 +115,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -127,7 +126,6 @@
"y": 1
},
"id": 10,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -146,7 +144,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -188,8 +186,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -200,7 +197,6 @@
"y": 1
},
"id": 22,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -219,7 +215,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -261,8 +257,7 @@
}
]
},
"unit": "none",
"unitScale": true
"unit": "none"
},
"overrides": []
},
@ -273,7 +268,6 @@
"y": 1
},
"id": 25,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -292,7 +286,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -338,8 +332,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -350,7 +343,6 @@
"y": 1
},
"id": 30,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -369,7 +361,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -414,8 +406,7 @@
}
]
},
"unit": "s",
"unitScale": true
"unit": "s"
},
"overrides": []
},
@ -443,7 +434,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -483,8 +474,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -495,7 +485,6 @@
"y": 3
},
"id": 24,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -514,7 +503,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -556,8 +545,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -568,7 +556,6 @@
"y": 3
},
"id": 36,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -587,7 +574,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -629,8 +616,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -641,7 +627,6 @@
"y": 3
},
"id": 34,
"links": [],
"maxDataPoints": 100,
"options": {
"colorMode": "value",
@ -660,7 +645,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -751,8 +736,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -763,7 +747,6 @@
"y": 6
},
"id": 2,
"links": [],
"options": {
"legend": {
"calcs": [
@ -860,8 +843,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -872,7 +854,6 @@
"y": 6
},
"id": 14,
"links": [],
"options": {
"legend": {
"calcs": [
@ -970,8 +951,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -982,7 +962,6 @@
"y": 14
},
"id": 6,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1080,8 +1059,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1092,7 +1070,6 @@
"y": 14
},
"id": 26,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1203,8 +1180,7 @@
}
]
},
"unit": "percentunit",
"unitScale": true
"unit": "percentunit"
},
"overrides": []
},
@ -1215,7 +1191,6 @@
"y": 23
},
"id": 38,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1312,8 +1287,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -1324,7 +1298,6 @@
"y": 23
},
"id": 40,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1471,8 +1444,7 @@
}
]
},
"unit": "percentunit",
"unitScale": true
"unit": "percentunit"
},
"overrides": []
},
@ -1483,7 +1455,6 @@
"y": 31
},
"id": 42,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1579,8 +1550,7 @@
}
]
},
"unit": "percentunit",
"unitScale": true
"unit": "percentunit"
},
"overrides": []
},
@ -1591,7 +1561,6 @@
"y": 31
},
"id": 44,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1691,8 +1660,7 @@
}
]
},
"unit": "percentunit",
"unitScale": true
"unit": "percentunit"
},
"overrides": [
{
@ -1719,7 +1687,6 @@
"y": 39
},
"id": 46,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1818,8 +1785,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": [
{
@ -1846,7 +1812,6 @@
"y": 39
},
"id": 48,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1958,8 +1923,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1970,7 +1934,6 @@
"y": 47
},
"id": 50,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2066,8 +2029,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": [
{
@ -2091,7 +2053,6 @@
"y": 47
},
"id": 52,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2201,8 +2162,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -2213,7 +2173,6 @@
"y": 55
},
"id": 54,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2309,8 +2268,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": [
{
@ -2334,7 +2292,6 @@
"y": 55
},
"id": 56,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2448,8 +2405,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -2460,7 +2416,6 @@
"y": 63
},
"id": 58,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2558,8 +2513,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -2570,7 +2524,6 @@
"y": 63
},
"id": 60,
"links": [],
"options": {
"legend": {
"calcs": [
@ -2617,7 +2570,7 @@
"list": [
{
"current": {
"selected": true,
"selected": false,
"text": "VictoriaMetrics",
"value": "P4169E866C3094E38"
},

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

View file

@ -7,7 +7,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.3.1"
"version": "10.4.2"
},
{
"type": "datasource",
@ -151,8 +151,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -186,7 +185,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -221,8 +220,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -256,7 +254,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -292,8 +290,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -304,7 +301,6 @@
"y": 1
},
"id": 134,
"links": [],
"options": {
"colorMode": "value",
"graphMode": "none",
@ -322,7 +318,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -357,8 +353,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -417,7 +412,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -471,8 +466,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -507,7 +501,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -545,8 +539,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -574,7 +567,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -621,8 +614,7 @@
"value": 80
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -670,7 +662,7 @@
},
"showHeader": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -747,8 +739,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -883,8 +874,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": [
{
@ -1022,8 +1012,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -1136,8 +1125,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1148,7 +1136,6 @@
"y": 18
},
"id": 107,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1248,8 +1235,7 @@
}
]
},
"unit": "decbytes",
"unitScale": true
"unit": "decbytes"
},
"overrides": []
},
@ -1351,8 +1337,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1372,7 +1357,6 @@
"y": 26
},
"id": 15,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1457,8 +1441,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1637,8 +1620,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1754,8 +1736,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1868,8 +1849,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2006,8 +1986,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2139,8 +2118,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2247,8 +2225,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2356,8 +2333,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2464,8 +2440,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3583,8 +3558,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3691,8 +3665,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3797,8 +3770,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3905,8 +3877,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4012,8 +3983,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4118,8 +4088,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4223,8 +4192,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4328,8 +4296,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",

View file

@ -7,7 +7,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.3.1"
"version": "10.4.2"
},
{
"type": "datasource",
@ -179,8 +179,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -208,7 +207,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -242,8 +241,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -271,7 +269,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -305,8 +303,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -334,7 +331,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -372,8 +369,7 @@
"value": 1
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -401,7 +397,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -439,8 +435,7 @@
"value": 1
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -468,7 +463,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -518,8 +513,7 @@
"value": 80
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -567,7 +561,7 @@
},
"showHeader": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -644,8 +638,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -775,8 +768,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -881,8 +873,7 @@
}
]
},
"unit": "s",
"unitScale": true
"unit": "s"
},
"overrides": []
},
@ -987,8 +978,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1091,8 +1081,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1430,8 +1419,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1545,8 +1533,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1660,8 +1647,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1777,8 +1763,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1911,8 +1896,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2020,8 +2004,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2129,8 +2112,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3271,7 +3253,7 @@
"list": [
{
"current": {
"selected": true,
"selected": false,
"text": "VictoriaMetrics",
"value": "P4169E866C3094E38"
},

View file

@ -6,7 +6,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.3.1"
"version": "10.4.2"
},
{
"type": "datasource",
@ -150,8 +150,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -185,7 +184,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -220,8 +219,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -255,7 +253,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -291,8 +289,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -303,7 +300,6 @@
"y": 1
},
"id": 134,
"links": [],
"options": {
"colorMode": "value",
"graphMode": "none",
@ -321,7 +317,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -356,8 +352,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -416,7 +411,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -470,8 +465,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -506,7 +500,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -544,8 +538,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -573,7 +566,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -620,8 +613,7 @@
"value": 80
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -669,7 +661,7 @@
},
"showHeader": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -746,8 +738,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -882,8 +873,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": [
{
@ -1021,8 +1011,7 @@
}
]
},
"unit": "bytes",
"unitScale": true
"unit": "bytes"
},
"overrides": []
},
@ -1135,8 +1124,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1147,7 +1135,6 @@
"y": 18
},
"id": 107,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1247,8 +1234,7 @@
}
]
},
"unit": "decbytes",
"unitScale": true
"unit": "decbytes"
},
"overrides": []
},
@ -1350,8 +1336,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1371,7 +1356,6 @@
"y": 26
},
"id": 15,
"links": [],
"options": {
"legend": {
"calcs": [
@ -1456,8 +1440,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1636,8 +1619,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1753,8 +1735,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1867,8 +1848,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2005,8 +1985,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2138,8 +2117,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2246,8 +2224,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2355,8 +2332,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2463,8 +2439,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3582,8 +3557,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3690,8 +3664,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3796,8 +3769,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3904,8 +3876,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4011,8 +3982,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4117,8 +4087,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4222,8 +4191,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -4327,8 +4295,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",

View file

@ -6,7 +6,7 @@
"type": "grafana",
"id": "grafana",
"name": "Grafana",
"version": "10.3.1"
"version": "10.4.2"
},
{
"type": "datasource",
@ -178,8 +178,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -207,7 +206,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -241,8 +240,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -270,7 +268,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -304,8 +302,7 @@
"value": null
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -333,7 +330,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -371,8 +368,7 @@
"value": 1
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -400,7 +396,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -438,8 +434,7 @@
"value": 1
}
]
},
"unitScale": true
}
},
"overrides": []
},
@ -467,7 +462,7 @@
"textMode": "auto",
"wideLayout": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -517,8 +512,7 @@
"value": 80
}
]
},
"unitScale": true
}
},
"overrides": [
{
@ -566,7 +560,7 @@
},
"showHeader": true
},
"pluginVersion": "10.3.1",
"pluginVersion": "10.4.2",
"targets": [
{
"datasource": {
@ -643,8 +637,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -774,8 +767,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -880,8 +872,7 @@
}
]
},
"unit": "s",
"unitScale": true
"unit": "s"
},
"overrides": []
},
@ -986,8 +977,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1090,8 +1080,7 @@
}
]
},
"unit": "short",
"unitScale": true
"unit": "short"
},
"overrides": []
},
@ -1429,8 +1418,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1544,8 +1532,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1659,8 +1646,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1776,8 +1762,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -1910,8 +1895,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2019,8 +2003,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -2128,8 +2111,7 @@
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
"color": "green"
},
{
"color": "red",
@ -3270,7 +3252,7 @@
"list": [
{
"current": {
"selected": true,
"selected": false,
"text": "VictoriaMetrics",
"value": "P4169E866C3094E38"
},

View file

@ -5,7 +5,7 @@ DOCKER_NAMESPACE ?= victoriametrics
ROOT_IMAGE ?= alpine:3.19.1
CERTS_IMAGE := alpine:3.19.1
GO_BUILDER_IMAGE := golang:1.22.2-alpine
GO_BUILDER_IMAGE := golang:1.22.3-alpine
BUILDER_IMAGE := local/builder:2.0.0-$(shell echo $(GO_BUILDER_IMAGE) | tr :/ __)-1
BASE_IMAGE := local/base:1.1.4-$(shell echo $(ROOT_IMAGE) | tr :/ __)-$(shell echo $(CERTS_IMAGE) | tr :/ __)
DOCKER ?= docker
@ -102,24 +102,23 @@ run-via-docker: package-via-docker
app-via-docker-goos-goarch:
APP_SUFFIX='-$(GOOS)-$(GOARCH)' \
DOCKER_OPTS='--env CGO_ENABLED=$(CGO_ENABLED) --env GOOS=$(GOOS) --env GOARCH=$(GOARCH)' \
DOCKER_OPTS='--env CGO_ENABLED=$(CGO_ENABLED) --env GOOS=$(GOOS) --env GOARCH=$(GOARCH) $(foreach v,$(EXTRA_ENVS),--env $(v))' \
$(MAKE) app-via-docker
app-via-docker-pure:
APP_SUFFIX='-pure' DOCKER_OPTS='--env CGO_ENABLED=0' $(MAKE) app-via-docker
app-via-docker-linux-amd64:
EXTRA_ENVS='CC=/opt/cross-builder/x86_64-linux-musl-cross/bin/x86_64-linux-musl-gcc' \
CGO_ENABLED=1 GOOS=linux GOARCH=amd64 $(MAKE) app-via-docker-goos-goarch
app-via-docker-linux-arm:
APP_SUFFIX='-linux-arm' \
DOCKER_OPTS='--env CGO_ENABLED=0 --env GOOS=linux --env GOARCH=arm --env GOARM=5' \
$(MAKE) app-via-docker
EXTRA_ENVS='GOARM=5' \
CGO_ENABLED=0 GOOS=linux GOARCH=arm $(MAKE) app-via-docker-goos-goarch
app-via-docker-linux-arm64:
APP_SUFFIX='-linux-arm64' \
DOCKER_OPTS='--env CGO_ENABLED=1 --env GOOS=linux --env GOARCH=arm64 --env CC=/opt/cross-builder/aarch64-linux-musl-cross/bin/aarch64-linux-musl-gcc' \
$(MAKE) app-via-docker
EXTRA_ENVS='CC=/opt/cross-builder/aarch64-linux-musl-cross/bin/aarch64-linux-musl-gcc' \
CGO_ENABLED=1 GOOS=linux GOARCH=arm64 $(MAKE) app-via-docker-goos-goarch
app-via-docker-linux-ppc64le:
CGO_ENABLED=0 GOOS=linux GOARCH=ppc64le $(MAKE) app-via-docker-goos-goarch
@ -146,37 +145,28 @@ app-via-docker-windows-amd64:
package-via-docker-goarch:
APP_SUFFIX='-$(GOARCH)' \
DOCKER_OPTS='--env CGO_ENABLED=$(CGO_ENABLED) --env GOOS=linux --env GOARCH=$(GOARCH)' \
DOCKER_OPTS='--env CGO_ENABLED=$(CGO_ENABLED) --env GOOS=linux --env GOARCH=$(GOARCH) $(foreach v,$(EXTRA_ENVS),--env $(v))' \
$(MAKE) package-via-docker
package-via-docker-goarch-arm64:
APP_SUFFIX='-arm64' \
DOCKER_OPTS='--env CGO_ENABLED=1 --env GOOS=linux --env GOARCH=arm64 --env CC=/opt/cross-builder/aarch64-linux-musl-cross/bin/aarch64-linux-musl-gcc' \
$(MAKE) package-via-docker
package-via-docker-goarch-cgo:
CGO_ENABLED=1 $(MAKE) package-via-docker-goarch
package-via-docker-goarch-nocgo:
CGO_ENABLED=0 $(MAKE) package-via-docker-goarch
package-via-docker-pure:
APP_SUFFIX='-pure' DOCKER_OPTS='--env CGO_ENABLED=0' $(MAKE) package-via-docker
package-via-docker-amd64:
GOARCH=amd64 $(MAKE) package-via-docker-goarch-cgo
EXTRA_ENVS='CC=/opt/cross-builder/x86_64-linux-musl-cross/bin/x86_64-linux-musl-gcc' \
GOARCH=amd64 GOARCH=amd64 $(MAKE) package-via-docker-goarch
package-via-docker-arm:
GOARCH=arm $(MAKE) package-via-docker-goarch-nocgo
GOARCH=arm CGO_ENABLED=0 $(MAKE) package-via-docker-goarch
package-via-docker-arm64:
$(MAKE) package-via-docker-goarch-arm64
EXTRA_ENVS='CC=/opt/cross-builder/aarch64-linux-musl-cross/bin/aarch64-linux-musl-gcc' \
CGO_ENABLED=1 GOARCH=arm64 $(MAKE) package-via-docker-goarch
package-via-docker-ppc64le:
GOARCH=ppc64le $(MAKE) package-via-docker-goarch-nocgo
GOARCH=ppc64le CGO_ENABLED=0 $(MAKE) package-via-docker-goarch
package-via-docker-386:
GOARCH=386 $(MAKE) package-via-docker-goarch-nocgo
GOARCH=386 CGO_ENABLED=0 $(MAKE) package-via-docker-goarch
remove-docker-images:
docker image ls --format '{{.ID}}' | xargs docker image rm -f

View file

@ -43,7 +43,7 @@ groups:
dashboard: "http://localhost:3000/d/wNf0q_kZk?viewPanel=53&var-instance={{ $labels.instance }}"
summary: "Instance {{ $labels.instance }} (job={{ $labels.job }}) will run out of disk space soon"
description: "Disk utilisation on instance {{ $labels.instance }} is more than 80%.\n
Having less than 20% of free disk space could cripple merges processes and overall performance.
Having less than 20% of free disk space could cripple merge processes and overall performance.
Consider to limit the ingestion rate, decrease retention or scale the disk space if possible."
- alert: RequestErrorsToAPI
@ -129,5 +129,5 @@ groups:
dashboard: "http://localhost:3000/d/wNf0q_kZk?viewPanel=74&var-instance={{ $labels.instance }}"
summary: "Metrics ingested in ({{ $labels.instance }}) are exceeding labels limit"
description: "VictoriaMetrics limits the number of labels per each metric with `-maxLabelsPerTimeseries` command-line flag.\n
This prevents from ingesting metrics with too many labels. Please verify that `-maxLabelsPerTimeseries` is configured
This prevents ingestion of metrics with too many labels. Please verify that `-maxLabelsPerTimeseries` is configured
correctly or that clients which send these metrics aren't misbehaving."

View file

@ -3,7 +3,12 @@ FROM $go_builder_image
STOPSIGNAL SIGINT
RUN apk add git gcc musl-dev make wget --no-cache && \
mkdir /opt/cross-builder && \
wget https://musl.cc/aarch64-linux-musl-cross.tgz -O /opt/cross-builder/aarch64-musl.tgz --no-verbose && \
cd /opt/cross-builder && \
tar zxf aarch64-musl.tgz -C ./ && \
rm /opt/cross-builder/aarch64-musl.tgz
for arch in aarch64 x86_64; do \
wget \
https://musl.cc/${arch}-linux-musl-cross.tgz \
-O /opt/cross-builder/${arch}-musl.tgz \
--no-verbose && \
tar zxf ${arch}-musl.tgz -C ./ && \
rm /opt/cross-builder/${arch}-musl.tgz; \
done

View file

@ -5,7 +5,7 @@ services:
# And forward them to --remoteWrite.url
vmagent:
container_name: vmagent
image: victoriametrics/vmagent:v1.100.1
image: victoriametrics/vmagent:v1.101.0
depends_on:
- "vminsert"
ports:
@ -21,7 +21,7 @@ services:
# Grafana instance configured with VictoriaMetrics as datasource
grafana:
container_name: grafana
image: grafana/grafana:10.3.1
image: grafana/grafana:10.4.2
depends_on:
- "vmauth"
ports:
@ -39,7 +39,7 @@ services:
# where N is number of vmstorages (2 in this case).
vmstorage-1:
container_name: vmstorage-1
image: victoriametrics/vmstorage:v1.100.1-cluster
image: victoriametrics/vmstorage:v1.101.0-cluster
ports:
- 8482
- 8400
@ -51,7 +51,7 @@ services:
restart: always
vmstorage-2:
container_name: vmstorage-2
image: victoriametrics/vmstorage:v1.100.1-cluster
image: victoriametrics/vmstorage:v1.101.0-cluster
ports:
- 8482
- 8400
@ -66,7 +66,7 @@ services:
# pre-process them and distributes across configured vmstorage shards.
vminsert:
container_name: vminsert
image: victoriametrics/vminsert:v1.100.1-cluster
image: victoriametrics/vminsert:v1.101.0-cluster
depends_on:
- "vmstorage-1"
- "vmstorage-2"
@ -81,7 +81,7 @@ services:
# vmselect collects results from configured `--storageNode` shards.
vmselect-1:
container_name: vmselect-1
image: victoriametrics/vmselect:v1.100.1-cluster
image: victoriametrics/vmselect:v1.101.0-cluster
depends_on:
- "vmstorage-1"
- "vmstorage-2"
@ -94,7 +94,7 @@ services:
restart: always
vmselect-2:
container_name: vmselect-2
image: victoriametrics/vmselect:v1.100.1-cluster
image: victoriametrics/vmselect:v1.101.0-cluster
depends_on:
- "vmstorage-1"
- "vmstorage-2"
@ -112,7 +112,7 @@ services:
# It can be used as an authentication proxy.
vmauth:
container_name: vmauth
image: victoriametrics/vmauth:v1.100.1
image: victoriametrics/vmauth:v1.101.0
depends_on:
- "vmselect-1"
- "vmselect-2"
@ -127,7 +127,7 @@ services:
# vmalert executes alerting and recording rules
vmalert:
container_name: vmalert
image: victoriametrics/vmalert:v1.100.1
image: victoriametrics/vmalert:v1.101.0
depends_on:
- "vmauth"
ports:

View file

@ -3,7 +3,7 @@ services:
# Grafana instance configured with VictoriaLogs as datasource
grafana:
container_name: grafana
image: grafana/grafana:10.3.1
image: grafana/grafana:10.4.2
depends_on:
- "victoriametrics"
- "victorialogs"
@ -58,7 +58,7 @@ services:
# scraping, storing metrics and serve read requests.
victoriametrics:
container_name: victoriametrics
image: victoriametrics/victoria-metrics:v1.100.1
image: victoriametrics/victoria-metrics:v1.101.0
ports:
- 8428:8428
volumes:

View file

@ -5,7 +5,7 @@ services:
# And forward them to --remoteWrite.url
vmagent:
container_name: vmagent
image: victoriametrics/vmagent:v1.100.1
image: victoriametrics/vmagent:v1.101.0
depends_on:
- "victoriametrics"
ports:
@ -23,7 +23,7 @@ services:
# storing metrics and serve read requests.
victoriametrics:
container_name: victoriametrics
image: victoriametrics/victoria-metrics:v1.100.1
image: victoriametrics/victoria-metrics:v1.101.0
ports:
- 8428:8428
- 8089:8089
@ -47,7 +47,7 @@ services:
# Grafana instance configured with VictoriaMetrics as datasource
grafana:
container_name: grafana
image: grafana/grafana:10.3.1
image: grafana/grafana:10.4.2
depends_on:
- "victoriametrics"
ports:
@ -66,7 +66,7 @@ services:
# vmalert executes alerting and recording rules
vmalert:
container_name: vmalert
image: victoriametrics/vmalert:v1.100.1
image: victoriametrics/vmalert:v1.101.0
depends_on:
- "victoriametrics"
- "alertmanager"

View file

@ -2,7 +2,7 @@ version: "3"
services:
fluentbit:
image: cr.fluentbit.io/fluent/fluent-bit:2.1.4
image: cr.fluentbit.io/fluent/fluent-bit:3.0.2
volumes:
- /var/lib/docker/containers:/var/lib/docker/containers:ro
- ./fluent-bit.conf:/fluent-bit/etc/fluent-bit.conf

View file

@ -1,7 +1,7 @@
services:
vmagent:
container_name: vmagent
image: victoriametrics/vmagent:v1.100.1
image: victoriametrics/vmagent:v1.101.0
depends_on:
- "victoriametrics"
ports:
@ -18,7 +18,7 @@ services:
victoriametrics:
container_name: victoriametrics
image: victoriametrics/victoria-metrics:v1.100.1
image: victoriametrics/victoria-metrics:v1.101.0
ports:
- 8428:8428
volumes:
@ -51,7 +51,7 @@ services:
vmalert:
container_name: vmalert
image: victoriametrics/vmalert:v1.100.1
image: victoriametrics/vmalert:v1.101.0
depends_on:
- "victoriametrics"
ports:

View file

@ -18,7 +18,7 @@ services:
- vlogs
generator:
image: golang:1.22.2-alpine
image: golang:1.22.3-alpine
restart: always
working_dir: /go/src/app
volumes:

View file

@ -2,7 +2,7 @@ version: '3'
services:
generator:
image: golang:1.22.2-alpine
image: golang:1.22.3-alpine
restart: always
working_dir: /go/src/app
volumes:

View file

@ -46,7 +46,7 @@ services:
- '--config=/config.yml'
vmsingle:
image: victoriametrics/victoria-metrics:v1.100.1
image: victoriametrics/victoria-metrics:v1.101.0
ports:
- '8428:8428'
command:

View file

@ -19,8 +19,8 @@ On the server:
* VictoriaMetrics is running on ports: 8428, 8089, 4242, 2003 and they are bound to the local interface.
********************************************************************************
# This image includes v1.100.1 release of VictoriaMetrics.
# See Release notes https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.100.1
# This image includes v1.101.0 release of VictoriaMetrics.
# See Release notes https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0
# Welcome to VictoriaMetrics droplet!

View file

@ -30,27 +30,56 @@ See also [LTS releases](https://docs.victoriametrics.com/lts-releases/).
## tip
* SECURITY: upgrade Go builder from Go1.22.2 to Go1.22.3. See [the list of issues addressed in Go1.22.3](https://github.com/golang/go/issues?q=milestone%3AGo1.22.3+label%3ACherryPickApproved).
* FEATURE: [dashboards/single](https://grafana.com/grafana/dashboards/10229): support selecting of multiple instances on the dashboard. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5869) for details.
* FEATURE: [dashboards/single](https://grafana.com/grafana/dashboards/10229): properly display version in the Stats row for the custom builds of VictoriaMetrics.
* FEATURE: [dashboards/single](https://grafana.com/grafana/dashboards/10229): add `Network Usage` panel to `Resource Usage` row.
* FEATURE: [dashboards/operator](https://grafana.com/grafana/dashboards/17869), [dashboards/backupmanager](https://grafana.com/grafana/dashboards/17798) and [dashboard/tenant-statistic](https://grafana.com/grafana/dashboards/16399): update dashboard to be compatible with Grafana 10+ version.
* FEATURE: [dashboards/cluster](https://grafana.com/grafana/dashboards/11176): add new panel `Concurrent selects` to `vmstorage` row. The panel will show how many ongoing select queries are processed by vmstorage and should help to identify resource bottlenecks. See panel description for more details.
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add service discovery support for [Vultr](https://www.vultr.com/). See [these docs](https://docs.victoriametrics.com/sd_configs/#vultr_sd_configs) and [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6041).
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): allow configuring `-remoteWrite.disableOnDiskQueue` and `-remoteWrite.dropSamplesOnOverload` cmd-line flags per each `-remoteWrite.url`. See this [pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/6065). Thanks to @rbizos for implementaion!
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add labels `path` and `url` to metrics `vmagent_remotewrite_push_failures_total` and `vmagent_remotewrite_samples_dropped_total`. Now number of failed pushes and dropped samples can be tracked per `-remoteWrite.url`.
* BUGFIX: [vmui](https://docs.victoriametrics.com/#vmui): fix bug that prevents the first query trace from expanding on click event. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6186). The issue was introduced in [v1.100.0](https://docs.victoriametrics.com/changelog/#v11000) release.
* BUGFIX: [vmagent](https://docs.victoriametrics.com/vmagent/): prevent potential panic during [stream aggregation](https://docs.victoriametrics.com/stream-aggregation.html) if more than one `--remoteWrite.streamAggr.dedupInterval` is configured. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6205).
* BUGFIX: [stream aggregation](https://docs.victoriametrics.com/stream-aggregation/): set correct suffix `<output>_prometheus` for aggregation outputs [increase_prometheus](https://docs.victoriametrics.com/stream-aggregation/#increase_prometheus) and [total_prometheus](https://docs.victoriametrics.com/stream-aggregation/#total_prometheus). Before, outputs `total` and `total_prometheus` or `increase` and `increase_prometheus` had the same suffix.
## [v1.101.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0)
Released at 2024-04-26
* FEATURE: [MetricsQL](https://docs.victoriametrics.com/metricsql/): support filtering by multiple numeric constants via `q == (C1, ..., CN)` and `q != (C1, ..., CN)` syntax. For example, `status_code == (200, 201, 300)` returns `status_code` metrics with any of `200`, `201` or `300` values, while `status_code != (400, 404, 500)` returns `status_code` metrics with all the values except of `400`, `404` and `500`.
* FEATURE: [VictoriaMetrics cluster](https://docs.victoriametrics.com/cluster-victoriametrics/): add support for fault domain awareness to `vmselect`. It can be configured to return full responses if up to `-globalReplicationFactor - 1` fault domains (aka `vmstorage` groups) are unavailable. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6054) and [these docs](https://docs.victoriametrics.com/cluster-victoriametrics/#vmstorage-groups-at-vmselect).
* FEATURE: all VictoriaMetrics [enterprise](https://docs.victoriametrics.com/enterprise/) components: add support for automatic issuing of TLS certificates for HTTPS server at `-httpListenAddr` via [Let's Encrypt service](https://letsencrypt.org/). See [these docs](https://docs.victoriametrics.com/#automatic-issuing-of-tls-certificates) and [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5949).
* FEATURE: [vmsingle](https://docs.victoriametrics.com/single-server-victoriametrics/): support disable or log [implicit conversions](https://docs.victoriametrics.com/metricsql/#implicit-query-conversions) for subquery with cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion`. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4338).
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent/): support data replication additionally to sharding among remote storage systems if `-remoteWrite.shardByURLReplicas=N` command-line flag is set additionally to `-remoteWrite.shardByURL` command-line flag, where `N` is desired replication factor. This allows setting up data replication among failure domains when the replication factor is smaller than the number of failure domains. See [these docs](https://docs.victoriametrics.com/vmagent/#sharding-among-remote-storages) and [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6054).
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent/): reduce CPU usage when [sharding among remote storage systems](https://docs.victoriametrics.com/vmagent/#sharding-among-remote-storages) is enabled.
* FEATURE: [stream aggregation](https://docs.victoriametrics.com/stream-aggregation/): reduce memory usage during stream aggregation if multiple aggregation configs are used for the same set of data.
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent/): support [DNS SRV](https://en.wikipedia.org/wiki/SRV_record) addresses in `-remoteWrite.url` command-line option and in scrape target urls. For example, `-remoteWrite.url=http://srv+victoria-metrics/api/v1/write` automatically resolves the `victoria-metrics` DNS SRV to a list of hostnames with TCP ports and then sends the collected metrics to these TCP addresses. See [these docs](https://docs.victoriametrics.com/vmagent/#srv-urls) and [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6053).
* FEATURE: [stream aggregation](https://docs.victoriametrics.com/stream-aggregation/): allow skipping first N aggregation intervals via cmd-line flag `-streamAggr.ignoreFirstIntervals` for [Single-node VictoriaMetrics](https://docs.victoriametrics.com/) or `-remoteWrite.streamAggr.ignoreFirstIntervals` for [vmagent](https://docs.victoriametrics.com/vmagent/). See more details [here](https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start).
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth/): support automatic discovering and load balancing for TCP addresses behind DNS SRV addresses. These addresses can be put inside `url_prefix` urls in the form `http://srv+addr/path`, where the `addr` is the [DNS SRV](https://en.wikipedia.org/wiki/SRV_record) address, which is automatically resolved to hostnames with TCP ports. See [these docs](https://docs.victoriametrics.com/vmauth/#srv-urls) for details.
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth/): support specifying client TLS certificates and TLS ServerName for requests to HTTPS backends. See [these docs](https://docs.victoriametrics.com/vmauth/#backend-tls-setup).
* FEATURE: [vmauth](https://docs.victoriametrics.com/vmauth/): support regex matching when routing incoming requests based on HTTP [query args](https://en.wikipedia.org/wiki/Query_string) via `src_query_args` option at `url_map`. See [these docs](https://docs.victoriametrics.com/vmauth/#generic-http-proxy-for-different-backends) and [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6070).
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): optimize auto-suggestion performance for metric names when the database contains big number of unique time series.
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): in the Select component, user-entered values are now preserved on blur if they match options in the list.
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): auto-suggestion triggers at any cursor position in the query input. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5864).
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): update error messages on the Query page for enhanced clarity. See [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/6177).
* BUGFIX: [downsampling](https://docs.victoriametrics.com/#downsampling): skip unnecessary index lookups if downsampling wasn't set for ENT versions of VictoriaMetrics. Before, users of VictoriaMetrics ENT could have experience elevated CPU usage even if no downsampling was configured. The issue was introduced in [v1.100.0](https://docs.victoriametrics.com/changelog/#v11000).
* BUGFIX: [downsampling](https://docs.victoriametrics.com/#downsampling): properly populate downsampling metadata for data parts created by VictoriaMetrics ENT versions lower than v1.100.0. The bug could trigger the downsampling actions for parts that were downsampled already. This bug doesn't have any negative effect apart from spending extra CPU resources on the repeated downsampling. The issue was introduced in [v1.100.0](https://docs.victoriametrics.com/changelog/#v11000).
* BUGFIX: [vmalert](https://docs.victoriametrics.com/vmalert/): supported any status codes from the range 200-299 from alertmanager. Previously, only 200 status code considered a successful action. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6110).
* BUGFIX: [vmalert](https://docs.victoriametrics.com/vmalert/): avoid blocking `/api/v1/rules`, `/api/v1/alerts`, `/metrics` APIs when alerting rule uses template functions `query`, which could takes a while to execute. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6079).
* BUGFIX: [vmalert](https://docs.victoriametrics.com/vmalert/): fix links with anchors in vmalert's UI. Starting from [v1.99.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.99.0) vmalert could ignore anchors pointing to specific rule groups if `search` param was present in URL.
* BUGFIX: [vmauth](https://docs.victoriametrics.com/vmauth/): don't treat concurrency limit hit as an error of the backend. Previously, hitting the concurrency limit would increment both `vmauth_concurrent_requests_limit_reached_total` and `vmauth_user_request_backend_errors_total` counters. Now, only concurrency limit counter is incremented. Updates [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5565).
## [v1.100.1](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.100.1)
Released at 2024-04-11
**Update note 1: When upgrading to this release from [v1.99.0](https://docs.victoriametrics.com/changelog/#v1990) or [v1.100.0](https://docs.victoriametrics.com/changelog/#v11000) it is recommended to reset caches stored on disk according to [these](https://docs.victoriametrics.com/single-server-victoriametrics/#cache-removal) docs.**
**Update note 1: This release contains the issue which could lead to extra CPU usage of storage component in ENT distribution of VictoriaMetrics. The issue is caused by [downsampling per distinct sets of time series](https://docs.victoriametrics.com/#downsampling) feature introduced in 1.100.0 ENT version. Please, upgrade to [v1.101.0](https://docs.victoriametrics.com/changelog/#v11010) or rollback to [v1.98.0](https://docs.victoriametrics.com/changelog/#v1980) ENT version if you're affected.**
**Update note 2: When upgrading to this release from [v1.99.0](https://docs.victoriametrics.com/changelog/#v1990) or [v1.100.0](https://docs.victoriametrics.com/changelog/#v11000) it is recommended to reset caches stored on disk according to [these](https://docs.victoriametrics.com/single-server-victoriametrics/#cache-removal) docs.**
* FEATURE: [vmbackupmanager](https://docs.victoriametrics.com/vmbackupmanager/): allow specifying custom backup interval via `-backupInterval` command-line flag. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5966).
@ -61,7 +90,7 @@ Released at 2024-04-11
Released at 2024-04-04
**This release contains [the issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5959), which can prevent from storing data for new time series under high rate of search queries. Please rollback to [v1.98.0](https://docs.victoriametrics.com/changelog/#v1980) or upgrade to [v1.100.1](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.100.1).**
**This release contains [the issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5959), which can prevent from storing data for new time series under high rate of search queries. Please rollback to [v1.98.0](https://docs.victoriametrics.com/changelog/#v1980) or upgrade to [v1.101.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0).**
**Update note 1: the `-datasource.lookback` command-line flag at `vmalert` is no-op starting from this release. This flag will be removed in the future, so please switch to [`eval_delay` option](https://docs.victoriametrics.com/vmalert/#groups). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5155) for more details.**
@ -119,7 +148,7 @@ Released at 2024-04-04
Released at 2024-03-01
**This release contains [the issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5959), which can prevent from storing data for new time series under high rate of search queries. Please rollback to [v1.98.0](https://docs.victoriametrics.com/changelog/#v1980) or upgrade to [v1.100.1](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.100.1).**
**This release contains [the issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5959), which can prevent from storing data for new time series under high rate of search queries. Please rollback to [v1.98.0](https://docs.victoriametrics.com/changelog/#v1980) or upgrade to [v1.101.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0).**
* FEATURE: [MetricsQL](https://docs.victoriametrics.com/metricsql/): propagate [label filters](https://docs.victoriametrics.com/keyconcepts/#filtering) via all the [label manipulation functions](https://docs.victoriametrics.com/metricsql/#label-manipulation-functions). For example, `label_del(some_metric{job="foo"}, "instance") + other_metric{pod="bar"}` is now transformed to `label_del(some_metric{job="foo",pod="bar"}, "instance") + other_metric{job="foo",pod="bar"}`. This should reduce the amounts of time series processed during query execution.
* FEATURE: [MetricsQL](https://docs.victoriametrics.com/metricsql/): add [count_values_over_time](https://docs.victoriametrics.com/metricsql/#count_values_over_time) function. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5847).

View file

@ -384,7 +384,7 @@ Check practical examples of VictoriaMetrics API [here](https://docs.victoriametr
- `datadog/api/v1/series` - for ingesting data with DataDog submit metrics API v1. See [these docs](https://docs.victoriametrics.com/url-examples/#datadogapiv1series) for details.
- `datadog/api/v2/series` - for ingesting data with [DataDog submit metrics API](https://docs.datadoghq.com/api/latest/metrics/#submit-metrics). See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-datadog-agent) for details.
- `datadog/api/beta/sketches` - for ingesting data with [DataDog lambda extension](https://docs.datadoghq.com/serverless/libraries_integrations/extension/).
- `influx/write` and `influx/api/v2/write` - for ingesting data with [InfluxDB line protocol](https://docs.influxdata.com/influxdb/v1.7/write_protocols/line_protocol_tutorial/). See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) for details.
- `influx/write` and `influx/api/v2/write` - for ingesting data with [InfluxDB line protocol](https://docs.influxdata.com/influxdb/v1.7/write_protocols/line_protocol_tutorial/). TCP and UDP receiver is disabled by default. It is exposed on a distinct TCP address set via `-influxListenAddr` command-line flag. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) for details.
- `newrelic/infra/v2/metrics/events/bulk` - for accepting data from [NewRelic infrastructure agent](https://docs.newrelic.com/docs/infrastructure/install-infrastructure-agent). See [these docs](https://docs.victoriametrics.com/#how-to-send-data-from-newrelic-agent) for details.
- `opentsdb/api/put` - for accepting [OpenTSDB HTTP /api/put requests](http://opentsdb.net/docs/build/html/api_http/put.html). This handler is disabled by default. It is exposed on a distinct TCP address set via `-opentsdbHTTPListenAddr` command-line flag. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#sending-opentsdb-data-via-http-apiput-requests) for details.

View file

@ -2229,6 +2229,7 @@ Any [rollup function](#rollup-functions) for something other than [series select
Nested rollup functions can be implicit thanks to the [implicit query conversions](#implicit-query-conversions).
For example, `delta(sum(m))` is implicitly converted to `delta(sum(default_rollup(m))[1i:1i])`, so it becomes a subquery,
since it contains [default_rollup](#default_rollup) nested into [delta](#delta).
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.
VictoriaMetrics performs subqueries in the following way:
@ -2262,3 +2263,4 @@ VictoriaMetrics performs the following implicit conversions for incoming queries
* If something other than [series selector](https://docs.victoriametrics.com/keyconcepts/#filtering)
is passed to [rollup function](#rollup-functions), then a [subquery](#subqueries) with `1i` lookbehind window and `1i` step is automatically formed.
For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`.
This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0.

View file

@ -89,6 +89,7 @@ VictoriaMetrics has the following prominent features:
* [Prometheus exposition format](#how-to-import-data-in-prometheus-exposition-format).
* [InfluxDB line protocol](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) over HTTP, TCP and UDP.
* [Graphite plaintext protocol](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) with [tags](https://graphite.readthedocs.io/en/latest/tags.html#carbon).
* [Statsd plaintext protocol](#how-to-send-data-from-statsd-compatible-clients)
* [OpenTSDB put message](#sending-data-via-telnet-put-protocol).
* [HTTP OpenTSDB /api/put requests](#sending-opentsdb-data-via-http-apiput-requests).
* [JSON line format](#how-to-import-data-in-json-line-format).
@ -704,6 +705,45 @@ The `/api/v1/export` endpoint should return the following response:
{"metric":{"__name__":"measurement_field2","tag1":"value1","tag2":"value2"},"values":[1.23],"timestamps":[1695902762311]}
```
## How to send data from Statsd-compatible clients
VictoriaMetrics supports extended statsd protocol with tags. Also it does not support sampling and metric types(it will be ignored).
Enable Statsd receiver in VictoriaMetrics by setting `-statsdListenAddr` command line flag. For instance,
the following command will enable Statsd receiver in VictoriaMetrics on TCP and UDP port `8125`:
```console
/path/to/victoria-metrics-prod -statsdListenAddr=:8125
```
Example for writing data with Statsd plaintext protocol to local VictoriaMetrics using `nc`:
```console
echo "foo.bar:123|g|#foo:bar" | nc -N localhost 8125
```
Explicit setting of timestamps is not supported for statsd protocol. Timestamp is set to the current time when VictoriaMetrics or vmagent receives it.
An arbitrary number of lines delimited by `\n` (aka newline char) can be sent in one go.
After that the data may be read via [/api/v1/export](#how-to-export-data-in-json-line-format) endpoint:
<div class="with-copy" markdown="1">
```console
curl -G 'http://localhost:8428/api/v1/export' -d 'match=foo.bar.baz'
```
</div>
The `/api/v1/export` endpoint should return the following response:
```json
{"metric":{"__name__":"foo.bar.baz","tag1":"value1","tag2":"value2"},"values":[123],"timestamps":[1560277406000]}
```
Some examples of compatible statsd clients:
- [statsd-instrument](https://github.com/Shopify/statsd-instrument)
- [dogstatsd-ruby](https://github.com/DataDog/dogstatsd-ruby)
- [go-statsd-client](https://github.com/cactus/go-statsd-client)
## How to send data from Graphite-compatible agents such as [StatsD](https://github.com/etsy/statsd)
Enable Graphite receiver in VictoriaMetrics by setting `-graphiteListenAddr` command line flag. For instance,
@ -3165,6 +3205,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li
An optional list of labels to drop from samples before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-streamAggr.ignoreFirstIntervals int
Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start
-streamAggr.ignoreOldSamples
Whether to ignore input samples with old timestamps outside the current aggregation interval. See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples
-streamAggr.keepInput

View file

@ -38,11 +38,6 @@ docker buildx create --use --name=qemu
docker buildx inspect --bootstrap
```
For ARM arch (M1/M2 processors) additionally configure docker with preferred platform:
```
export DOCKER_DEFAULT_PLATFORM=linux/amd64
```
By default, docker on MacOS has limited amount of resources (CPU, mem) to use.
Bumping the limits may significantly improve build speed.

View file

@ -97,6 +97,7 @@ VictoriaMetrics has the following prominent features:
* [Prometheus exposition format](#how-to-import-data-in-prometheus-exposition-format).
* [InfluxDB line protocol](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) over HTTP, TCP and UDP.
* [Graphite plaintext protocol](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) with [tags](https://graphite.readthedocs.io/en/latest/tags.html#carbon).
* [Statsd plaintext protocol](#how-to-send-data-from-statsd-compatible-clients)
* [OpenTSDB put message](#sending-data-via-telnet-put-protocol).
* [HTTP OpenTSDB /api/put requests](#sending-opentsdb-data-via-http-apiput-requests).
* [JSON line format](#how-to-import-data-in-json-line-format).
@ -712,6 +713,45 @@ The `/api/v1/export` endpoint should return the following response:
{"metric":{"__name__":"measurement_field2","tag1":"value1","tag2":"value2"},"values":[1.23],"timestamps":[1695902762311]}
```
## How to send data from Statsd-compatible clients
VictoriaMetrics supports extended statsd protocol with tags. Also it does not support sampling and metric types(it will be ignored).
Enable Statsd receiver in VictoriaMetrics by setting `-statsdListenAddr` command line flag. For instance,
the following command will enable Statsd receiver in VictoriaMetrics on TCP and UDP port `8125`:
```console
/path/to/victoria-metrics-prod -statsdListenAddr=:8125
```
Example for writing data with Statsd plaintext protocol to local VictoriaMetrics using `nc`:
```console
echo "foo.bar:123|g|#foo:bar" | nc -N localhost 8125
```
Explicit setting of timestamps is not supported for statsd protocol. Timestamp is set to the current time when VictoriaMetrics or vmagent receives it.
An arbitrary number of lines delimited by `\n` (aka newline char) can be sent in one go.
After that the data may be read via [/api/v1/export](#how-to-export-data-in-json-line-format) endpoint:
<div class="with-copy" markdown="1">
```console
curl -G 'http://localhost:8428/api/v1/export' -d 'match=foo.bar.baz'
```
</div>
The `/api/v1/export` endpoint should return the following response:
```json
{"metric":{"__name__":"foo.bar.baz","tag1":"value1","tag2":"value2"},"values":[123],"timestamps":[1560277406000]}
```
Some examples of compatible statsd clients:
- [statsd-instrument](https://github.com/Shopify/statsd-instrument)
- [dogstatsd-ruby](https://github.com/DataDog/dogstatsd-ruby)
- [go-statsd-client](https://github.com/cactus/go-statsd-client)
## How to send data from Graphite-compatible agents such as [StatsD](https://github.com/etsy/statsd)
Enable Graphite receiver in VictoriaMetrics by setting `-graphiteListenAddr` command line flag. For instance,
@ -3173,6 +3213,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li
An optional list of labels to drop from samples before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-streamAggr.ignoreFirstIntervals int
Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start
-streamAggr.ignoreOldSamples
Whether to ignore input samples with old timestamps outside the current aggregation interval. See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples
-streamAggr.keepInput

View file

@ -16,6 +16,44 @@ aliases:
## Next release
## [v0.44.0](https://github.com/VictoriaMetrics/operator/releases/tag/v0.44.0) - 9 May 2024
- [vmagent](./api.md#vmagent): adds new fields into `streamAggrConfig`: `dedup_interval`, `ignore_old_samples`, `keep_metric_names`, `no_align_flush_to_interval`. It's only possible to use it with v1.100+ version of `vmagent`. See this [issue](https://github.com/VictoriaMetrics/operator/issues/936) for details.
- [operator](./README.md): use `Patch` for `finalizers` set/unset operations. It must fix possible issues with `CRD` objects mutations. See this [issue](https://github.com/VictoriaMetrics/operator/issues/946) for details.
- [operator](./README.md): adds `spec.pause` field to `VMAgent`, `VMAlert`, `VMAuth`, `VMCluster`, `VMAlertmanager` and `VMSingle`. It allows to suspend object reconcile by operator. See this [issue](https://github.com/VictoriaMetrics/operator/issues/943) for details. Thanks @just1900
- [vmagent](./api.md#vmagent): set `status.selector` field. It allows correctly use `VPA` with `vmagent`. See this [issue](https://github.com/VictoriaMetrics/operator/issues/693) for details.
- [prometheus-converter](./README.md): fixes bug with prometheus-operator ScrapeConfig converter. Only copy `spec` field for it. See this [issue](https://github.com/VictoriaMetrics/operator/issues/942) for details.
- [vmscrapeconfig](./resources/vmscrapeconfig.md): `authorization` section in sd configs works properly with empty `type` field (default value for this field is `Bearer`).
- [prometheus-converter](./README.md): fixes owner reference type on VMScrapeConfig objects
- [vmauth&vmuser](./api.md#vmauth): sync config fields from [upstream](https://docs.victoriametrics.com/vmauth/), e.g., src_query_args, discover_backend_ips.
<a name="v0.43.5"></a>
## [v0.43.5](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.5) - 26 Apr 2024
- Update VictoriaMetrics image tags to [v1.101.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0).
<a name="v0.43.4"></a>
## [v0.43.4](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.4) - 25 Apr 2024
- [operator](./README.md): properly set status to `expanding` for `VMCluster` during initial creation. Previously, it was always `operational`.
- [operator](./README.md): adds more context to `Deployment` and `Statefulset` watch ready functions. Now, it reports state of unhealthy pod. It allows to find issue with it faster.
<a name="v0.43.3"></a>
## [v0.43.3](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.3) - 23 Apr 2024
- [operator](./README.md): fix conversion from `ServiceMonitor` to `VMServiceScrape`, `bearerTokenSecret` is dropped mistakenly since [v0.43.0](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.0). See [this issue](https://github.com/VictoriaMetrics/operator/issues/932).
- [operator](./README.md): fix selector match for config resources like VMUser, VMRule... , before it could be ignored when update resource labels.
<a name="v0.43.2"></a>
## [v0.43.2](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.2) - 22 Apr 2024
- [vmagent](./api.md#vmagent): fixes bug with `ServiceAccount` not found with `ingestOnlyMode`.
- [vmagent](./api.md#vmagent): fixes `unknown long flag '--rules-dir'` for prometheus-config-reloader.
<a name="v0.43.1"></a>
## [v0.43.1](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.1) - 18 Apr 2024

View file

@ -136,11 +136,13 @@ This Document documents the types introduced by the VictoriaMetrics to be consum
* [VMUserList](#vmuserlist)
* [VMUserSpec](#vmuserspec)
* [EmbeddedIngress](#embeddedingress)
* [URLMapCommon](#urlmapcommon)
* [UnauthorizedAccessConfigURLMap](#unauthorizedaccessconfigurlmap)
* [UserConfigOption](#userconfigoption)
* [VMAuth](#vmauth)
* [VMAuthList](#vmauthlist)
* [VMAuthSpec](#vmauthspec)
* [VMAuthStatus](#vmauthstatus)
* [VMAuthUnauthorizedPath](#vmauthunauthorizedpath)
* [TargetEndpoint](#targetendpoint)
* [VMStaticScrape](#vmstaticscrape)
* [VMStaticScrapeList](#vmstaticscrapelist)
@ -925,6 +927,7 @@ VMAgentSpec defines the desired state of VMAgent
| useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false |
| ingestOnlyMode | IngestOnlyMode switches vmagent into unmanaged mode it disables any config generation for scraping Currently it prevents vmagent from managing tls and auth options for remote write | bool | false |
| license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false |
| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false |
[Back to TOC](#table-of-contents)
@ -951,7 +954,7 @@ ServiceSpec defines additional service for CRD with user-defined params. by defa
| Field | Description | Scheme | Required |
| ----- | ----------- | ------ | -------- |
| useAsDefault | UseAsDefault applies changes from given service definition to the main object Service Chaning from headless service to clusterIP or loadbalancer may break cross-component communication | bool | false |
| useAsDefault | UseAsDefault applies changes from given service definition to the main object Service Changing from headless service to clusterIP or loadbalancer may break cross-component communication | bool | false |
| metadata | EmbeddedObjectMetadata defines objectMeta for additional service. | [EmbeddedObjectMetadata](#embeddedobjectmetadata) | false |
| spec | ServiceSpec describes the attributes that a user creates on a service. More info: https://kubernetes.io/docs/concepts/services-networking/service/ | v1.ServiceSpec | true |
@ -1132,11 +1135,16 @@ StreamAggrRule defines the rule in stream aggregation config
| ----- | ----------- | ------ | -------- |
| match | Match is a label selector (or list of label selectors) for filtering time series for the given selector.\n\nIf the match isn&#39;t set, then all the input time series are processed. | StringOrArray | false |
| interval | Interval is the interval between aggregations. | string | true |
| staleness_interval | StalenessInterval defines an interval after which the series state will be reset if no samples have been sent during it. | string | false |
| no_align_flush_to_interval | NoAlighFlushToInterval disables aligning of flushes to multiples of Interval. By default flushes are aligned to Interval. | *bool | false |
| flush_on_shutdown | FlushOnShutdown defines whether to flush the aggregation state on process termination or config reload. Is `false` by default. It is not recommended changing this setting, unless unfinished aggregations states are preferred to missing data points. | bool | false |
| dedup_interval | DedupInterval is an optional interval for deduplication. | string | false |
| staleness_interval | Staleness interval is interval after which the series state will be reset if no samples have been sent during it. The parameter is only relevant for outputs: total, total_prometheus, increase, increase_prometheus and histogram_bucket. | string | false |
| outputs | Outputs is a list of output aggregate functions to produce.\n\nThe following names are allowed:\n\n- total - aggregates input counters - increase - counts the increase over input counters - count_series - counts the input series - count_samples - counts the input samples - sum_samples - sums the input samples - last - the last biggest sample value - min - the minimum sample value - max - the maximum sample value - avg - the average value across all the samples - stddev - standard deviation across all the samples - stdvar - standard variance across all the samples - histogram_bucket - creates VictoriaMetrics histogram for input samples - quantiles(phi1, ..., phiN) - quantiles&#39; estimation for phi in the range [0..1]\n\nThe output time series will have the following names:\n\n input_name:aggr_&lt;interval&gt;_&lt;output&gt; | []string | true |
| keep_metric_names | KeepMetricNames instructs to leave metric names as is for the output time series without adding any suffix. | *bool | false |
| ignore_old_samples | IgnoreOldSamples instructs to ignore samples with old timestamps outside the current aggregation interval. | *bool | false |
| by | By is an optional list of labels for grouping input series.\n\nSee also Without.\n\nIf neither By nor Without are set, then the Outputs are calculated individually per each input time series. | []string | false |
| without | Without is an optional list of labels, which must be excluded when grouping input series.\n\nSee also By.\n\nIf neither By nor Without are set, then the Outputs are calculated individually per each input time series. | []string | false |
| drop_input_labels | DropInputLabels is an optional list with labels, which must be dropped before further processing of input samples.\n\nLabels are dropped before de-duplication and aggregation. | *[]string | false |
| input_relabel_configs | InputRelabelConfigs is an optional relabeling rules, which are applied on the input before aggregation. | [][RelabelConfig](#relabelconfig) | false |
| output_relabel_configs | OutputRelabelConfigs is an optional relabeling rules, which are applied on the aggregated output before being sent to remote storage. | [][RelabelConfig](#relabelconfig) | false |
@ -1297,6 +1305,7 @@ VMAlertSpec defines the desired state of VMAlert
| readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false |
| useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false |
| license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false |
| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false |
[Back to TOC](#table-of-contents)
@ -1391,6 +1400,7 @@ VMSingleSpec defines the desired state of VMSingle
| readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false |
| streamAggrConfig | StreamAggrConfig defines stream aggregation configuration for VMSingle | *[StreamAggrConfig](#streamaggrconfig) | false |
| useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false |
| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false |
[Back to TOC](#table-of-contents)
@ -1877,6 +1887,7 @@ VMClusterSpec defines the desired state of VMCluster
| vmselect | | *[VMSelect](#vmselect) | false |
| vminsert | | *[VMInsert](#vminsert) | false |
| vmstorage | | *[VMStorage](#vmstorage) | false |
| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false |
| useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false |
[Back to TOC](#table-of-contents)
@ -2160,12 +2171,8 @@ TargetRef describes target for user traffic forwarding. one of target types can
| static | Static - user defined url for traffic forward, for instance http://vmsingle:8429 | *[StaticRef](#staticref) | false |
| paths | Paths - matched path to route. | []string | false |
| hosts | | []string | false |
| target_path_suffix | QueryParams []string `json:\&#34;queryParams,omitempty\&#34;` TargetPathSuffix allows to add some suffix to the target path It allows to hide tenant configuration from user with crd as ref. it also may contain any url encoded params. | string | false |
| headers | Headers represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries Can be defined per target or at VMUser.spec level e.g. [429,503] | []int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
| URLMapCommon | | [URLMapCommon](#urlmapcommon) | false |
| target_path_suffix | TargetPathSuffix allows to add some suffix to the target path It allows to hide tenant configuration from user with crd as ref. it also may contain any url encoded params. | string | false |
| targetRefBasicAuth | TargetRefBasicAuth allow an target endpoint to authenticate over basic authentication | *[TargetRefBasicAuth](#targetrefbasicauth) | false |
[Back to TOC](#table-of-contents)
@ -2230,14 +2237,15 @@ VMUserSpec defines the desired state of VMUser
| bearerToken | BearerToken Authorization header value for accessing protected endpoint. | *string | false |
| targetRefs | TargetRefs - reference to endpoints, which user may access. | [][TargetRef](#targetref) | true |
| default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false |
| tlsConfig | | *[TLSConfig](#tlsconfig) | false |
| ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false |
| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false |
| headers | Headers represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false |
| max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
| tls_insecure_skip_verify | TLSInsecureSkipVerify - whether to skip TLS verification when connecting to backend over HTTPS. See https://docs.victoriametrics.com/vmauth.html#backend-tls-setup | bool | false |
| metric_labels | MetricLabels - additional labels for metrics exported by vmauth for given user. | map[string]string | false |
| disable_secret_creation | DisableSecretCreation skips related secret creation for vmuser | bool | false |
@ -2261,6 +2269,55 @@ EmbeddedIngress describes ingress configuration options.
[Back to TOC](#table-of-contents)
## URLMapCommon
URLMapCommon contains common fields for unauthorized user and user in vmuser
| Field | Description | Scheme | Required |
| ----- | ----------- | ------ | -------- |
| src_query_args | SrcQueryArgs is an optional list of query args, which must match request URL query args. | []string | false |
| src_headers | SrcHeaders is an optional list of headers, which must match request headers. | []string | false |
| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false |
| headers | RequestHeaders represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries Can be defined per target or at VMUser.spec level e.g. [429,503] | []int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
[Back to TOC](#table-of-contents)
## UnauthorizedAccessConfigURLMap
| Field | Description | Scheme | Required |
| ----- | ----------- | ------ | -------- |
| src_paths | SrcPaths is an optional list of regular expressions, which must match the request path. | []string | false |
| src_hosts | SrcHosts is an optional list of regular expressions, which must match the request hostname. | []string | false |
| url_prefix | UrlPrefix contains backend url prefixes for the proxied request url. | []string | false |
| URLMapCommon | | [URLMapCommon](#urlmapcommon) | false |
[Back to TOC](#table-of-contents)
## UserConfigOption
| Field | Description | Scheme | Required |
| ----- | ----------- | ------ | -------- |
| default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false |
| tlsConfig | | *[TLSConfig](#tlsconfig) | false |
| ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false |
| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false |
| headers | Headers represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false |
| max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
[Back to TOC](#table-of-contents)
## VMAuth
VMAuth is the Schema for the vmauths API
@ -2334,10 +2391,21 @@ VMAuthSpec defines the desired state of VMAuth
| nodeSelector | NodeSelector Define which Nodes the Pods are scheduled on. | map[string]string | false |
| terminationGracePeriodSeconds | TerminationGracePeriodSeconds period for container graceful termination | *int64 | false |
| readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false |
| unauthorizedAccessConfig | UnauthorizedAccessConfig configures access for un authorized users | [][VMAuthUnauthorizedPath](#vmauthunauthorizedpath) | false |
| unauthorizedAccessConfig | UnauthorizedAccessConfig configures access for un authorized users | [][UnauthorizedAccessConfigURLMap](#unauthorizedaccessconfigurlmap) | false |
| default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false |
| tlsConfig | | *[TLSConfig](#tlsconfig) | false |
| ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false |
| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false |
| headers | Headers represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false |
| max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
| useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false |
| license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false |
| configSecret | ConfigSecret is the name of a Kubernetes Secret in the same namespace as the VMAuth object, which contains auth configuration for vmauth, configuration must be inside secret key: config.yaml. It must be created and managed manually. If it&#39;s defined, configuration for vmauth becomes unmanaged and operator&#39;ll not create any related secrets/config-reloaders | string | false |
| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false |
[Back to TOC](#table-of-contents)
@ -2352,24 +2420,6 @@ VMAuthStatus defines the observed state of VMAuth
[Back to TOC](#table-of-contents)
## VMAuthUnauthorizedPath
VMAuthUnauthorizedPath defines url_map for unauthorized access
| Field | Description | Scheme | Required |
| ----- | ----------- | ------ | -------- |
| src_paths | Paths src request paths | []string | false |
| url_prefix | URLs defines url_prefix for dst routing | []string | false |
| ip_filters | IPFilters defines filter for src ip address enterprise only | [VMUserIPFilters](#vmuseripfilters) | false |
| src_hosts | SrcHosts is the list of regular expressions, which match the request hostname. | []string | false |
| headers | Headers represent additional http headers, that vmauth uses in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.68.0 version of vmauth | []string | false |
| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\&#34;header_key: header_value\&#34;] multiple values for header key: [\&#34;header_key: value1,value2\&#34;] it&#39;s available since 1.93.0 version of vmauth | []string | false |
| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false |
| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \&#34;least_loaded\&#34;) | *string | false |
| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false |
[Back to TOC](#table-of-contents)
## TargetEndpoint
TargetEndpoint defines single static target endpoint.

View file

@ -33,7 +33,7 @@ The CRD specifies which `VMUser`s should be covered by the deployed `VMAuth` ins
The Operator then generates a configuration based on the included `VMUser`s and updates the `Configmaps` containing
the configuration. It continuously does so for all changes that are made to `VMUser`s or to the `VMAuth` resource itself.
[VMUser](./vmrule.md) objects generate part of `VMAuth` configuration.
[VMUser](./vmuser.md) objects generate part of `VMAuth` configuration.
For filtering users `VMAuth` uses selectors `userNamespaceSelector` and `userSelector`.
It allows configuring rules access control across namespaces and different environments.
@ -101,8 +101,8 @@ metadata:
name: vmauth-unauthorized-example
spec:
unauthorizedAccessConfig:
- paths: ["/metrics"]
urls:
- src_paths: ["/metrics"]
url_prefix:
- http://vmsingle-example.default.svc:8428
```
@ -245,8 +245,8 @@ spec:
- 5.6.7.8
# allow read vmsingle metrics without authorization for users from internal network
unauthorizedAccessConfig:
- paths: ["/metrics"]
urls: ["http://vmsingle-example.default.svc:8428"]
- src_paths: ["/metrics"]
url_prefix: ["http://vmsingle-example.default.svc:8428"]
ip_filters:
allow_list:
- 192.168.0.0/16

View file

@ -10,7 +10,7 @@ menu:
<!-- this doc autogenerated - don't edit it manually -->
# Auto Generated vars for package config
updated at Thu Apr 18 19:07:23 UTC 2024
updated at Thu May 9 16:51:28 UTC 2024
| varible name | variable default value | variable required | variable description |
@ -20,7 +20,7 @@ menu:
| VM_CUSTOMCONFIGRELOADERIMAGE | victoriametrics/operator:config-reloader-v0.43.0 | false | - |
| VM_PSPAUTOCREATEENABLED | false | false | - |
| VM_VMALERTDEFAULT_IMAGE | victoriametrics/vmalert | false | - |
| VM_VMALERTDEFAULT_VERSION | v1.100.1 | false | - |
| VM_VMALERTDEFAULT_VERSION | v1.101.0 | false | - |
| VM_VMALERTDEFAULT_PORT | 8080 | false | - |
| VM_VMALERTDEFAULT_USEDEFAULTRESOURCES | true | false | - |
| VM_VMALERTDEFAULT_RESOURCE_LIMIT_MEM | 500Mi | false | - |
@ -31,7 +31,7 @@ menu:
| VM_VMALERTDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - |
| VM_VMALERTDEFAULT_CONFIGRELOADIMAGE | jimmidyson/configmap-reload:v0.3.0 | false | - |
| VM_VMAGENTDEFAULT_IMAGE | victoriametrics/vmagent | false | - |
| VM_VMAGENTDEFAULT_VERSION | v1.100.1 | false | - |
| VM_VMAGENTDEFAULT_VERSION | v1.101.0 | false | - |
| VM_VMAGENTDEFAULT_CONFIGRELOADIMAGE | quay.io/prometheus-operator/prometheus-config-reloader:v0.68.0 | false | - |
| VM_VMAGENTDEFAULT_PORT | 8429 | false | - |
| VM_VMAGENTDEFAULT_USEDEFAULTRESOURCES | true | false | - |
@ -42,7 +42,7 @@ menu:
| VM_VMAGENTDEFAULT_CONFIGRELOADERCPU | 100m | false | - |
| VM_VMAGENTDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - |
| VM_VMSINGLEDEFAULT_IMAGE | victoriametrics/victoria-metrics | false | - |
| VM_VMSINGLEDEFAULT_VERSION | v1.100.1 | false | - |
| VM_VMSINGLEDEFAULT_VERSION | v1.101.0 | false | - |
| VM_VMSINGLEDEFAULT_PORT | 8429 | false | - |
| VM_VMSINGLEDEFAULT_USEDEFAULTRESOURCES | true | false | - |
| VM_VMSINGLEDEFAULT_RESOURCE_LIMIT_MEM | 1500Mi | false | - |
@ -53,14 +53,14 @@ menu:
| VM_VMSINGLEDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - |
| VM_VMCLUSTERDEFAULT_USEDEFAULTRESOURCES | true | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_IMAGE | victoriametrics/vmselect | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_VERSION | v1.100.1-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_VERSION | v1.101.0-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_PORT | 8481 | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_LIMIT_MEM | 1000Mi | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_LIMIT_CPU | 500m | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_REQUEST_MEM | 500Mi | false | - |
| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_REQUEST_CPU | 100m | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_IMAGE | victoriametrics/vmstorage | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VERSION | v1.100.1-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VERSION | v1.101.0-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VMINSERTPORT | 8400 | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VMSELECTPORT | 8401 | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_PORT | 8482 | false | - |
@ -69,7 +69,7 @@ menu:
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_RESOURCE_REQUEST_MEM | 500Mi | false | - |
| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_RESOURCE_REQUEST_CPU | 250m | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_IMAGE | victoriametrics/vminsert | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_VERSION | v1.100.1-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_VERSION | v1.101.0-cluster | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_PORT | 8480 | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_RESOURCE_LIMIT_MEM | 500Mi | false | - |
| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_RESOURCE_LIMIT_CPU | 500m | false | - |
@ -88,7 +88,7 @@ menu:
| VM_VMALERTMANAGER_RESOURCE_REQUEST_CPU | 30m | false | - |
| VM_DISABLESELFSERVICESCRAPECREATION | false | false | - |
| VM_VMBACKUP_IMAGE | victoriametrics/vmbackupmanager | false | - |
| VM_VMBACKUP_VERSION | v1.100.1-enterprise | false | - |
| VM_VMBACKUP_VERSION | v1.101.0-enterprise | false | - |
| VM_VMBACKUP_PORT | 8300 | false | - |
| VM_VMBACKUP_USEDEFAULTRESOURCES | true | false | - |
| VM_VMBACKUP_RESOURCE_LIMIT_MEM | 500Mi | false | - |
@ -97,7 +97,7 @@ menu:
| VM_VMBACKUP_RESOURCE_REQUEST_CPU | 150m | false | - |
| VM_VMBACKUP_LOGLEVEL | INFO | false | - |
| VM_VMAUTHDEFAULT_IMAGE | victoriametrics/vmauth | false | - |
| VM_VMAUTHDEFAULT_VERSION | v1.100.1 | false | - |
| VM_VMAUTHDEFAULT_VERSION | v1.101.0 | false | - |
| VM_VMAUTHDEFAULT_CONFIGRELOADIMAGE | quay.io/prometheus-operator/prometheus-config-reloader:v0.68.0 | false | - |
| VM_VMAUTHDEFAULT_PORT | 8427 | false | - |
| VM_VMAUTHDEFAULT_USEDEFAULTRESOURCES | true | false | - |

View file

@ -250,7 +250,7 @@ scrape_configs:
# If node-exporter containers have another name in your Kubernetes cluster,
# then adjust the regex value accordingly.
#
- soruce_labels: [__meta_kubernetes_pod_container_name]
- source_labels: [__meta_kubernetes_pod_container_name]
regex: node-exporter
action: keep

View file

@ -35,6 +35,7 @@ supports the following Prometheus-compatible service discovery options for Prome
* `nomad_sd_configs` is for discovering and scraping targets registered in [HashiCorp Nomad](https://www.nomadproject.io/). See [these docs](#nomad_sd_configs).
* `openstack_sd_configs` is for discovering and scraping OpenStack targets. See [these docs](#openstack_sd_configs).
* `static_configs` is for scraping statically defined targets. See [these docs](#static_configs).
* `vultr_sd_configs` is for discovering and scraping [Vultr](https://www.vultr.com/) targets. See [these docs](#vultr_sd_configs).
* `yandexcloud_sd_configs` is for discovering and scraping [Yandex Cloud](https://cloud.yandex.com/en/) targets. See [these docs](#yandexcloud_sd_configs).
Note that the `refresh_interval` option isn't supported for these scrape configs. Use the corresponding `-promscrape.*CheckInterval`
@ -1498,6 +1499,79 @@ scrape_configs:
See [these examples](https://docs.victoriametrics.com/scrape_config_examples/#static-configs) on how to configure scraping for static targets.
## vultr_sd_configs
Vultr SD configuration discovers scrape targets from [Vultr](https://www.vultr.com/) Instances.
Configuration example:
```yaml
scrape_configs:
- job_name: vultr
vultr_sd_configs:
# bearer_token is a Bearer token to send in every HTTP API request during service discovery (mandatory).
# See: https://my.vultr.com/settings/#settingsapi
- bearer_token: "..."
# Vultr provides query arguments to filter instances.
# See: https://www.vultr.com/api/#tag/instances
# label is an optional query arguments to filter instances by label.
#
# label: "..."
# main_ip is an optional query arguments to filter instances by main ip address.
#
# main_ip: "..."
# region is an optional query arguments to filter instances by region id.
#
# region: "..."
# firewall_group_id is an optional query arguments to filter instances by firewall group id.
#
# firewall_group_id: "..."
# hostname is an optional query arguments to filter instances by hostname.
#
# hostname: "..."
# port is an optional port to scrape metrics from.
# By default, port 80 is used.
#
# port: ...
# Additional HTTP API client options can be specified here.
# See https://docs.victoriametrics.com/sd_configs.html#http-api-client-options
```
Each discovered target has an [`__address__`](https://docs.victoriametrics.com/relabeling.html#how-to-modify-scrape-urls-in-targets) label set
to `<FQDN>:<port>`, where FQDN is discovered instance address and `<port>` is the port from the `vultr_sd_configs` (default port is `80`).
The following meta labels are available on discovered targets during [relabeling](https://docs.victoriametrics.com/vmagent.html#relabeling):
* `__meta_vultr_instance_id`: A unique ID for the VPS Instance.
* `__meta_vultr_instance_label`: The user-supplied label for this instance.
* `__meta_vultr_instance_os`: The [Operating System name](https://www.vultr.com/api/#operation/list-os).
* `__meta_vultr_instance_os_id`: The [Operating System id](https://www.vultr.com/api/#operation/list-os) used by this instance.
* `__meta_vultr_instance_region`: The [Region id](https://www.vultr.com/api/#operation/list-regions) where the Instance is located.
* `__meta_vultr_instance_plan`: A unique ID for the Plan.
* `__meta_vultr_instance_main_ip`: The main IPv4 address.
* `__meta_vultr_instance_internal_ip`: The internal IP used by this instance, if set. Only relevant when a VPC is attached.
* `__meta_vultr_instance_main_ipv6`: The main IPv6 network address.
* `__meta_vultr_instance_hostname`: The hostname for this instance.
* `__meta_vultr_instance_server_status`: The server health status, which could be `none`, `locked`, `installingbooting`, `ok`.
* `__meta_vultr_instance_vcpu_count`: Number of vCPUs.
* `__meta_vultr_instance_ram_mb`: The amount of RAM in MB.
* `__meta_vultr_instance_allowed_bandwidth_gb`: Monthly bandwidth quota in GB.
* `__meta_vultr_instance_disk_gb`: The size of the disk in GB.
* `__meta_vultr_instance_features`: "auto_backups", "ipv6", "ddos_protection".
* `__meta_vultr_instance_tags`: Tags to apply to the instance.
The list of discovered Vultr targets is refreshed at the interval, which can be configured via `-promscrape.vultrSDCheckInterval` command-line flag, default: 30s.
## yandexcloud_sd_configs
[Yandex Cloud](https://cloud.yandex.com/en/) SD configurations allow retrieving scrape targets from accessible folders.

View file

@ -19,7 +19,7 @@ The aggregation is applied to all the metrics received via any [supported data i
and/or scraped from [Prometheus-compatible targets](https://docs.victoriametrics.com/#how-to-scrape-prometheus-exporters-such-as-node-exporter)
after applying all the configured [relabeling stages](https://docs.victoriametrics.com/vmagent/#relabeling).
By default stream aggregation ignores timestamps associated with the input [samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples).
By default, stream aggregation ignores timestamps associated with the input [samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples).
It expects that the ingested samples have timestamps close to the current time. See [how to ignore old samples](#ignoring-old-samples).
Stream aggregation can be configured via the following command-line flags:
@ -82,7 +82,7 @@ The online de-duplication uses the same logic as [`-dedup.minScrapeInterval` com
## Ignoring old samples
By default all the input samples are taken into account during stream aggregation. If samples with old timestamps outside the current [aggregation interval](#stream-aggregation-config)
By default, all the input samples are taken into account during stream aggregation. If samples with old timestamps outside the current [aggregation interval](#stream-aggregation-config)
must be ignored, then the following options can be used:
- To pass `-remoteWrite.streamAggr.ignoreOldSamples` command-line flag to [vmagent](https://docs.victoriametrics.com/vmagent/)
@ -92,9 +92,25 @@ must be ignored, then the following options can be used:
- To set `ignore_old_samples: true` option at the particular [aggregation config](#stream-aggregation-config).
This enables ignoring old samples for that particular aggregation config.
## Ignore aggregation intervals on start
Stream aggregation may yield inaccurate results if it processes incomplete data. This issue can arise when data is
received from clients that maintain a queue of unsent data, such as Prometheus or vmagent. If the queue isn't fully
cleared within the aggregation `interval`, only a portion of the time series may be processed, leading to distorted
calculations. To mitigate this, consider the following options:
- Set `-remoteWrite.streamAggr.ignoreFirstIntervals=<intervalsCount>` command-line flag to [vmagent](https://docs.victoriametrics.com/vmagent/)
or `-streamAggr.ignoreFirstIntervals=<intervalsCount>` command-line flag to [single-node VictoriaMetrics](https://docs.victoriametrics.com/)
to skip first `<intervalsCount>` [aggregation intervals](#stream-aggregation-config)
from persisting to the storage. It is expected that all incomplete or queued data will be processed during
specified `<intervalsCount>` and all subsequent aggregation intervals will produce correct data.
- Set `ignore_first_intervals: <intervalsCount>` option individually per [aggregation config](#stream-aggregation-config).
This enables ignoring first `<intervalsCount>` aggregation intervals for that particular aggregation config.
## Flush time alignment
By default the time for aggregated data flush is aligned by the `interval` option specified in [aggregate config](#stream-aggregation-config).
By default, the time for aggregated data flush is aligned by the `interval` option specified in [aggregate config](#stream-aggregation-config).
For example:
- if `interval: 1m` is set, then the aggregated data is flushed to the storage at the end of every minute
- if `interval: 1h` is set, then the aggregated data is flushed to the storage at the end of every hour
@ -117,7 +133,7 @@ Stream aggregation can be used in the following cases:
### Statsd alternative
Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) alternative in the following cases:
Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) drop-in replacement in the following cases:
* [Counting input samples](#counting-input-samples)
* [Summing input metrics](#summing-input-metrics)
@ -125,9 +141,6 @@ Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) alt
* [Histograms over input metrics](#histograms-over-input-metrics)
* [Aggregating histograms](#aggregating-histograms)
Currently, streaming aggregation is available only for [supported data ingestion protocols](https://docs.victoriametrics.com/#how-to-import-time-series-data)
and not available for [Statsd metrics format](https://github.com/statsd/statsd/blob/master/docs/metric_types.md).
### Recording rules alternative
Sometimes [alerting queries](https://docs.victoriametrics.com/vmalert/#alerting-rules) may require non-trivial amounts of CPU, RAM,
@ -582,9 +595,6 @@ some_counter:5m_increase / 5m
This is similar to `rate(some_counter[5m])`.
Please note, opposite to [rate](https://docs.victoriametrics.com/metricsql/#rate), `increase` aggregations can be
combined safely afterwards. This is helpful when the aggregation is calculated by more than one vmagent.
Aggregating irregular and sporadic metrics (received from [Lambdas](https://aws.amazon.com/lambda/)
or [Cloud Functions](https://cloud.google.com/functions)) can be controlled via [staleness_interval](#staleness) option.
@ -875,7 +885,7 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server-
# Samples are de-duplicated on a per-series basis. See https://docs.victoriametrics.com/keyconcepts/#time-series
# and https://docs.victoriametrics.com/#deduplication
# The deduplication is performed after input_relabel_configs relabeling is applied.
# By default the deduplication is disabled unless -remoteWrite.streamAggr.dedupInterval or -streamAggr.dedupInterval
# By default, the deduplication is disabled unless -remoteWrite.streamAggr.dedupInterval or -streamAggr.dedupInterval
# command-line flags are set.
#
# dedup_interval: 30s
@ -892,7 +902,7 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server-
# staleness_interval: 2m
# no_align_flush_to_interval disables aligning of flush times for the aggregated data to multiples of interval.
# By default flush times for the aggregated data is aligned to multiples of interval.
# By default, flush times for the aggregated data is aligned to multiples of interval.
# For example:
# - if `interval: 1m` is set, then flushes happen at the end of every minute,
# - if `interval: 1h` is set, then flushes happen at the end of every hour
@ -922,16 +932,23 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server-
# keep_metric_names instructs keeping the original metric names for the aggregated samples.
# This option can be set only if outputs list contains only a single output.
# By default a special suffix is added to original metric names in the aggregated samples.
# By default, a special suffix is added to original metric names in the aggregated samples.
# See https://docs.victoriametrics.com/stream-aggregation/#output-metric-names
#
# keep_metric_names: false
# ignore_old_samples instructs ignoring input samples with old timestamps outside the current aggregation interval.
# See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples
# See also -streamAggr.ignoreOldSamples command-line flag.
#
# ignore_old_samples: false
# ignore_first_intervals instructs ignoring first N aggregation intervals after process start.
# See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start
# See also -remoteWrite.streamAggr.ignoreFirstIntervals or -streamAggr.ignoreFirstIntervals
#
# ignore_first_intervals: false
# drop_input_labels instructs dropping the given labels from input samples.
# The labels' dropping is performed before input_relabel_configs are applied.
# This also means that the labels are dropped before de-duplication ( https://docs.victoriametrics.com/stream-aggregation/#deduplication )
@ -1004,7 +1021,7 @@ These issues can be be fixed in the following ways:
- By increasing the `interval` option at [stream aggregation config](#stream-aggregation-config), so it covers the expected
delays in data ingestion pipelines.
- By specifying the `staleness_interval` option at [stream aggregation config](#stream-aggregation-config), so it covers the expected
delays in data ingestion pipelines. By default the `staleness_interval` equals to `2 x interval`.
delays in data ingestion pipelines. By default, the `staleness_interval` equals to `2 x interval`.
### High resource usage

View file

@ -107,6 +107,7 @@ additionally to pull-based Prometheus-compatible targets' scraping:
* DataDog "submit metrics" API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-datadog-agent).
* InfluxDB line protocol via `http://<vmagent>:8429/write`. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf).
* Graphite plaintext protocol if `-graphiteListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-graphite-compatible-agents-such-as-statsd).
* Statsd plaintext protocol if `-statsdListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-statsd-compatible-clients).
* OpenTelemetry http API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#sending-data-via-opentelemetry).
* NewRelic API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-newrelic-agent).
* OpenTSDB telnet and http protocols if `-opentsdbListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-opentsdb-compatible-agents).
@ -1186,7 +1187,7 @@ If you have suggestions for improvements or have found a bug - please open an is
with `-remoteWrite.maxDiskUsagePerURL` command-line flag.
If you don't want to send all the buffered data from the directory to remote storage then simply stop `vmagent` and delete the directory.
* If `vmagent` runs on a host with slow persistent storage, which cannot keep up with the volume of processed samples, then is is possible to disable
* If `vmagent` runs on a host with slow persistent storage, which cannot keep up with the volume of processed samples, then is possible to disable
the persistent storage with `-remoteWrite.disableOnDiskQueue` command-line flag. See [these docs](#disabling-on-disk-persistence) for more details.
* By default `vmagent` masks `-remoteWrite.url` with `secret-url` values in logs and at `/metrics` page because
@ -1707,6 +1708,10 @@ See the docs at https://docs.victoriametrics.com/vmagent/ .
Whether to use proxy protocol for connections accepted at -graphiteListenAddr . See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt
-graphiteTrimTimestamp duration
Trim timestamps for Graphite data to this duration. Minimum practical duration is 1s. Higher duration (i.e. 1m) may be used for reducing disk space usage for timestamp data (default 1s)
-statsdListenAddr string
TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. See also -statsdListenAddr.useProxyProtocol
-statsdListenAddr.useProxyProtocol
Whether to use proxy protocol for connections accepted at -statsdListenAddr . See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt
-http.connTimeout duration
Incoming connections to -httpListenAddr are closed after the configured timeout. This may help evenly spreading load among a cluster of services behind TCP-level load balancer. Zero value disables closing of incoming connections (default 2m0s)
-http.disableResponseCompression
@ -1977,6 +1982,8 @@ See the docs at https://docs.victoriametrics.com/vmagent/ .
Whether to suppress scrape errors logging. The last error for each target is always available at '/targets' page even if scrape errors logging is suppressed. See also -promscrape.suppressScrapeErrorsDelay
-promscrape.suppressScrapeErrorsDelay duration
The delay for suppressing repeated scrape errors logging per each scrape targets. This may be used for reducing the number of log lines related to scrape errors. See also -promscrape.suppressScrapeErrors
-promscrape.vultrSDCheckInterval duration
Interval for checking for changes in Vultr. This works only if vultr_sd_configs is configured in '-promscrape.config' file. See https://docs.victoriametrics.com/sd_configs.html#vultr_sd_configs for details (default 30s)
-promscrape.yandexcloudSDCheckInterval duration
Interval for checking for changes in Yandex Cloud API. This works only if yandexcloud_sd_configs is configured in '-promscrape.config' file. See https://docs.victoriametrics.com/sd_configs/#yandexcloud_sd_configs for details (default 30s)
-pushmetrics.disableCompression
@ -2050,10 +2057,14 @@ See the docs at https://docs.victoriametrics.com/vmagent/ .
Optional path to bearer token file to use for the corresponding -remoteWrite.url. The token is re-read from the file every second
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-remoteWrite.disableOnDiskQueue
Whether to disable storing pending data to -remoteWrite.tmpDataPath when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence .See also -remoteWrite.dropSamplesOnOverload
-remoteWrite.dropSamplesOnOverload
Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence
-remoteWrite.disableOnDiskQueue array
Whether to disable storing pending data to -remoteWrite.tmpDataPath when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence .See also -remoteWrite.dropSamplesOnOverload
Supports array of values separated by comma or specified via multiple flags.
Empty values are set to false.
-remoteWrite.dropSamplesOnOverload array
Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence
Supports array of values separated by comma or specified via multiple flags.
Empty values are set to false.
-remoteWrite.flushInterval duration
Interval for flushing the data to remote storage. This option takes effect only when less than 10K data points per second are pushed to -remoteWrite.url (default 1s)
-remoteWrite.forcePromProto array
@ -2166,6 +2177,8 @@ See the docs at https://docs.victoriametrics.com/vmagent/ .
Whether to drop all the input samples after the aggregation with -remoteWrite.streamAggr.config. By default, only aggregates samples are dropped, while the remaining samples are written to the corresponding -remoteWrite.url . See also -remoteWrite.streamAggr.keepInput and https://docs.victoriametrics.com/stream-aggregation/
Supports array of values separated by comma or specified via multiple flags.
Empty values are set to false.
-remoteWrite.streamAggr.ignoreFirstIntervals int
Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after vmagent restarts. It could be caused by receiving unordered delayed data from clients pushing data into the vmagent. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start
-remoteWrite.streamAggr.ignoreOldSamples array
Whether to ignore input samples with old timestamps outside the current aggregation interval for the corresponding -remoteWrite.streamAggr.config . See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples
Supports array of values separated by comma or specified via multiple flags.

View file

@ -902,6 +902,25 @@ max(vmalert_alerting_rules_last_evaluation_series_fetched) by(group, alertname)
See more details [here](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4039).
This feature is available only if vmalert is using VictoriaMetrics v1.90 or higher as a datasource.
### Series with the same labelset
vmalert can produce the following error message:
```
result contains metrics with the same labelset during evaluation
```
The error means there is a collision between [time series](https://docs.victoriametrics.com/keyConcepts.html#time-series)
during evaluation.
For example, a rule with `expr: {__name__=~"vmalert_alerts_.*"} > 0` returns two distinct time series in response:
```
{__name__="vmalert_alerts_pending",job="vmalert",alertname="HostContextSwitching"} 12
{__name__="vmalert_alerts_firing",job="vmalert",alertname="HostContextSwitching"} 0
```
As label `__name__` will be dropped during evaluation, leads to duplicated time series.
To fix this, one could use function like [label_replace](https://docs.victoriametrics.com/metricsql/#label_replace) to preserve the distinct labelset.
## mTLS protection
By default `vmalert` accepts http requests at `8880` port (this port can be changed via `-httpListenAddr` command-line flags),
@ -1363,13 +1382,10 @@ The shortlist of configuration flags is the following:
-rule="dir/*.yaml" -rule="/*.yaml" -rule="gcs://vmalert-rules/tenant_%{TENANT_ID}/prod".
-rule="dir/**/*.yaml". Includes all the .yaml files in "dir" subfolders recursively.
Rule files may contain %{ENV_VAR} placeholders, which are substituted by the corresponding env vars.
Enterprise version of vmalert supports S3 and GCS paths to rules.
For example: gs://bucket/path/to/rules, s3://bucket/path/to/rules
S3 and GCS paths support only matching by prefix, e.g. s3://bucket/dir/rule_ matches
all files with prefix rule_ in folder dir.
See https://docs.victoriametrics.com/vmalert/#reading-rules-from-object-storage
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-rule.evalDelay time
@ -1387,7 +1403,6 @@ The shortlist of configuration flags is the following:
-rule.templates="dir/*.tpl" -rule.templates="/*.tpl". Relative path to all .tpl files in "dir" folder,
absolute path to all .tpl files in root.
-rule.templates="dir/**/*.tpl". Includes all the .tpl files in "dir" subfolders recursively.
Supports an array of values separated by comma or specified via multiple flags.
Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces.
-rule.updateEntriesLimit int

View file

@ -0,0 +1,173 @@
package statsd
import (
"errors"
"io"
"net"
"strings"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/netutil"
"github.com/VictoriaMetrics/metrics"
)
var (
writeRequestsTCP = metrics.NewCounter(`vm_ingestserver_requests_total{type="statsd", name="write", net="tcp"}`)
writeErrorsTCP = metrics.NewCounter(`vm_ingestserver_request_errors_total{type="statsd", name="write", net="tcp"}`)
writeRequestsUDP = metrics.NewCounter(`vm_ingestserver_requests_total{type="statsd", name="write", net="udp"}`)
writeErrorsUDP = metrics.NewCounter(`vm_ingestserver_request_errors_total{type="statsd", name="write", net="udp"}`)
)
// Server accepts Statsd plaintext lines over TCP and UDP.
type Server struct {
addr string
lnTCP net.Listener
lnUDP net.PacketConn
wg sync.WaitGroup
cm ingestserver.ConnsMap
}
// MustStart starts statsd server on the given addr.
//
// The incoming connections are processed with insertHandler.
//
// If useProxyProtocol is set to true, then the incoming connections are accepted via proxy protocol.
// See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt
//
// MustStop must be called on the returned server when it is no longer needed.
func MustStart(addr string, useProxyProtocol bool, insertHandler func(r io.Reader) error) *Server {
logger.Infof("starting TCP Statsd server at %q", addr)
lnTCP, err := netutil.NewTCPListener("statsd", addr, useProxyProtocol, nil)
if err != nil {
logger.Fatalf("cannot start TCP Statsd server at %q: %s", addr, err)
}
logger.Infof("starting UDP Statsd server at %q", addr)
lnUDP, err := net.ListenPacket(netutil.GetUDPNetwork(), addr)
if err != nil {
logger.Fatalf("cannot start UDP Statsd server at %q: %s", addr, err)
}
s := &Server{
addr: addr,
lnTCP: lnTCP,
lnUDP: lnUDP,
}
s.cm.Init("statsd")
s.wg.Add(1)
go func() {
defer s.wg.Done()
s.serveTCP(insertHandler)
logger.Infof("stopped TCP Statsd server at %q", addr)
}()
s.wg.Add(1)
go func() {
defer s.wg.Done()
s.serveUDP(insertHandler)
logger.Infof("stopped UDP Statsd server at %q", addr)
}()
return s
}
// MustStop stops the server.
func (s *Server) MustStop() {
logger.Infof("stopping TCP Statsd server at %q...", s.addr)
if err := s.lnTCP.Close(); err != nil {
logger.Errorf("cannot close TCP Statsd server: %s", err)
}
logger.Infof("stopping UDP Statsd server at %q...", s.addr)
if err := s.lnUDP.Close(); err != nil {
logger.Errorf("cannot close UDP Statsd server: %s", err)
}
s.cm.CloseAll(0)
s.wg.Wait()
logger.Infof("TCP and UDP Statsd servers at %q have been stopped", s.addr)
}
func (s *Server) serveTCP(insertHandler func(r io.Reader) error) {
var wg sync.WaitGroup
for {
c, err := s.lnTCP.Accept()
if err != nil {
var ne net.Error
if errors.As(err, &ne) {
if ne.Temporary() {
logger.Errorf("statsd: temporary error when listening for TCP addr %q: %s", s.lnTCP.Addr(), err)
time.Sleep(time.Second)
continue
}
if strings.Contains(err.Error(), "use of closed network connection") {
break
}
logger.Fatalf("unrecoverable error when accepting TCP Statsd connections: %s", err)
}
logger.Fatalf("unexpected error when accepting TCP Statsd connections: %s", err)
}
if !s.cm.Add(c) {
_ = c.Close()
break
}
wg.Add(1)
go func() {
defer func() {
s.cm.Delete(c)
_ = c.Close()
wg.Done()
}()
writeRequestsTCP.Inc()
if err := insertHandler(c); err != nil {
writeErrorsTCP.Inc()
logger.Errorf("error in TCP Statsd conn %q<->%q: %s", c.LocalAddr(), c.RemoteAddr(), err)
}
}()
}
wg.Wait()
}
func (s *Server) serveUDP(insertHandler func(r io.Reader) error) {
gomaxprocs := cgroup.AvailableCPUs()
var wg sync.WaitGroup
for i := 0; i < gomaxprocs; i++ {
wg.Add(1)
go func() {
defer wg.Done()
var bb bytesutil.ByteBuffer
bb.B = bytesutil.ResizeNoCopyNoOverallocate(bb.B, 64*1024)
for {
bb.Reset()
bb.B = bb.B[:cap(bb.B)]
n, addr, err := s.lnUDP.ReadFrom(bb.B)
if err != nil {
writeErrorsUDP.Inc()
var ne net.Error
if errors.As(err, &ne) {
if ne.Temporary() {
logger.Errorf("statsd: temporary error when listening for UDP addr %q: %s", s.lnUDP.LocalAddr(), err)
time.Sleep(time.Second)
continue
}
if strings.Contains(err.Error(), "use of closed network connection") {
break
}
}
logger.Errorf("cannot read Statsd UDP data: %s", err)
continue
}
bb.B = bb.B[:n]
writeRequestsUDP.Inc()
if err := insertHandler(bb.NewReader()); err != nil {
writeErrorsUDP.Inc()
logger.Errorf("error in UDP Statsd conn %q<->%q: %s", s.lnUDP.LocalAddr(), addr, err)
continue
}
}
}()
}
wg.Wait()
}

View file

@ -167,7 +167,7 @@ var rawItemsShardsPerTable = func() int {
return cpus * multiplier
}()
const maxBlocksPerShard = 256
var maxBlocksPerShard = 256
func (riss *rawItemsShards) init() {
riss.shards = make([]rawItemsShard, rawItemsShardsPerTable)

View file

@ -176,6 +176,66 @@ func TestTableCreateSnapshotAt(t *testing.T) {
_ = os.RemoveAll(path)
}
func TestTableAddItemsConcurrentStress(t *testing.T) {
const path = "TestTableAddItemsConcurrentStress"
if err := os.RemoveAll(path); err != nil {
t.Fatalf("cannot remove %q: %s", path, err)
}
defer func() {
_ = os.RemoveAll(path)
}()
rawItemsShardsPerTableOrig := rawItemsShardsPerTable
maxBlocksPerShardOrig := maxBlocksPerShard
rawItemsShardsPerTable = 10
maxBlocksPerShard = 3
defer func() {
rawItemsShardsPerTable = rawItemsShardsPerTableOrig
maxBlocksPerShard = maxBlocksPerShardOrig
}()
var flushes atomic.Uint64
flushCallback := func() {
flushes.Add(1)
}
prepareBlock := func(data []byte, items []Item) ([]byte, []Item) {
return data, items
}
blocksNeeded := rawItemsShardsPerTable * maxBlocksPerShard * 10
testAddItems := func(tb *Table) {
itemsBatch := make([][]byte, 0)
for j := 0; j < blocksNeeded; j++ {
item := bytes.Repeat([]byte{byte(j)}, maxInmemoryBlockSize-10)
itemsBatch = append(itemsBatch, item)
}
tb.AddItems(itemsBatch)
}
var isReadOnly atomic.Bool
tb := MustOpenTable(path, flushCallback, prepareBlock, &isReadOnly)
testAddItems(tb)
// Verify items count after pending items flush.
tb.DebugFlush()
if flushes.Load() == 0 {
t.Fatalf("unexpected zero flushes")
}
var m TableMetrics
tb.UpdateMetrics(&m)
if n := m.TotalItemsCount(); n != uint64(blocksNeeded) {
t.Fatalf("unexpected itemsCount; got %d; want %v", n, blocksNeeded)
}
tb.MustClose()
// Re-open the table and make sure itemsCount remains the same.
testReopenTable(t, path, blocksNeeded)
}
func TestTableAddItemsConcurrent(t *testing.T) {
const path = "TestTableAddItemsConcurrent"
if err := os.RemoveAll(path); err != nil {

View file

@ -63,7 +63,7 @@ func MustOpenFastQueue(path, name string, maxInmemoryBlocks int, maxPendingBytes
return float64(n)
})
pendingBytes := fq.GetPendingBytes()
logger.Infof("opened fast persistent queue at %q with maxInmemoryBlocks=%d, it contains %d pending bytes", path, maxInmemoryBlocks, pendingBytes)
logger.Infof("opened fast persistent queue at %q with maxInmemoryBlocks=%d isPQDisabled=%t, it contains %d pending bytes", path, maxInmemoryBlocks, isPQDisabled, pendingBytes)
return fq
}

View file

@ -10,9 +10,10 @@ import (
type WriteRequest struct {
// Timeseries is a list of time series in the given WriteRequest
Timeseries []TimeSeries
labelsPool []Label
exemplarLabelsPool []Label
samplesPool []Sample
exemplarsPool []Exemplar
}
// Reset resets wr for subsequent re-use.
@ -29,11 +30,33 @@ func (wr *WriteRequest) Reset() {
}
wr.labelsPool = labelsPool[:0]
exemplarLabelsPool := wr.exemplarLabelsPool
for i := range exemplarLabelsPool {
exemplarLabelsPool[i] = Label{}
}
wr.labelsPool = labelsPool[:0]
samplesPool := wr.samplesPool
for i := range samplesPool {
samplesPool[i] = Sample{}
}
wr.samplesPool = samplesPool[:0]
exemplarsPool := wr.exemplarsPool
for i := range exemplarsPool {
exemplarsPool[i] = Exemplar{}
}
wr.exemplarsPool = exemplarsPool[:0]
}
// Exemplar is an exemplar
type Exemplar struct {
// Labels a list of labels that uniquely identifies exemplar
// Optional, can be empty.
Labels []Label
// Value: the value of the exemplar
Value float64
// timestamp is in ms format, see model/timestamp/timestamp.go for
// conversion from time.Time to Prometheus timestamp.
Timestamp int64
}
// TimeSeries is a timeseries.
@ -43,6 +66,7 @@ type TimeSeries struct {
// Samples is a list of samples for the given TimeSeries
Samples []Sample
Exemplars []Exemplar
}
// Sample is a timeseries sample.
@ -74,7 +98,10 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) {
// }
tss := wr.Timeseries
labelsPool := wr.labelsPool
exemplarLabelsPool := wr.exemplarLabelsPool
samplesPool := wr.samplesPool
exemplarsPool := wr.exemplarsPool
var fc easyproto.FieldContext
for len(src) > 0 {
src, err = fc.NextField(src)
@ -93,7 +120,7 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) {
tss = append(tss, TimeSeries{})
}
ts := &tss[len(tss)-1]
labelsPool, samplesPool, err = ts.unmarshalProtobuf(data, labelsPool, samplesPool)
labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, err = ts.unmarshalProtobuf(data, labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool)
if err != nil {
return fmt.Errorf("cannot unmarshal timeseries: %w", err)
}
@ -102,28 +129,31 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) {
wr.Timeseries = tss
wr.labelsPool = labelsPool
wr.samplesPool = samplesPool
wr.exemplarsPool = exemplarsPool
return nil
}
func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesPool []Sample) ([]Label, []Sample, error) {
func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, exemplarLabelsPool []Label, samplesPool []Sample, exemplarsPool []Exemplar) ([]Label, []Label, []Sample, []Exemplar, error) {
// message TimeSeries {
// repeated Label labels = 1;
// repeated Sample samples = 2;
// repeated Exemplar exemplars = 3
// }
labelsPoolLen := len(labelsPool)
samplesPoolLen := len(samplesPool)
exemplarsPoolLen := len(exemplarsPool)
var fc easyproto.FieldContext
for len(src) > 0 {
var err error
src, err = fc.NextField(src)
if err != nil {
return labelsPool, samplesPool, fmt.Errorf("cannot read the next field: %w", err)
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the next field: %w", err)
}
switch fc.FieldNum {
case 1:
data, ok := fc.MessageData()
if !ok {
return labelsPool, samplesPool, fmt.Errorf("cannot read label data")
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read label data")
}
if len(labelsPool) < cap(labelsPool) {
labelsPool = labelsPool[:len(labelsPool)+1]
@ -132,12 +162,12 @@ func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesP
}
label := &labelsPool[len(labelsPool)-1]
if err := label.unmarshalProtobuf(data); err != nil {
return labelsPool, samplesPool, fmt.Errorf("cannot unmarshal label: %w", err)
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal label: %w", err)
}
case 2:
data, ok := fc.MessageData()
if !ok {
return labelsPool, samplesPool, fmt.Errorf("cannot read the sample data")
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the sample data")
}
if len(samplesPool) < cap(samplesPool) {
samplesPool = samplesPool[:len(samplesPool)+1]
@ -146,15 +176,78 @@ func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesP
}
sample := &samplesPool[len(samplesPool)-1]
if err := sample.unmarshalProtobuf(data); err != nil {
return labelsPool, samplesPool, fmt.Errorf("cannot unmarshal sample: %w", err)
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal sample: %w", err)
}
case 3:
data, ok := fc.MessageData()
if !ok {
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the exemplar data")
}
if len(exemplarsPool) < cap(exemplarsPool) {
exemplarsPool = exemplarsPool[:len(exemplarsPool)+1]
} else {
exemplarsPool = append(exemplarsPool, Exemplar{})
}
exemplar := &exemplarsPool[len(exemplarsPool)-1]
if exemplarLabelsPool, err = exemplar.unmarshalProtobuf(data, exemplarLabelsPool); err != nil {
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal exemplar: %w", err)
}
}
}
ts.Labels = labelsPool[labelsPoolLen:]
ts.Samples = samplesPool[samplesPoolLen:]
return labelsPool, samplesPool, nil
ts.Exemplars = exemplarsPool[exemplarsPoolLen:]
return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, nil
}
func (exemplar *Exemplar) unmarshalProtobuf(src []byte, labelsPool []Label) ([]Label, error) {
// message Exemplar {
// repeated Label Labels = 1;
// float64 Value = 2;
// int64 Timestamp = 3;
// }
var fc easyproto.FieldContext
labelsPoolLen := len(labelsPool)
for len(src) > 0 {
var err error
src, err = fc.NextField(src)
if err != nil {
return labelsPool, fmt.Errorf("cannot read the next field: %w", err)
}
switch fc.FieldNum {
case 1:
data, ok := fc.MessageData()
if !ok {
return labelsPool, fmt.Errorf("cannot read label data")
}
if len(labelsPool) < cap(labelsPool) {
labelsPool = labelsPool[:len(labelsPool)+1]
} else {
labelsPool = append(labelsPool, Label{})
}
label := &labelsPool[len(labelsPool)-1]
if err := label.unmarshalProtobuf(data); err != nil {
return labelsPool, fmt.Errorf("cannot unmarshal label: %w", err)
}
case 2:
value, ok := fc.Double()
if !ok {
return labelsPool, fmt.Errorf("cannot read exemplar value")
}
exemplar.Value = value
case 3:
timestamp, ok := fc.Int64()
if !ok {
return labelsPool, fmt.Errorf("cannot read exemplar timestamp")
}
exemplar.Timestamp = timestamp
}
}
exemplar.Labels = labelsPool[labelsPoolLen:]
return labelsPool, nil
}
func (lbl *Label) unmarshalProtobuf(src []byte) (err error) {
// message Label {
// string name = 1;

View file

@ -36,9 +36,25 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) {
Timestamp: sample.Timestamp,
})
}
var exemplars []prompbmarshal.Exemplar
for _, exemplar := range ts.Exemplars {
exemplarLabels := make([]prompbmarshal.Label, len(exemplar.Labels))
for i, label := range exemplar.Labels {
exemplarLabels[i] = prompbmarshal.Label{
Name: label.Name,
Value: label.Value,
}
}
exemplars = append(exemplars, prompbmarshal.Exemplar{
Labels: exemplarLabels,
Value: exemplar.Value,
Timestamp: exemplar.Timestamp,
})
}
wrm.Timeseries = append(wrm.Timeseries, prompbmarshal.TimeSeries{
Labels: labels,
Samples: samples,
Exemplars: exemplars,
})
}
dataResult := wrm.MarshalProtobuf(nil)
@ -121,6 +137,19 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) {
Timestamp: 18939432423,
},
},
Exemplars: []prompbmarshal.Exemplar{
{
Labels: []prompbmarshal.Label{
{Name: "trace-id",
Value: "123456",
},
{Name: "log_id",
Value: "987664"},
},
Value: 12345.6,
Timestamp: 456,
},
},
},
}
data = wrm.MarshalProtobuf(data[:0])
@ -153,6 +182,18 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) {
Timestamp: 18939432423,
},
},
Exemplars: []prompbmarshal.Exemplar{
{
Labels: []prompbmarshal.Label{
{
Name: "trace-id",
Value: "123456",
},
},
Value: 12345.6,
Timestamp: 456,
},
},
},
{
Labels: []prompbmarshal.Label{
@ -166,6 +207,22 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) {
Value: 9873,
},
},
Exemplars: []prompbmarshal.Exemplar{
{
Labels: []prompbmarshal.Label{
{
Name: "trace-id",
Value: "123456",
},
{
Name: "log_id",
Value: "987654",
},
},
Value: 12345.6,
Timestamp: 456,
},
},
},
}
data = wrm.MarshalProtobuf(data[:0])

View file

@ -36,6 +36,22 @@ func TestWriteRequestMarshalProtobuf(t *testing.T) {
Timestamp: 18939432423,
},
},
Exemplars: []prompbmarshal.Exemplar{
{
Labels: []prompbmarshal.Label{
{
Name: "trace-id",
Value: "123456",
},
{
Name: "log_id",
Value: "987654",
},
},
Value: 12345.6,
Timestamp: 456,
},
},
},
},
}
@ -64,9 +80,25 @@ func TestWriteRequestMarshalProtobuf(t *testing.T) {
Timestamp: sample.Timestamp,
})
}
var exemplars []prompbmarshal.Exemplar
for _, exemplar := range ts.Exemplars {
exemplarLabels := make([]prompbmarshal.Label, len(exemplar.Labels))
for i, label := range exemplar.Labels {
exemplarLabels[i] = prompbmarshal.Label{
Name: label.Name,
Value: label.Value,
}
}
exemplars = append(exemplars, prompbmarshal.Exemplar{
Labels: exemplarLabels,
Value: exemplar.Value,
Timestamp: exemplar.Timestamp,
})
}
wrm.Timeseries = append(wrm.Timeseries, prompbmarshal.TimeSeries{
Labels: labels,
Samples: samples,
Exemplars: exemplars,
})
}
dataResult := wrm.MarshalProtobuf(nil)

View file

@ -13,10 +13,70 @@ type Sample struct {
Timestamp int64
}
type Exemplar struct {
// Optional, can be empty.
Labels []Label
Value float64
// timestamp is in ms format, see model/timestamp/timestamp.go for
// conversion from time.Time to Prometheus timestamp.
Timestamp int64
}
func (m *Exemplar) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
if m.Timestamp != 0 {
i = encodeVarint(dAtA, i, uint64(m.Timestamp))
i--
dAtA[i] = 0x18
}
if m.Value != 0 {
i -= 8
binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value))))
i--
dAtA[i] = 0x11
}
if len(m.Labels) > 0 {
for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Labels[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarint(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
}
func (m *Exemplar) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.Labels) > 0 {
for _, e := range m.Labels {
l = e.Size()
n += 1 + l + sov(uint64(l))
}
}
if m.Value != 0 {
n += 9
}
if m.Timestamp != 0 {
n += 1 + sov(uint64(m.Timestamp))
}
return n
}
// TimeSeries represents samples and labels for a single time series.
type TimeSeries struct {
Labels []Label
Samples []Sample
Exemplars []Exemplar
}
type Label struct {
@ -42,6 +102,16 @@ func (m *Sample) MarshalToSizedBuffer(dst []byte) (int, error) {
func (m *TimeSeries) MarshalToSizedBuffer(dst []byte) (int, error) {
i := len(dst)
for j := len(m.Exemplars) - 1; j >= 0; j-- {
size, err := m.Exemplars[j].MarshalToSizedBuffer(dst[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarint(dst, i, uint64(size))
i--
dst[i] = 0x1a
}
for j := len(m.Samples) - 1; j >= 0; j-- {
size, err := m.Samples[j].MarshalToSizedBuffer(dst[:i])
if err != nil {
@ -109,6 +179,10 @@ func (m *TimeSeries) Size() (n int) {
l := e.Size()
n += 1 + l + sov(uint64(l))
}
for _, e := range m.Exemplars {
l := e.Size()
n += 1 + l + sov(uint64(l))
}
return n
}

View file

@ -30,6 +30,7 @@ var (
streamParse = flag.Bool("promscrape.streamParse", false, "Whether to enable stream parsing for metrics obtained from scrape targets. This may be useful "+
"for reducing memory usage when millions of metrics are exposed per each scrape target. "+
"It is possible to set 'stream_parse: true' individually per each 'scrape_config' section in '-promscrape.config' for fine-grained control")
scrapeExemplars = flag.Bool("promscrape.scrapeExemplars", false, "Whether to enable scraping of exemplars from scrape targets.")
)
type client struct {
@ -107,6 +108,12 @@ func (c *client) ReadData(dst *bytesutil.ByteBuffer) error {
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/608 for details.
// Do not bloat the `Accept` header with OpenMetrics shit, since it looks like dead standard now.
req.Header.Set("Accept", "text/plain;version=0.0.4;q=1,*/*;q=0.1")
// We set to support exemplars to be compatible with Prometheus Exposition format which uses
// Open Metrics Specification
// See https://github.com/prometheus/docs/blob/main/content/docs/instrumenting/exposition_formats.md#openmetrics-text-format
if *scrapeExemplars {
req.Header.Set("Accept", "application/openmetrics-text")
}
// Set X-Prometheus-Scrape-Timeout-Seconds like Prometheus does, since it is used by some exporters such as PushProx.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1179#issuecomment-813117162
req.Header.Set("X-Prometheus-Scrape-Timeout-Seconds", c.scrapeTimeoutSecondsStr)

View file

@ -36,6 +36,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/kuma"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/nomad"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/openstack"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/vultr"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/yandexcloud"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
@ -308,6 +309,7 @@ type ScrapeConfig struct {
NomadSDConfigs []nomad.SDConfig `yaml:"nomad_sd_configs,omitempty"`
OpenStackSDConfigs []openstack.SDConfig `yaml:"openstack_sd_configs,omitempty"`
StaticConfigs []StaticConfig `yaml:"static_configs,omitempty"`
VultrConfigs []vultr.SDConfig `yaml:"vultr_configs,omitempty"`
YandexCloudSDConfigs []yandexcloud.SDConfig `yaml:"yandexcloud_sd_configs,omitempty"`
// These options are supported only by lib/promscrape.
@ -388,6 +390,9 @@ func (sc *ScrapeConfig) mustStop() {
for i := range sc.OpenStackSDConfigs {
sc.OpenStackSDConfigs[i].MustStop()
}
for i := range sc.VultrConfigs {
sc.VultrConfigs[i].MustStop()
}
}
// FileSDConfig represents file-based service discovery config.
@ -745,6 +750,16 @@ func (cfg *Config) getOpenStackSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork {
return cfg.getScrapeWorkGeneric(visitConfigs, "openstack_sd_config", prev)
}
// getVultrSDScrapeWork returns `vultr_sd_configs` ScrapeWork from cfg.
func (cfg *Config) getVultrSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork {
visitConfigs := func(sc *ScrapeConfig, visitor func(sdc targetLabelsGetter)) {
for i := range sc.VultrConfigs {
visitor(&sc.VultrConfigs[i])
}
}
return cfg.getScrapeWorkGeneric(visitConfigs, "vultr_sd_config", prev)
}
// getYandexCloudSDScrapeWork returns `yandexcloud_sd_configs` ScrapeWork from cfg.
func (cfg *Config) getYandexCloudSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork {
visitConfigs := func(sc *ScrapeConfig, visitor func(sdc targetLabelsGetter)) {

View file

@ -0,0 +1,76 @@
package vultr
import (
"fmt"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
)
// apiConfig contains config for API server.
type apiConfig struct {
c *discoveryutils.Client
port int
listParams
}
// listParams is the query params of vultr ListInstance API.
type listParams struct {
// paging params are not exposed to user, they will be filled
// dynamically during request. See `getInstances`.
// perPage int
// cursor string
// API query params for filtering.
label string
mainIP string
region string
firewallGroupID string
hostname string
}
// getAPIConfig get or create API config from configMap.
func getAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) {
v, err := configMap.Get(sdc, func() (interface{}, error) { return newAPIConfig(sdc, baseDir) })
if err != nil {
return nil, err
}
return v.(*apiConfig), nil
}
// newAPIConfig create API Config.
func newAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) {
port := sdc.Port
if port == 0 {
port = 80
}
// See: https://www.vultr.com/api/
apiServer := "https://api.vultr.com"
ac, err := sdc.HTTPClientConfig.NewConfig(baseDir)
if err != nil {
return nil, fmt.Errorf("cannot parse auth config: %w", err)
}
proxyAC, err := sdc.ProxyClientConfig.NewConfig(baseDir)
if err != nil {
return nil, fmt.Errorf("cannot parse proxy auth config: %w", err)
}
c, err := discoveryutils.NewClient(apiServer, ac, sdc.ProxyURL, proxyAC, &sdc.HTTPClientConfig)
if err != nil {
return nil, fmt.Errorf("cannot create client for %q: %w", apiServer, err)
}
cfg := &apiConfig{
c: c,
port: port,
listParams: listParams{
label: sdc.Label,
mainIP: sdc.MainIP,
region: sdc.Region,
firewallGroupID: sdc.FirewallGroupID,
hostname: sdc.Hostname,
},
}
return cfg, nil
}

View file

@ -0,0 +1,16 @@
package vultr
import (
"testing"
)
func TestNewAPIConfig(t *testing.T) {
sdc := &SDConfig{}
baseDir := "."
_, err := newAPIConfig(sdc, baseDir)
if err != nil {
t.Errorf("newAPIConfig failed with, err: %v", err)
return
}
}

View file

@ -0,0 +1,109 @@
package vultr
import (
"encoding/json"
"fmt"
"net/url"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
)
// ListInstanceResponse is the response structure of Vultr ListInstance API.
type ListInstanceResponse struct {
Instances []Instance `json:"instances"`
Meta *Meta `json:"Meta"`
}
// Instance represents Vultr Instance (VPS).
// See: https://github.com/vultr/govultr/blob/5125e02e715ae6eb3ce854f0e7116c7ce545a710/instance.go#L81
type Instance struct {
ID string `json:"id"`
Os string `json:"os"`
RAM int `json:"ram"`
Disk int `json:"disk"`
MainIP string `json:"main_ip"`
VCPUCount int `json:"vcpu_count"`
Region string `json:"region"`
ServerStatus string `json:"server_status"`
AllowedBandwidth int `json:"allowed_bandwidth"`
V6MainIP string `json:"v6_main_ip"`
Hostname string `json:"hostname"`
Label string `json:"label"`
InternalIP string `json:"internal_ip"`
OsID int `json:"os_id"`
Features []string `json:"features"`
Plan string `json:"plan"`
Tags []string `json:"tags"`
// The following fields are defined in the response but are not used during service discovery.
//DefaultPassword string `json:"default_password,omitempty"`
//DateCreated string `json:"date_created"`
//Status string `json:"status"`
//PowerStatus string `json:"power_status"`
//NetmaskV4 string `json:"netmask_v4"`
//GatewayV4 string `json:"gateway_v4"`
//V6Network string `json:"v6_network"`
//V6NetworkSize int `json:"v6_network_size"`
//// Deprecated: Tag should no longer be used. Instead, use Tags.
//Tag string `json:"tag"`
//KVM string `json:"kvm"`
//AppID int `json:"app_id"`
//ImageID string `json:"image_id"`
//FirewallGroupID string `json:"firewall_group_id"`
//UserScheme string `json:"user_scheme"`
}
// Meta represents the available pagination information
type Meta struct {
Total int `json:"total"`
Links *Links
}
// Links represent the next/previous cursor in your pagination calls
type Links struct {
Next string `json:"next"`
Prev string `json:"prev"`
}
// getInstances retrieve instance from Vultr HTTP API.
func getInstances(cfg *apiConfig) ([]Instance, error) {
var instances []Instance
// prepare GET params
params := url.Values{}
params.Set("per_page", "100")
params.Set("label", cfg.label)
params.Set("main_ip", cfg.mainIP)
params.Set("region", cfg.region)
params.Set("firewall_group_id", cfg.firewallGroupID)
params.Set("hostname", cfg.hostname)
// send request to vultr API
for {
// See: https://www.vultr.com/api/#tag/instances/operation/list-instances
path := fmt.Sprintf("/v2/instances?%s", params.Encode())
resp, err := cfg.c.GetAPIResponse(path)
if err != nil {
logger.Errorf("get response from vultr failed, path:%s, err: %v", path, err)
return nil, err
}
var listInstanceResp ListInstanceResponse
if err = json.Unmarshal(resp, &listInstanceResp); err != nil {
logger.Errorf("unmarshal response from vultr failed, err: %v", err)
return nil, err
}
instances = append(instances, listInstanceResp.Instances...)
if listInstanceResp.Meta != nil && listInstanceResp.Meta.Links != nil && listInstanceResp.Meta.Links.Next != "" {
// if `next page` is available, set the cursor param and request again.
params.Set("cursor", listInstanceResp.Meta.Links.Next)
} else {
// otherwise exit the loop
break
}
}
return instances, nil
}

View file

@ -0,0 +1,301 @@
package vultr
import (
"errors"
"reflect"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
)
// TestGetInstances runs general test cases for GetInstances
func TestGetInstances(t *testing.T) {
testCases := []struct {
name string
apiResponse string
apiError bool
expectError bool
expectResponse []Instance
}{
{
name: "success response",
apiResponse: mockListInstanceSuccessResp,
apiError: false,
expectError: false,
expectResponse: expectSuccessInstances,
},
{
name: "failed response",
apiResponse: mockListInstanceFailedResp,
apiError: true,
expectError: true,
expectResponse: nil,
},
}
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
// Prepare a mock Vultr server.
mockServer := newMockVultrServer(func() ([]byte, error) {
var e error
if tt.apiError {
e = errors.New("mock error")
}
return []byte(tt.apiResponse), e
})
// Prepare a discovery HTTP client who calls mock server.
client, _ := discoveryutils.NewClient(mockServer.URL, nil, nil, nil, &promauth.HTTPClientConfig{})
cfg := &apiConfig{
c: client,
}
// execute `getInstances`
instances, err := getInstances(cfg)
// evaluate test result
if tt.expectError != (err != nil) {
t.Errorf("getInstances expect (error != nil): %t, got error: %v", tt.expectError, err)
}
if !reflect.DeepEqual(tt.expectResponse, instances) {
t.Errorf("getInstances expect result: %v, got: %v", tt.expectResponse, instances)
}
})
}
}
// TestGetInstancesPaging run test cases for response with multiple pages.
func TestGetInstancesPaging(t *testing.T) {
// Prepare a mock Vultr server.
// requestCount control the mock response for different page request.
requestCount := 0
mockServer := newMockVultrServer(func() ([]byte, error) {
// for the 1st request, response with `next` cursor
if requestCount == 0 {
requestCount++
return []byte(mockListInstanceSuccessPage0Resp), nil
}
// for the 2nd+ request, response with `prev` cursor and empty `next`.
return []byte(mockListInstanceSuccessPage1Resp), nil
})
// Prepare a discovery HTTP client who calls mock server.
client, _ := discoveryutils.NewClient(mockServer.URL, nil, nil, nil, &promauth.HTTPClientConfig{})
cfg := &apiConfig{
c: client,
}
// execute `getInstances`
instances, err := getInstances(cfg)
// evaluate test result
if err != nil {
t.Errorf("getInstances expect error: %v, got error: %v", nil, err)
}
if !reflect.DeepEqual(expectSuccessPagingInstances, instances) {
t.Errorf("getInstances expect result: %v, got: %v", expectSuccessPagingInstances, instances)
}
}
// ------------ Test dataset ------------
var (
// mockListInstanceSuccessResp is crawled from a real-world response of ListInstance API
// with sensitive info removed/modified.
mockListInstanceSuccessResp = `{
"instances": [{
"id": "fake-id-07f7-4b68-88ac-fake-id",
"os": "Ubuntu 22.04 x64",
"ram": 1024,
"disk": 25,
"main_ip": "64.176.84.27",
"vcpu_count": 1,
"region": "sgp",
"plan": "vc2-1c-1gb",
"date_created": "2024-04-05T05:41:28+00:00",
"status": "active",
"allowed_bandwidth": 1,
"netmask_v4": "255.255.254.0",
"gateway_v4": "64.176.63.2",
"power_status": "running",
"server_status": "installingbooting",
"v6_network": "2002:18f0:4100:263a::",
"v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
"v6_network_size": 64,
"label": "vultr-sd",
"internal_ip": "",
"kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string",
"hostname": "vultr-sd",
"tag": "",
"tags": [],
"os_id": 1743,
"app_id": 0,
"image_id": "",
"firewall_group_id": "",
"features": ["ipv6"],
"user_scheme": "root"
}],
"meta": {
"total": 1,
"links": {
"next": "",
"prev": ""
}
}
}`
expectSuccessInstances = []Instance{
{
ID: "fake-id-07f7-4b68-88ac-fake-id",
Os: "Ubuntu 22.04 x64",
RAM: 1024,
Disk: 25,
MainIP: "64.176.84.27",
VCPUCount: 1,
Region: "sgp",
Plan: "vc2-1c-1gb",
AllowedBandwidth: 1,
ServerStatus: "installingbooting",
V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
Label: "vultr-sd",
InternalIP: "",
Hostname: "vultr-sd",
Tags: []string{},
OsID: 1743,
Features: []string{"ipv6"},
},
}
)
var (
mockListInstanceFailedResp = `{"error":"Invalid API token.","status":401}`
)
var (
// mockListInstanceSuccessPage0Resp contains `next` cursor
mockListInstanceSuccessPage0Resp = `{
"instances": [{
"id": "fake-id-07f7-4b68-88ac-fake-id",
"os": "Ubuntu 22.04 x64",
"ram": 1024,
"disk": 25,
"main_ip": "64.176.84.27",
"vcpu_count": 1,
"region": "sgp",
"plan": "vc2-1c-1gb",
"date_created": "2024-04-05T05:41:28+00:00",
"status": "active",
"allowed_bandwidth": 1,
"netmask_v4": "255.255.254.0",
"gateway_v4": "64.176.63.2",
"power_status": "running",
"server_status": "installingbooting",
"v6_network": "2002:18f0:4100:263a::",
"v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
"v6_network_size": 64,
"label": "vultr-sd",
"internal_ip": "",
"kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string",
"hostname": "vultr-sd",
"tag": "",
"tags": [],
"os_id": 1743,
"app_id": 0,
"image_id": "",
"firewall_group_id": "",
"features": ["ipv6"],
"user_scheme": "root"
}],
"meta": {
"total": 2,
"links": {
"next": "fake-cursor-string",
"prev": ""
}
}
}`
// mockListInstanceSuccessPage1Resp contains `prev` cursor
mockListInstanceSuccessPage1Resp = `{
"instances": [{
"id": "fake-id-07f7-4b68-88ac-fake-id",
"os": "Ubuntu 22.04 x64",
"ram": 1024,
"disk": 25,
"main_ip": "64.176.84.27",
"vcpu_count": 1,
"region": "sgp",
"plan": "vc2-1c-1gb",
"date_created": "2024-04-05T05:41:28+00:00",
"status": "active",
"allowed_bandwidth": 1,
"netmask_v4": "255.255.254.0",
"gateway_v4": "64.176.63.2",
"power_status": "running",
"server_status": "installingbooting",
"v6_network": "2002:18f0:4100:263a::",
"v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
"v6_network_size": 64,
"label": "vultr-sd",
"internal_ip": "",
"kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string",
"hostname": "vultr-sd",
"tag": "",
"tags": [],
"os_id": 1743,
"app_id": 0,
"image_id": "",
"firewall_group_id": "",
"features": ["ipv6"],
"user_scheme": "root"
}],
"meta": {
"total": 2,
"links": {
"next": "",
"prev": "fake-cursor-string"
}
}
}`
expectSuccessPagingInstances = []Instance{
{
ID: "fake-id-07f7-4b68-88ac-fake-id",
Os: "Ubuntu 22.04 x64",
RAM: 1024,
Disk: 25,
MainIP: "64.176.84.27",
VCPUCount: 1,
Region: "sgp",
Plan: "vc2-1c-1gb",
AllowedBandwidth: 1,
ServerStatus: "installingbooting",
V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
Label: "vultr-sd",
InternalIP: "",
Hostname: "vultr-sd",
Tags: []string{},
OsID: 1743,
Features: []string{"ipv6"},
},
{
ID: "fake-id-07f7-4b68-88ac-fake-id",
Os: "Ubuntu 22.04 x64",
RAM: 1024,
Disk: 25,
MainIP: "64.176.84.27",
VCPUCount: 1,
Region: "sgp",
Plan: "vc2-1c-1gb",
AllowedBandwidth: 1,
ServerStatus: "installingbooting",
V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
Label: "vultr-sd",
InternalIP: "",
Hostname: "vultr-sd",
Tags: []string{},
OsID: 1743,
Features: []string{"ipv6"},
},
}
)

View file

@ -0,0 +1,40 @@
package vultr
import (
"fmt"
"net/http"
"net/http/httptest"
)
func newMockVultrServer(jsonResponse func() ([]byte, error)) *vultrServer {
rw := &vultrServer{}
rw.Server = httptest.NewServer(http.HandlerFunc(rw.handler))
rw.jsonResponse = jsonResponse
return rw
}
type vultrServer struct {
*httptest.Server
jsonResponse func() ([]byte, error)
}
func (rw *vultrServer) err(w http.ResponseWriter, err error) {
w.WriteHeader(http.StatusBadRequest)
w.Write([]byte(err.Error()))
}
func (rw *vultrServer) handler(w http.ResponseWriter, r *http.Request) {
if r.Method != http.MethodGet {
rw.err(w, fmt.Errorf("bad method %q", r.Method))
return
}
resp, err := rw.jsonResponse()
if err != nil {
rw.err(w, err)
return
}
w.Write(resp)
w.WriteHeader(http.StatusOK)
}

View file

@ -0,0 +1,105 @@
package vultr
import (
"flag"
"fmt"
"strconv"
"strings"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
)
const (
separator = ","
)
// SDCheckInterval defines interval for docker targets refresh.
var SDCheckInterval = flag.Duration("promscrape.vultrSDCheckInterval", 30*time.Second, "Interval for checking for changes in Vultr. "+
"This works only if vultr_sd_configs is configured in '-promscrape.config' file. "+
"See https://docs.victoriametrics.com/sd_configs.html#vultr_sd_configs for details")
// SDConfig represents service discovery config for Vultr.
// See: https://prometheus.io/docs/prometheus/latest/configuration/configuration/#vultr_sd_config
// Additional query params are supported, while Prometheus only supports `Port` and HTTP auth.
type SDConfig struct {
// API query params for filtering. All of them are optional.
// See: https://www.vultr.com/api/#tag/instances/operation/list-instances
Label string `yaml:"label,omitempty"`
MainIP string `yaml:"main_ip,omitempty"`
Region string `yaml:"region,omitempty"`
FirewallGroupID string `yaml:"firewall_group_id,omitempty"`
Hostname string `yaml:"hostname,omitempty"`
// The port to scrape metrics from. Default 80.
Port int `yaml:"port"`
// General HTTP / Auth configs.
HTTPClientConfig promauth.HTTPClientConfig `yaml:",inline"`
ProxyURL *proxy.URL `yaml:"proxy_url,omitempty"`
ProxyClientConfig promauth.ProxyClientConfig `yaml:",inline"`
// refresh_interval is obtained from `-promscrape.vultrSDCheckInterval` command-line option.
}
var configMap = discoveryutils.NewConfigMap()
// GetLabels returns gce labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
ac, err := getAPIConfig(sdc, baseDir)
if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err)
}
instances, err := getInstances(ac)
if err != nil {
return nil, err
}
return getInstanceLabels(instances, ac.port), nil
}
// MustStop stops further usage for sdc.
func (sdc *SDConfig) MustStop() {
configMap.Delete(sdc)
}
// getInstanceLabels returns labels for vultr instances obtained from the given cfg
func getInstanceLabels(instances []Instance, port int) []*promutils.Labels {
ms := make([]*promutils.Labels, 0, len(instances))
for _, instance := range instances {
m := promutils.NewLabels(18)
m.Add("__address__", discoveryutils.JoinHostPort(instance.MainIP, port))
m.Add("__meta_vultr_instance_id", instance.ID)
m.Add("__meta_vultr_instance_label", instance.Label)
m.Add("__meta_vultr_instance_os", instance.Os)
m.Add("__meta_vultr_instance_os_id", strconv.Itoa(instance.OsID))
m.Add("__meta_vultr_instance_region", instance.Region)
m.Add("__meta_vultr_instance_plan", instance.Plan)
m.Add("__meta_vultr_instance_main_ip", instance.MainIP)
m.Add("__meta_vultr_instance_internal_ip", instance.InternalIP)
m.Add("__meta_vultr_instance_main_ipv6", instance.V6MainIP)
m.Add("__meta_vultr_instance_hostname", instance.Hostname)
m.Add("__meta_vultr_instance_server_status", instance.ServerStatus)
m.Add("__meta_vultr_instance_vcpu_count", strconv.Itoa(instance.VCPUCount))
m.Add("__meta_vultr_instance_ram_mb", strconv.Itoa(instance.RAM))
m.Add("__meta_vultr_instance_allowed_bandwidth_gb", strconv.Itoa(instance.AllowedBandwidth))
m.Add("__meta_vultr_instance_disk_gb", strconv.Itoa(instance.Disk))
// We surround the separated list with the separator as well. This way regular expressions
// in relabeling rules don't have to consider feature positions.
if len(instance.Features) > 0 {
features := separator + strings.Join(instance.Features, separator) + separator
m.Add("__meta_vultr_instance_features", features)
}
if len(instance.Tags) > 0 {
tags := separator + strings.Join(instance.Tags, separator) + separator
m.Add("__meta_vultr_instance_tags", tags)
}
ms = append(ms, m)
}
return ms
}

View file

@ -0,0 +1,96 @@
package vultr
import (
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
)
func TestGetInstanceLabels(t *testing.T) {
input := []Instance{
{
ID: "fake-id-07f7-4b68-88ac-fake-id",
Os: "Ubuntu 22.04 x64",
RAM: 1024,
Disk: 25,
MainIP: "64.176.84.27",
VCPUCount: 1,
Region: "sgp",
Plan: "vc2-1c-1gb",
AllowedBandwidth: 1,
ServerStatus: "installingbooting",
V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
Label: "vultr-sd",
InternalIP: "",
Hostname: "vultr-sd",
Tags: []string{"mock tags"},
OsID: 1743,
Features: []string{"ipv6"},
},
{
ID: "fake-id-07f7-4b68-88ac-fake-id",
Os: "Ubuntu 22.04 x64",
RAM: 1024,
Disk: 25,
MainIP: "64.176.84.27",
VCPUCount: 1,
Region: "sgp",
Plan: "vc2-1c-1gb",
AllowedBandwidth: 1,
ServerStatus: "installingbooting",
V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
Label: "vultr-sd",
InternalIP: "",
Hostname: "vultr-sd",
Tags: []string{"mock tags"},
OsID: 1743,
Features: []string{"ipv6"},
},
}
expect := []*promutils.Labels{
promutils.NewLabelsFromMap(map[string]string{
"__address__": "64.176.84.27:8080",
"__meta_vultr_instance_id": "fake-id-07f7-4b68-88ac-fake-id",
"__meta_vultr_instance_label": "vultr-sd",
"__meta_vultr_instance_os": "Ubuntu 22.04 x64",
"__meta_vultr_instance_os_id": "1743",
"__meta_vultr_instance_region": "sgp",
"__meta_vultr_instance_plan": "vc2-1c-1gb",
"__meta_vultr_instance_main_ip": "64.176.84.27",
"__meta_vultr_instance_internal_ip": "",
"__meta_vultr_instance_main_ipv6": "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
"__meta_vultr_instance_hostname": "vultr-sd",
"__meta_vultr_instance_server_status": "installingbooting",
"__meta_vultr_instance_vcpu_count": "1",
"__meta_vultr_instance_ram_mb": "1024",
"__meta_vultr_instance_allowed_bandwidth_gb": "1",
"__meta_vultr_instance_disk_gb": "25",
"__meta_vultr_instance_features": ",ipv6,",
"__meta_vultr_instance_tags": ",mock tags,",
}),
promutils.NewLabelsFromMap(map[string]string{
"__address__": "64.176.84.27:8080",
"__meta_vultr_instance_id": "fake-id-07f7-4b68-88ac-fake-id",
"__meta_vultr_instance_label": "vultr-sd",
"__meta_vultr_instance_os": "Ubuntu 22.04 x64",
"__meta_vultr_instance_os_id": "1743",
"__meta_vultr_instance_region": "sgp",
"__meta_vultr_instance_plan": "vc2-1c-1gb",
"__meta_vultr_instance_main_ip": "64.176.84.27",
"__meta_vultr_instance_internal_ip": "",
"__meta_vultr_instance_main_ipv6": "2002:18f0:4100:263a:5300:07ff:fdd7:691c",
"__meta_vultr_instance_hostname": "vultr-sd",
"__meta_vultr_instance_server_status": "installingbooting",
"__meta_vultr_instance_vcpu_count": "1",
"__meta_vultr_instance_ram_mb": "1024",
"__meta_vultr_instance_allowed_bandwidth_gb": "1",
"__meta_vultr_instance_disk_gb": "25",
"__meta_vultr_instance_features": ",ipv6,",
"__meta_vultr_instance_tags": ",mock tags,",
}),
}
labels := getInstanceLabels(input, 8080)
discoveryutils.TestEqualLabelss(t, labels, expect)
}

Some files were not shown because too many files have changed in this diff Show more