app/vmagent: initial implementation for vmagent

This commit is contained in:
Aliaksandr Valialkin 2020-02-23 13:35:47 +02:00
parent 4e905d6501
commit 04762344c6
162 changed files with 36290 additions and 693 deletions

1
.gitignore vendored
View File

@ -8,6 +8,7 @@
*.swp
/gocache-for-docker
/victoria-metrics-data
/vmagent-remotewrite-data
/vmstorage-data
/vmselect-cache
/package/temp-deb-*

View File

@ -11,7 +11,10 @@ endif
GO_BUILDINFO = -X '$(PKG_PREFIX)/lib/buildinfo.Version=$(APP_NAME)-$(shell date -u +'%Y%m%d-%H%M%S')-$(BUILDINFO_TAG)'
all: \
victoria-metrics-prod
victoria-metrics-prod \
vmagent-prod \
vmbackup-prod \
vmrestore-prod
include app/*/Makefile
include deployment/*/Makefile
@ -21,15 +24,18 @@ clean:
publish: \
publish-victoria-metrics \
publish-vmagent \
publish-vmbackup \
publish-vmrestore
package: \
package-victoria-metrics \
package-vmagent \
package-vmbackup \
package-vmrestore
vmutils: \
vmagent \
vmbackup \
vmrestore
@ -42,9 +48,10 @@ release-victoria-metrics: victoria-metrics-prod
sha256sum victoria-metrics-$(PKG_TAG).tar.gz > victoria-metrics-$(PKG_TAG)_checksums.txt
release-vmutils: \
vmagent-prod \
vmbackup-prod \
vmrestore-prod
cd bin && tar czf vmutils-$(PKG_TAG).tar.gz vmbackup-prod vmrestore-prod && \
cd bin && tar czf vmutils-$(PKG_TAG).tar.gz vmagent-prod vmbackup-prod vmrestore-prod && \
sha256sum vmutils-$(PKG_TAG).tar.gz > vmutils-$(PKG_TAG)_checksums.txt
pprof-cpu:
@ -70,6 +77,7 @@ errcheck: install-errcheck
errcheck -exclude=errcheck_excludes.txt ./app/vminsert/...
errcheck -exclude=errcheck_excludes.txt ./app/vmselect/...
errcheck -exclude=errcheck_excludes.txt ./app/vmstorage/...
errcheck -exclude=errcheck_excludes.txt ./app/vmagent/...
errcheck -exclude=errcheck_excludes.txt ./app/vmbackup/...
errcheck -exclude=errcheck_excludes.txt ./app/vmrestore/...
errcheck -exclude=errcheck_excludes.txt ./app/vmalert/...

View File

@ -53,7 +53,9 @@ Cluster version is available [here](https://github.com/VictoriaMetrics/VictoriaM
to S3 or GCS with [vmbackup](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/app/vmbackup/README.md) / [vmrestore](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/app/vmrestore/README.md).
See [this article](https://medium.com/@valyala/speeding-up-backups-for-big-time-series-databases-533c1a927883) for more details.
* Storage is protected from corruption on unclean shutdown (i.e. OOM, hardware reset or `kill -9`) thanks to [the storage architecture](https://medium.com/@valyala/how-victoriametrics-makes-instant-snapshots-for-multi-terabyte-time-series-data-e1f3fb0e0282).
* Supports metrics' ingestion and [backfilling](#backfilling) via the following protocols:
* Supports metrics' scraping, ingestion and [backfilling](#backfilling) via the following protocols:
* [Metrics from Prometheus exporters](https://github.com/prometheus/docs/blob/master/content/docs/instrumenting/exposition_formats.md#text-based-format)
such as [node_exporter](https://github.com/prometheus/node_exporter). See [these docs](#how-to-scrape-prometheus-exporters-such-as-node-exporter) for details.
* [Prometheus remote write API](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#remote_write)
* [InfluxDB line protocol](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf)
* [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)
@ -75,6 +77,7 @@ Cluster version is available [here](https://github.com/VictoriaMetrics/VictoriaM
- [Grafana setup](#grafana-setup)
- [How to upgrade VictoriaMetrics?](#how-to-upgrade-victoriametrics)
- [How to apply new config to VictoriaMetrics?](#how-to-apply-new-config-to-victoriametrics)
- [How to scrape Prometheus exporters such as node_exporter?](#how-to-scrape-prometheus-exporters-such-as-node-exporter)
- [How to send data from InfluxDB-compatible agents such as Telegraf?](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf)
- [How to send data from Graphite-compatible agents such as StatsD?](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd)
- [Querying Graphite data](#querying-graphite-data)
@ -238,6 +241,20 @@ Prometheus doesn't drop data during VictoriaMetrics restart.
See [this article](https://grafana.com/blog/2019/03/25/whats-new-in-prometheus-2.8-wal-based-remote-write/) for details.
### How to scrape Prometheus exporters such as [node-exporter](https://github.com/prometheus/node_exporter)?
VictoriaMetrics can be used as drop-in replacement for Prometheus for scraping targets configured in `prometheus.yml` config file according to [the specification](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#configuration-file).
Just set `-promscrape.config` command-line flag to the path to `prometheus.yml` config - and VictoriaMetrics should start scraping the configured targets.
Currently the following [scrape_config](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config) types are supported:
* [static_config](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#static_config)
* [file_sd_config](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#file_sd_config)
In the future other `*_sd_config` types will be supported.
See also [vmagent](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/app/vmagent/README.md), which can be used as drop-in replacement for Prometheus.
### How to send data from InfluxDB-compatible agents such as [Telegraf](https://www.influxdata.com/time-series-platform/telegraf/)?
Just use `http://<victoriametric-addr>:8428` url instead of InfluxDB url in agents' configs.

73
app/vmagent/Makefile Normal file
View File

@ -0,0 +1,73 @@
# All these commands must run from repository root.
vmagent:
APP_NAME=vmagent $(MAKE) app-local
vmagent-prod:
APP_NAME=vmagent $(MAKE) app-via-docker
vmagent-pure-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-pure
vmagent-amd64-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-amd64
vmagent-arm-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-arm
vmagent-arm64-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-arm64
vmagent-ppc64le-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-ppc64le
vmagent-386-prod:
APP_NAME=vmagent $(MAKE) app-via-docker-386
package-vmagent:
APP_NAME=vmagent $(MAKE) package-via-docker
package-vmagent-pure:
APP_NAME=vmagent $(MAKE) package-via-docker-pure
package-vmagent-amd64:
APP_NAME=vmagent $(MAKE) package-via-docker-amd64
package-vmagent-arm:
APP_NAME=vmagent $(MAKE) package-via-docker-arm
package-vmagent-arm64:
APP_NAME=vmagent $(MAKE) package-via-docker-arm64
package-vmagent-ppc64le:
APP_NAME=vmagent $(MAKE) package-via-docker-ppc64le
package-vmagent-386:
APP_NAME=vmagent $(MAKE) package-via-docker-386
publish-vmagent:
APP_NAME=vmagent $(MAKE) publish-via-docker
run-vmagent:
mkdir -p vmagent-data
DOCKER_OPTS='-v $(shell pwd)/vmagent-data:/vmagent-data' \
APP_NAME=vmagent \
$(MAKE) run-via-docker
vmagent-amd64:
CGO_ENABLED=1 GOOS=linux GOARCH=amd64 GO111MODULE=on go build -mod=vendor -ldflags "$(GO_BUILDINFO)" -o bin/vmagent-amd64 ./app/vmagent
vmagent-arm:
CGO_ENABLED=0 GOOS=linux GOARCH=arm GO111MODULE=on go build -mod=vendor -ldflags "$(GO_BUILDINFO)" -o bin/vmagent-arm ./app/vmagent
vmagent-arm64:
CGO_ENABLED=0 GOOS=linux GOARCH=arm64 GO111MODULE=on go build -mod=vendor -ldflags "$(GO_BUILDINFO)" -o bin/vmagent-arm64 ./app/vmagent
vmagent-ppc64le:
CGO_ENABLED=0 GOOS=linux GOARCH=ppc64le GO111MODULE=on go build -mod=vendor -ldflags "$(GO_BUILDINFO)" -o bin/vmagent-ppc64le ./app/vmagent
vmagent-386:
CGO_ENABLED=0 GOOS=linux GOARCH=386 GO111MODULE=on go build -mod=vendor -ldflags "$(GO_BUILDINFO)" -o bin/vmagent-386 ./app/vmagent
vmagent-pure:
APP_NAME=vmagent $(MAKE) app-local-pure

151
app/vmagent/README.md Normal file
View File

@ -0,0 +1,151 @@
## vmagent
`vmagent` collects metrics from various sources and pushes them to any remote storage for Prometheus
from [this list](https://prometheus.io/docs/operating/integrations/#remote-endpoints-and-storage).
The recommended remote storage is [VictoriaMetrics](https://github.com/VictoriaMetrics/VictoriaMetrics).
### Features
* Can be used as drop-in replacement for Prometheus for scraping targets such as [node_exporter](https://github.com/prometheus/node_exporter).
Just use `-promscrape.config=/path/to/prometheus.yml` command-line flag.
* Accepts data via all the ingestion protocols supported by VictoriaMetrics:
* Influx line protocol via `http://<vmagent>:8429/write`. See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/README.md#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf).
* JSON lines import protocol via `http://<vmagent>:8429/api/v1/import`. See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/README.md#how-to-import-time-series-data).
* Graphite plaintext protocol if `-graphiteListenAddr` command-line flag is set. See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/README.md#how-to-send-data-from-graphite-compatible-agents-such-as-statsd).
* OpenTSDB telnet and http protocols if `-opentsdbListenAddr` command-line flag is set. See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/README.md#how-to-send-data-from-opentsdb-compatible-agents).
* Prometheus remote write protocol via `http://<vmagent>:8429/api/v1/write`.
* Prometheus relabeling can be applied to all the collected metrics. See [these docs](#relabeling).
* Additional labels can be added to all the collected metrics before sending them to remote storage.
Just pass these labels to `-remoteWrite.label` command-line flag: `-remoteWrite.label="labelName=labelValue"`.
* Collected metrics can be sent simultaneously to multiple remote storage systems by providing multiple `-remoteWrite.url` args.
* Works in environments with unstable connections to remote storage. If the remote storage is unavailable, the collected metrics
are buffered at `-remoteWrite.tmpDataPath` until free space is available. The buffered metrics are sent to remote storage
as soon as connection to remote storage is recovered.
* Uses lower amounts of RAM, CPU, disk IO and network bandwidth comparing to Prometheus.
### Quick start
Just pass path the following flags to `vmagent` in order to start scraping Prometheus targets:
* `-promscrape.config` with the path to Prometheus config file (it is usually located at `/etc/prometheus/prometheus.yml`)
* `-remoteWrite.url` with the remote storage endpoint that accepts data over Prometheus remote_write API.
See [the list of supported remote storage systems](https://prometheus.io/docs/operating/integrations/#remote-endpoints-and-storage).
`vmagent` supports sending data to multiple remote storage systems in parallel. Just set multiple `-remoteWrite.url` args.
Example command line:
```
/path/to/vmagent -promscrape.config=/path/to/prometheus.yml -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
If you need collecting only Influx data, then the following command line would be enough:
```
/path/to/vmagent -remoteWrite.url=https://victoria-metrics-host:8428/api/v1/write
```
Then send Influx data to `http://vmagent-host:8429/write`. See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/README.md#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) for more details.
### How to collect metrics in Prometheus format?
`vmagent` can be used as Prometheus replacement for metrics' scraping if path to [Prometheus config file](https://prometheus.io/docs/prometheus/latest/configuration/configuration/)
path is passed to `-promscrape.config` command-line flag. This file is usually named `prometheus.yml`.
`vmagent` takes into account the following sections from [Prometheus config file](https://prometheus.io/docs/prometheus/latest/configuration/configuration/):
* `global`
* `scrape_configs`
All the other section are ignored, including [remote_write](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#remote_write) section.
Use `-remoteWrite.*` command-line flags instead for configuring remote write settings:
* `-remoteWrite.url` for pointing to remote storage. Data to remote storage can be sent either via HTTP or HTTPS. See `-remoteWrite.tls*` flags for details.
* `-remoteWrite.label` for adding labels to metrics before sending them to remote storage.
* `-remoteWrite.relabelConfig` for applying relabeling to metrics before sending them to remote storage.
The following scrape types in [scrape_config](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config) section are supported:
* `static_configs` - for scraping statically defined targets. See [these docs](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#static_config) for details.
* `file_sd_configs` - for scraping targets defined in external files aka file-based service discover.
See [these docs](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#file_sd_config) for details.
File feature requests at [our issue tracker](https://github.com/VictoriaMetrics/VictoriaMetrics/issues) if you need other service discovery mechanisms to be supported by `vmagent`.
### Adding labels to metrics
Labels can be added to metrics via the following mechanisms:
* Via `global -> external_labels` section in `-promscrape.config` file. These labels are added only to metrics scraped from targets configured in `-promscrape.config` file.
* Via `-remoteWrite.label` command-line flag. These labels are added to all the collected metrics before sending them to `-remoteWrite.url`.
### Relabeling
`vmagent` supports [Prometheus relabeling](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config).
Additionally it provides the following extra actions:
* `replace_all`: replaces all the occurences of `regex` in the values of `source_labels` with the `replacement`
* `labelmap_all`: replaces all the occurences of `regex` in all the labels with the `replacement`
The relabeling can be defined in the following places:
* At `scrape_config -> relabel_configs` section in `-promscrape.config` file. This relabeling is applied when parsing the file during `vmagent` startup
or during config reload after sending `SIGHUP` signal to `vmagent` via `kill -HUP`.
* At `scrape_config -> metric_relabel_configs` section in `-promscrape.config` file. This relabeling is applied to metrics after each scrape for configured targets.
Changes to this section can be applied after sending `SIGHUP` signal to `vmagent`.
* At `-remoteWrite.relabelConfig` file. This relabeling is aplied to all the collected metrics before sending them to `-remoteWrite.url`.
Read more about relabeling in the following articles:
* [Life of a label](https://www.robustperception.io/life-of-a-label)
* [Discarding targets and timeseries with relabeling](https://www.robustperception.io/relabelling-can-discard-targets-timeseries-and-alerts)
* [Dropping labels at scrape time](https://www.robustperception.io/dropping-metrics-at-scrape-time-with-prometheus)
* [Extracting labels from legacy metric names](https://www.robustperception.io/extracting-labels-from-legacy-metric-names)
* [relabel_configs vs metric_relabel_configs](https://www.robustperception.io/relabel_configs-vs-metric_relabel_configs)
### Monitoring
`vmagent` export various metrics in Prometheus exposition format at `/metrics` page. It is recommended setting up regular scraping of this page
either via `vmagent` itself or via Prometheus, so the exported metrics could be analyzed later.
### Troubleshooting
* It is recommended increasing the maximum number of open file in the system (`ulimit -n`) when scraping big number of targets,
since `vmagent` establishes at least a single TCP connection per each target.
* It is recommended increasing `-remoteWrite.queues` if `vmagent` collects more than 100K samples per second
and `vmagent_remotewrite_pending_data_bytes` metric exported by `vmagent` at `/metrics` page constantly grows.
* `vmagent` buffers scraped data at `-remoteWrite.tmpDataPath` directory until it is sent to `-remoteWrite.url`.
The directory can grow big when remote storage is unvailable during extended periods of time. If you don't want
sending all the data from the directory to remote storage, just stop `vmagent` and delete the directory.
### How to build from sources
It is recommended using [binary releases](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) - `vmagent` is located in `vmutils-*` archives there.
#### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.12.
2. Run `make vmagent` from the root folder of the repository.
It builds `vmagent` binary and puts it into the `bin` folder.
#### Production build
1. [Install docker](https://docs.docker.com/install/).
2. Run `make vmagent-prod` from the root folder of the repository.
It builds `vmagent-prod` binary and puts it into the `bin` folder.
#### Building docker images
Run `make package-vmagent`. It builds `victoriametrics/vmagent:<PKG_TAG>` docker image locally.
`<PKG_TAG>` is auto-generated image tag, which depends on source code in the repository.
The `<PKG_TAG>` may be manually set via `PKG_TAG=foobar make package-vmagent`.

View File

@ -0,0 +1,70 @@
package common
import (
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
// PushCtx is a context used for populating WriteRequest.
type PushCtx struct {
WriteRequest prompbmarshal.WriteRequest
// Labels contains flat list of all the labels used in WriteRequest.
Labels []prompbmarshal.Label
// Samples contains flat list of all the samples used in WriteRequest.
Samples []prompbmarshal.Sample
}
// Reset resets ctx.
func (ctx *PushCtx) Reset() {
tss := ctx.WriteRequest.Timeseries
for i := range tss {
ts := &tss[i]
ts.Labels = nil
ts.Samples = nil
}
ctx.WriteRequest.Timeseries = ctx.WriteRequest.Timeseries[:0]
labels := ctx.Labels
for i := range labels {
label := &labels[i]
label.Name = ""
label.Value = ""
}
ctx.Labels = ctx.Labels[:0]
ctx.Samples = ctx.Samples[:0]
}
// GetPushCtx returns PushCtx from pool.
//
// Call PutPushCtx when the ctx is no longer needed.
func GetPushCtx() *PushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*PushCtx)
}
return &PushCtx{}
}
}
// PutPushCtx returns ctx to the pool.
//
// ctx mustn't be used after returning to the pool.
func PutPushCtx(ctx *PushCtx) {
ctx.Reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *PushCtx, runtime.GOMAXPROCS(-1))

View File

@ -0,0 +1,8 @@
ARG certs_image
FROM $certs_image AS certs
FROM scratch
COPY --from=certs /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ca-certificates.crt
ARG src_binary
COPY $src_binary ./vmagent-prod
EXPOSE 8429
ENTRYPOINT ["/vmagent-prod"]

View File

@ -0,0 +1,65 @@
package graphite
import (
"io"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/graphite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="graphite"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="graphite"}`)
)
// InsertHandler processes remote write for graphite plaintext protocol.
//
// See https://graphite.readthedocs.io/en/latest/feeding-carbon.html#the-plaintext-protocol
func InsertHandler(r io.Reader) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(r, insertRows)
})
}
func insertRows(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
remotewrite.Push(&ctx.WriteRequest)
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return nil
}

View File

@ -0,0 +1,152 @@
package influx
import (
"flag"
"net/http"
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/influx"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
measurementFieldSeparator = flag.String("influxMeasurementFieldSeparator", "_", "Separator for '{measurement}{separator}{field_name}' metric name when inserted via Influx line protocol")
skipSingleField = flag.Bool("influxSkipSingleField", false, "Uses '{measurement}' instead of '{measurement}{separator}{field_name}' for metic name if Influx line contains only a single field")
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="influx"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="influx"}`)
)
// InsertHandler processes remote write for influx line protocol.
//
// See https://github.com/influxdata/influxdb/blob/4cbdc197b8117fee648d62e2e5be75c6575352f0/tsdb/README.md
func InsertHandler(req *http.Request) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertRows(db string, rows []parser.Row) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
rowsTotal := 0
tssDst := ctx.ctx.WriteRequest.Timeseries[:0]
labels := ctx.ctx.Labels[:0]
samples := ctx.ctx.Samples[:0]
commonLabels := ctx.commonLabels[:0]
buf := ctx.buf[:0]
for i := range rows {
r := &rows[i]
commonLabels = commonLabels[:0]
hasDBLabel := false
for j := range r.Tags {
tag := &r.Tags[j]
if tag.Key == "db" {
hasDBLabel = true
}
commonLabels = append(commonLabels, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
if len(db) > 0 && !hasDBLabel {
commonLabels = append(commonLabels, prompbmarshal.Label{
Name: "db",
Value: db,
})
}
ctx.metricGroupBuf = append(ctx.metricGroupBuf[:0], r.Measurement...)
skipFieldKey := len(r.Fields) == 1 && *skipSingleField
if len(ctx.metricGroupBuf) > 0 && !skipFieldKey {
ctx.metricGroupBuf = append(ctx.metricGroupBuf, *measurementFieldSeparator...)
}
for j := range r.Fields {
f := &r.Fields[j]
bufLen := len(buf)
buf = append(buf, ctx.metricGroupBuf...)
if !skipFieldKey {
buf = append(buf, f.Key...)
}
metricGroup := bytesutil.ToUnsafeString(buf[bufLen:])
labelsLen := len(labels)
labels = append(labels, prompbmarshal.Label{
Name: "__name__",
Value: metricGroup,
})
labels = append(labels, commonLabels...)
samples = append(samples, prompbmarshal.Sample{
Timestamp: r.Timestamp,
Value: f.Value,
})
tssDst = append(tssDst, prompbmarshal.TimeSeries{
Labels: labels[labelsLen:],
Samples: samples[len(samples)-1:],
})
}
rowsTotal += len(r.Fields)
}
ctx.buf = buf
ctx.ctx.WriteRequest.Timeseries = tssDst
ctx.ctx.Labels = labels
ctx.ctx.Samples = samples
ctx.commonLabels = commonLabels
remotewrite.Push(&ctx.ctx.WriteRequest)
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
return nil
}
type pushCtx struct {
ctx common.PushCtx
commonLabels []prompbmarshal.Label
metricGroupBuf []byte
buf []byte
}
func (ctx *pushCtx) reset() {
ctx.ctx.Reset()
commonLabels := ctx.commonLabels
for i := range commonLabels {
label := &commonLabels[i]
label.Name = ""
label.Value = ""
}
ctx.metricGroupBuf = ctx.metricGroupBuf[:0]
ctx.buf = ctx.buf[:0]
}
func getPushCtx() *pushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*pushCtx)
}
return &pushCtx{}
}
}
func putPushCtx(ctx *pushCtx) {
ctx.reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *pushCtx, runtime.GOMAXPROCS(-1))

149
app/vmagent/main.go Normal file
View File

@ -0,0 +1,149 @@
package main
import (
"flag"
"fmt"
"net/http"
"strings"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/graphite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/influx"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/opentsdb"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/buildinfo"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/envflag"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/httpserver"
graphiteserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/graphite"
opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb"
opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
httpListenAddr = flag.String("httpListenAddr", ":8429", "TCP address to listen for http connections")
graphiteListenAddr = flag.String("graphiteListenAddr", "", "TCP and UDP address to listen for Graphite plaintext data. Usually :2003 must be set. Doesn't work if empty")
opentsdbListenAddr = flag.String("opentsdbListenAddr", "", "TCP and UDP address to listen for OpentTSDB 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")
opentsdbHTTPListenAddr = flag.String("opentsdbHTTPListenAddr", "", "TCP address to listen for OpentTSDB HTTP put requests. Usually :4242 must be set. Doesn't work if empty")
)
var (
graphiteServer *graphiteserver.Server
opentsdbServer *opentsdbserver.Server
opentsdbhttpServer *opentsdbhttpserver.Server
)
func main() {
envflag.Parse()
buildinfo.Init()
logger.Init()
logger.Infof("starting vmagent at %q...", *httpListenAddr)
startTime := time.Now()
remotewrite.Init()
writeconcurrencylimiter.Init()
if len(*graphiteListenAddr) > 0 {
graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, graphite.InsertHandler)
}
if len(*opentsdbListenAddr) > 0 {
opentsdbServer = opentsdbserver.MustStart(*opentsdbListenAddr, opentsdb.InsertHandler, opentsdbhttp.InsertHandler)
}
if len(*opentsdbHTTPListenAddr) > 0 {
opentsdbhttpServer = opentsdbhttpserver.MustStart(*opentsdbHTTPListenAddr, opentsdbhttp.InsertHandler)
}
promscrape.Init(remotewrite.Push)
go httpserver.Serve(*httpListenAddr, requestHandler)
logger.Infof("started vmagent in %.3f seconds", time.Since(startTime).Seconds())
sig := procutil.WaitForSigterm()
logger.Infof("received signal %s", sig)
logger.Infof("gracefully shutting down webservice at %q", *httpListenAddr)
startTime = time.Now()
if err := httpserver.Stop(*httpListenAddr); err != nil {
logger.Fatalf("cannot stop the webservice: %s", err)
}
logger.Infof("successfully shut down the webservice in %.3f seconds", time.Since(startTime).Seconds())
promscrape.Stop()
if len(*graphiteListenAddr) > 0 {
graphiteServer.MustStop()
}
if len(*opentsdbListenAddr) > 0 {
opentsdbServer.MustStop()
}
if len(*opentsdbHTTPListenAddr) > 0 {
opentsdbhttpServer.MustStop()
}
remotewrite.Stop()
logger.Infof("successfully stopped vmagent in %.3f seconds", time.Since(startTime).Seconds())
}
func requestHandler(w http.ResponseWriter, r *http.Request) bool {
path := strings.Replace(r.URL.Path, "//", "/", -1)
switch path {
case "/api/v1/write":
prometheusWriteRequests.Inc()
if err := promremotewrite.InsertHandler(r); err != nil {
prometheusWriteErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", r.URL.Path, err)
return true
}
w.WriteHeader(http.StatusNoContent)
return true
case "/targets":
w.Header().Set("Content-Type", "text/plain")
promscrape.WriteHumanReadableTargetsStatus(w)
return true
case "/api/v1/import":
vmimportRequests.Inc()
if err := vmimport.InsertHandler(r); err != nil {
vmimportErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", r.URL.Path, err)
return true
}
w.WriteHeader(http.StatusNoContent)
return true
case "/write", "/api/v2/write":
influxWriteRequests.Inc()
if err := influx.InsertHandler(r); err != nil {
influxWriteErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", r.URL.Path, err)
return true
}
w.WriteHeader(http.StatusNoContent)
return true
case "/query":
// Emulate fake response for influx query.
// This is required for TSBS benchmark.
influxQueryRequests.Inc()
fmt.Fprintf(w, `{"results":[{"series":[{"values":[]}]}]}`)
return true
}
return false
}
var (
prometheusWriteRequests = metrics.NewCounter(`vmagent_http_requests_total{path="/api/v1/write", protocol="prometheus"}`)
prometheusWriteErrors = metrics.NewCounter(`vmagent_http_request_errors_total{path="/api/v1/write", protocol="prometheus"}`)
vmimportRequests = metrics.NewCounter(`vmagent_http_requests_total{path="/api/v1/import", protocol="vm"}`)
vmimportErrors = metrics.NewCounter(`vmagent_http_request_errors_total{path="/api/v1/import", protocol="vm"}`)
influxWriteRequests = metrics.NewCounter(`vmagent_http_requests_total{path="/write", protocol="influx"}`)
influxWriteErrors = metrics.NewCounter(`vmagent_http_request_errors_total{path="/write", protocol="influx"}`)
influxQueryRequests = metrics.NewCounter(`vmagent_http_requests_total{path="/query", protocol="influx"}`)
)

View File

@ -0,0 +1,65 @@
package opentsdb
import (
"io"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdb"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="opentsdb"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="opentsdb"}`)
)
// InsertHandler processes remote write for OpenTSDB put protocol.
//
// See http://opentsdb.net/docs/build/html/api_telnet/put.html
func InsertHandler(r io.Reader) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(r, insertRows)
})
}
func insertRows(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
remotewrite.Push(&ctx.WriteRequest)
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return nil
}

View File

@ -0,0 +1,64 @@
package opentsdbhttp
import (
"net/http"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="opentsdbhttp"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="opentsdbhttp"}`)
)
// InsertHandler processes HTTP OpenTSDB put requests.
// See http://opentsdb.net/docs/build/html/api_http/put.html
func InsertHandler(req *http.Request) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertRows(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
remotewrite.Push(&ctx.WriteRequest)
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return nil
}

View File

@ -0,0 +1,67 @@
package promremotewrite
import (
"net/http"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="promremotewrite"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="promremotewrite"}`)
)
// InsertHandler processes remote write for prometheus.
func InsertHandler(req *http.Request) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertRows(timeseries []prompb.TimeSeries) error {
ctx := common.GetPushCtx()
defer common.PutPushCtx(ctx)
rowsTotal := 0
tssDst := ctx.WriteRequest.Timeseries[:0]
labels := ctx.Labels[:0]
samples := ctx.Samples[:0]
for i := range timeseries {
ts := &timeseries[i]
labelsLen := len(labels)
for i := range ts.Labels {
label := &ts.Labels[i]
labels = append(labels, prompbmarshal.Label{
Name: bytesutil.ToUnsafeString(label.Name),
Value: bytesutil.ToUnsafeString(label.Value),
})
}
samplesLen := len(samples)
for i := range ts.Samples {
sample := &ts.Samples[i]
samples = append(samples, prompbmarshal.Sample{
Value: sample.Value,
Timestamp: sample.Timestamp,
})
}
tssDst = append(tssDst, prompbmarshal.TimeSeries{
Labels: labels[labelsLen:],
Samples: samples[samplesLen:],
})
rowsTotal += len(ts.Samples)
}
ctx.WriteRequest.Timeseries = tssDst
ctx.Labels = labels
ctx.Samples = samples
remotewrite.Push(&ctx.WriteRequest)
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
return nil
}

View File

@ -0,0 +1,267 @@
package remotewrite
import (
"crypto/tls"
"crypto/x509"
"encoding/base64"
"flag"
"fmt"
"io/ioutil"
"strings"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/netutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/persistentqueue"
"github.com/VictoriaMetrics/metrics"
"github.com/valyala/fasthttp"
)
var (
sendTimeout = flag.Duration("remoteWrite.sendTimeout", time.Minute, "Timeout for sending a single block of data to -remoteWrite.url")
tlsInsecureSkipVerify = flag.Bool("remoteWrite.tlsInsecureSkipVerify", false, "Whether to skip tls verification when connecting to -remoteWrite.url")
tlsCertFile = flag.String("remoteWrite.tlsCertFile", "", "Optional path to client-side TLS certificate file to use when connecting to -remoteWrite.url")
tlsKeyFile = flag.String("remoteWrite.tlsKeyFile", "", "Optional path to client-side TLS certificate key to use when connecting to -remoteWrite.url")
tlsCAFile = flag.String("remoteWrite.tlsCAFile", "", "Optional path to TLS CA file to use for verifying connections to -remoteWrite.url. "+
"By default system CA is used")
basicAuthUsername = flag.String("remoteWrite.basicAuth.username", "", "Optional basic auth username to use for -remoteWrite.url")
basicAuthPassword = flag.String("remoteWrite.basicAuth.password", "", "Optional basic auth password to use for -remoteWrite.url")
bearerToken = flag.String("remoteWrite.bearerToken", "", "Optional bearer auth token to use for -remoteWrite.url")
)
type client struct {
urlLabelValue string
remoteWriteURL string
host string
requestURI string
authHeader string
fq *persistentqueue.FastQueue
hc *fasthttp.HostClient
requestDuration *metrics.Histogram
requestsOKCount *metrics.Counter
errorsCount *metrics.Counter
retriesCount *metrics.Counter
wg sync.WaitGroup
stopCh chan struct{}
}
func newClient(remoteWriteURL, urlLabelValue string, fq *persistentqueue.FastQueue) *client {
authHeader := ""
if len(*basicAuthUsername) > 0 || len(*basicAuthPassword) > 0 {
// See https://en.wikipedia.org/wiki/Basic_access_authentication
token := *basicAuthUsername + ":" + *basicAuthPassword
token64 := base64.StdEncoding.EncodeToString([]byte(token))
authHeader = "Basic " + token64
}
if len(*bearerToken) > 0 {
if authHeader != "" {
logger.Panicf("FATAL: `-remoteWrite.bearerToken`=%q cannot be set when `-remoteWrite.basicAuth.*` flags are set", *bearerToken)
}
authHeader = "Bearer " + *bearerToken
}
readTimeout := *sendTimeout
if readTimeout <= 0 {
readTimeout = time.Minute
}
var u fasthttp.URI
u.Update(remoteWriteURL)
scheme := string(u.Scheme())
switch scheme {
case "http", "https":
default:
logger.Panicf("FATAL: unsupported scheme in -remoteWrite.url=%q: %q. It must be http or https", remoteWriteURL, scheme)
}
host := string(u.Host())
if len(host) == 0 {
logger.Panicf("FATAL: invalid -remoteWrite.url=%q: host cannot be empty. Make sure the url looks like `http://host:port/path`", remoteWriteURL)
}
requestURI := string(u.RequestURI())
isTLS := scheme == "https"
var tlsCfg *tls.Config
if isTLS {
var err error
tlsCfg, err = getTLSConfig()
if err != nil {
logger.Panicf("FATAL: cannot initialize TLS config: %s", err)
}
}
if !strings.Contains(host, ":") {
if isTLS {
host += ":443"
} else {
host += ":80"
}
}
maxConns := 2 * *queues
hc := &fasthttp.HostClient{
Addr: host,
Name: "vmagent",
Dial: statDial,
DialDualStack: netutil.TCP6Enabled(),
IsTLS: isTLS,
TLSConfig: tlsCfg,
MaxConns: maxConns,
MaxIdleConnDuration: 10 * readTimeout,
ReadTimeout: readTimeout,
WriteTimeout: 10 * time.Second,
MaxResponseBodySize: 1024 * 1024,
}
c := &client{
urlLabelValue: urlLabelValue,
remoteWriteURL: remoteWriteURL,
host: host,
requestURI: requestURI,
authHeader: authHeader,
fq: fq,
hc: hc,
stopCh: make(chan struct{}),
}
c.requestDuration = metrics.NewHistogram(fmt.Sprintf(`vmagent_remotewrite_duration_seconds{url=%q}`, c.urlLabelValue))
c.requestsOKCount = metrics.NewCounter(fmt.Sprintf(`vmagent_remotewrite_requests_total{url=%q, status_code="2XX"}`, c.urlLabelValue))
c.errorsCount = metrics.NewCounter(fmt.Sprintf(`vmagent_remotewrite_errors_total{url=%q}`, c.urlLabelValue))
c.retriesCount = metrics.NewCounter(fmt.Sprintf(`vmagent_remotewrite_retries_count_total{url=%q}`, c.urlLabelValue))
for i := 0; i < *queues; i++ {
c.wg.Add(1)
go func() {
defer c.wg.Done()
c.runWorker()
}()
}
logger.Infof("initialized client for -remoteWrite.url=%q", c.remoteWriteURL)
return c
}
func (c *client) MustStop() {
close(c.stopCh)
c.wg.Wait()
logger.Infof("stopped client for -remoteWrite.url=%q", c.remoteWriteURL)
}
func getTLSConfig() (*tls.Config, error) {
var tlsRootCA *x509.CertPool
var tlsCertificate *tls.Certificate
if *tlsCertFile != "" || *tlsKeyFile != "" {
cert, err := tls.LoadX509KeyPair(*tlsCertFile, *tlsKeyFile)
if err != nil {
return nil, fmt.Errorf("cannot load TLS certificate for -remoteWrite.tlsCertFile=%q and -remoteWrite.tlsKeyFile=%q: %s", *tlsCertFile, *tlsKeyFile, err)
}
tlsCertificate = &cert
}
if *tlsCAFile != "" {
data, err := ioutil.ReadFile(*tlsCAFile)
if err != nil {
return nil, fmt.Errorf("cannot read -remoteWrite.tlsCAFile=%q: %s", *tlsCAFile, err)
}
tlsRootCA = x509.NewCertPool()
if !tlsRootCA.AppendCertsFromPEM(data) {
return nil, fmt.Errorf("cannot parse data -remoteWrite.tlsCAFile=%q", *tlsCAFile)
}
}
tlsCfg := &tls.Config{
RootCAs: tlsRootCA,
ClientSessionCache: tls.NewLRUClientSessionCache(0),
}
if tlsCertificate != nil {
tlsCfg.Certificates = []tls.Certificate{*tlsCertificate}
}
tlsCfg.InsecureSkipVerify = *tlsInsecureSkipVerify
return tlsCfg, nil
}
func (c *client) runWorker() {
var ok bool
var block []byte
ch := make(chan struct{})
for {
block, ok = c.fq.MustReadBlock(block[:0])
if !ok {
return
}
go func() {
c.sendBlock(block)
ch <- struct{}{}
}()
select {
case <-ch:
// The block has been sent successfully
continue
case <-c.stopCh:
// c must be stopped. Wait for a while in the hope the block will be sent.
graceDuration := 5 * time.Second
select {
case <-ch:
// The block has been sent successfully.
case <-time.After(graceDuration):
logger.Errorf("couldn't sent block with size %d bytes to %q in %.3f seconds during shutdown; dropping it",
len(block), c.remoteWriteURL, graceDuration.Seconds())
}
return
}
}
}
func (c *client) sendBlock(block []byte) {
req := fasthttp.AcquireRequest()
req.SetRequestURI(c.requestURI)
req.SetHost(c.host)
req.Header.SetMethod("POST")
req.Header.Add("Content-Type", "application/x-protobuf")
req.Header.Add("Content-Encoding", "snappy")
if c.authHeader != "" {
req.Header.Set("Authorization", c.authHeader)
}
req.SetBody(block)
retryDuration := time.Second
resp := fasthttp.AcquireResponse()
again:
select {
case <-c.stopCh:
fasthttp.ReleaseRequest(req)
fasthttp.ReleaseResponse(resp)
return
default:
}
startTime := time.Now()
// There is no need in calling DoTimeout, since the timeout is set in c.hc.ReadTimeout.
err := c.hc.Do(req, resp)
c.requestDuration.UpdateDuration(startTime)
if err != nil {
c.errorsCount.Inc()
retryDuration *= 2
if retryDuration > time.Minute {
retryDuration = time.Minute
}
logger.Errorf("couldn't send a block with size %d bytes to %q: %s; re-sending the block in %.3f seconds",
len(block), c.remoteWriteURL, err, retryDuration.Seconds())
time.Sleep(retryDuration)
c.retriesCount.Inc()
goto again
}
statusCode := resp.StatusCode()
if statusCode/100 != 2 {
metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_requests_total{url=%q, status_code="%d"}`, c.urlLabelValue, statusCode)).Inc()
retryDuration *= 2
if retryDuration > time.Minute {
retryDuration = time.Minute
}
logger.Errorf("unexpected status code received after sending a block with size %d bytes to %q: %d; response body=%q; re-sending the block in %.3f seconds",
len(block), c.remoteWriteURL, statusCode, resp.Body(), retryDuration.Seconds())
time.Sleep(retryDuration)
c.retriesCount.Inc()
goto again
}
c.requestsOKCount.Inc()
// The block has been successfully sent to the remote storage.
fasthttp.ReleaseResponse(resp)
fasthttp.ReleaseRequest(req)
}

View File

@ -0,0 +1,191 @@
package remotewrite
import (
"flag"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/persistentqueue"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/metrics"
"github.com/golang/snappy"
)
var flushInterval = flag.Duration("remoteWrite.flushInterval", time.Second, "Interval for flushing the data to remote storage. "+
"Higher value reduces network bandwidth usage at the cost of delayed push of scraped data to remote storage")
// the maximum number of rows to send per each block.
const maxRowsPerBlock = 10000
type pendingSeries struct {
mu sync.Mutex
wr writeRequest
stopCh chan struct{}
periodicFlusherWG sync.WaitGroup
}
func newPendingSeries(pushBlock func(block []byte)) *pendingSeries {
var ps pendingSeries
ps.wr.pushBlock = pushBlock
ps.stopCh = make(chan struct{})
ps.periodicFlusherWG.Add(1)
go func() {
defer ps.periodicFlusherWG.Done()
ps.periodicFlusher()
}()
return &ps
}
func (ps *pendingSeries) MustStop() {
close(ps.stopCh)
ps.periodicFlusherWG.Wait()
}
func (ps *pendingSeries) Push(tss []prompbmarshal.TimeSeries) {
ps.mu.Lock()
ps.wr.push(tss)
ps.mu.Unlock()
}
func (ps *pendingSeries) periodicFlusher() {
ticker := time.NewTicker(*flushInterval)
defer ticker.Stop()
mustStop := false
for !mustStop {
select {
case <-ps.stopCh:
mustStop = true
case <-ticker.C:
if time.Since(ps.wr.lastFlushTime) < *flushInterval/2 {
continue
}
}
ps.mu.Lock()
ps.wr.flush()
ps.mu.Unlock()
}
}
type writeRequest struct {
wr prompbmarshal.WriteRequest
pushBlock func(block []byte)
lastFlushTime time.Time
tss []prompbmarshal.TimeSeries
labels []prompbmarshal.Label
samples []prompbmarshal.Sample
buf []byte
}
func (wr *writeRequest) reset() {
wr.wr.Timeseries = nil
for i := range wr.tss {
ts := &wr.tss[i]
ts.Labels = nil
ts.Samples = nil
}
wr.tss = wr.tss[:0]
for i := range wr.labels {
label := &wr.labels[i]
label.Name = ""
label.Value = ""
}
wr.labels = wr.labels[:0]
wr.samples = wr.samples[:0]
wr.buf = wr.buf[:0]
}
func (wr *writeRequest) flush() {
wr.wr.Timeseries = wr.tss
wr.lastFlushTime = time.Now()
pushWriteRequest(&wr.wr, wr.pushBlock)
wr.reset()
}
func (wr *writeRequest) push(src []prompbmarshal.TimeSeries) {
tssDst := wr.tss
for i := range src {
tssDst = append(tssDst, prompbmarshal.TimeSeries{})
dst := &tssDst[len(tssDst)-1]
wr.copyTimeSeries(dst, &src[i])
if len(wr.tss) >= maxRowsPerBlock {
wr.flush()
tssDst = wr.tss
}
}
wr.tss = tssDst
}
func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) {
labelsDst := wr.labels
labelsLen := len(wr.labels)
samplesDst := wr.samples
buf := wr.buf
for i := range src.Labels {
labelsDst = append(labelsDst, prompbmarshal.Label{})
dstLabel := &labelsDst[len(labelsDst)-1]
srcLabel := &src.Labels[i]
buf = append(buf, srcLabel.Name...)
dstLabel.Name = bytesutil.ToUnsafeString(buf[len(buf)-len(srcLabel.Name):])
buf = append(buf, srcLabel.Value...)
dstLabel.Value = bytesutil.ToUnsafeString(buf[len(buf)-len(srcLabel.Value):])
}
dst.Labels = labelsDst[labelsLen:]
samplesDst = append(samplesDst, prompbmarshal.Sample{})
dstSample := &samplesDst[len(samplesDst)-1]
if len(src.Samples) != 1 {
logger.Panicf("BUG: unexpected number of samples in time series; got %d; want 1", len(src.Samples))
}
*dstSample = src.Samples[0]
dst.Samples = samplesDst[len(samplesDst)-1:]
wr.samples = samplesDst
wr.labels = labelsDst
wr.buf = buf
}
func pushWriteRequest(wr *prompbmarshal.WriteRequest, pushBlock func(block []byte)) {
if len(wr.Timeseries) == 0 {
// Nothing to push
return
}
bb := writeRequestBufPool.Get()
bb.B = prompbmarshal.MarshalWriteRequest(bb.B[:0], wr)
zb := snappyBufPool.Get()
zb.B = snappy.Encode(zb.B[:cap(zb.B)], bb.B)
writeRequestBufPool.Put(bb)
if len(zb.B) <= persistentqueue.MaxBlockSize {
pushBlock(zb.B)
blockSizeRows.Update(float64(len(wr.Timeseries)))
blockSizeBytes.Update(float64(len(zb.B)))
snappyBufPool.Put(zb)
return
}
snappyBufPool.Put(zb)
// Too big block. Recursively split it into smaller parts.
timeseries := wr.Timeseries
n := len(timeseries) / 2
wr.Timeseries = timeseries[:n]
pushWriteRequest(wr, pushBlock)
wr.Timeseries = timeseries[n:]
pushWriteRequest(wr, pushBlock)
wr.Timeseries = timeseries
}
var (
blockSizeBytes = metrics.NewHistogram(`vmagent_remotewrite_block_size_bytes`)
blockSizeRows = metrics.NewHistogram(`vmagent_remotewrite_block_size_rows`)
)
var writeRequestBufPool bytesutil.ByteBufferPool
var snappyBufPool bytesutil.ByteBufferPool

View File

@ -0,0 +1,108 @@
package remotewrite
import (
"flag"
"strings"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
)
var (
extraLabelsUnparsed = flagutil.NewArray("remoteWrite.label", "Optional label in the form 'name=value' to add to all the metrics before sending them to -remoteWrite.url. "+
"Pass multiple -remoteWrite.label flags in order to add multiple flags to metrics before sending them to remote storage")
relabelConfigPath = flag.String("remoteWrite.relabelConfig", "", "Optional path to file with relabel_config entries. These entries are applied to all the metrics "+
"before sending them to -remoteWrite.url. See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config for details")
)
var extraLabels []prompbmarshal.Label
var prcs []promrelabel.ParsedRelabelConfig
// initRelabel must be called after parsing command-line flags.
func initRelabel() {
// Init extraLabels
for _, s := range *extraLabelsUnparsed {
n := strings.IndexByte(s, '=')
if n < 0 {
logger.Panicf("FATAL: missing '=' in `-remoteWrite.label`. It must contain label in the form `name=value`; got %q", s)
}
extraLabels = append(extraLabels, prompbmarshal.Label{
Name: s[:n],
Value: s[n+1:],
})
}
// Init prcs
if len(*relabelConfigPath) > 0 {
var err error
prcs, err = promrelabel.LoadRelabelConfigs(*relabelConfigPath)
if err != nil {
logger.Panicf("FATAL: cannot load relabel configs from -remoteWrite.relabelConfig=%q: %s", *relabelConfigPath, err)
}
}
}
func resetRelabel() {
extraLabels = nil
prcs = nil
}
func (rctx *relabelCtx) applyRelabeling(wr *prompbmarshal.WriteRequest) {
if len(extraLabels) == 0 && len(prcs) == 0 {
// Nothing to change.
return
}
tss := wr.Timeseries
tssDst := tss[:0]
labels := rctx.labels[:0]
for i := range tss {
ts := &tss[i]
labelsLen := len(labels)
labels = append(labels, ts.Labels...)
// extraLabels must be added before applying relabeling according to https://prometheus.io/docs/prometheus/latest/configuration/configuration/#remote_write
for j := range extraLabels {
extraLabel := &extraLabels[j]
tmp := promrelabel.GetLabelByName(labels[labelsLen:], extraLabel.Name)
if tmp != nil {
tmp.Value = extraLabel.Value
} else {
labels = append(labels, *extraLabel)
}
}
labels = promrelabel.ApplyRelabelConfigs(labels, labelsLen, prcs, true)
if len(labels) == labelsLen {
// Drop the current time series, since relabeling removed all the labels.
continue
}
tssDst = append(tssDst, prompbmarshal.TimeSeries{
Labels: labels[labelsLen:],
Samples: ts.Samples,
})
}
rctx.labels = labels
wr.Timeseries = tssDst
}
type relabelCtx struct {
// pool for labels, which are used during the relabeling.
labels []prompbmarshal.Label
}
func (rctx *relabelCtx) reset() {
labels := rctx.labels
for i := range labels {
label := &labels[i]
label.Name = ""
label.Value = ""
}
rctx.labels = rctx.labels[:0]
}
var relabelCtxPool = &sync.Pool{
New: func() interface{} {
return &relabelCtx{}
},
}

View File

@ -0,0 +1,127 @@
package remotewrite
import (
"flag"
"fmt"
"sync/atomic"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/httpserver"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/persistentqueue"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/metrics"
xxhash "github.com/cespare/xxhash/v2"
)
var (
remoteWriteURLs = flagutil.NewArray("remoteWrite.url", "Remote storage URL to write data to. It must support Prometheus remote_write API. "+
"It is recommended using VictoriaMetrics as remote storage. Example url: http://<victoriametrics-host>:8428/api/v1/write . "+
"Pass multiple -remoteWrite.url flags in order to write data concurrently to multiple remote storage systems")
tmpDataPath = flag.String("remoteWrite.tmpDataPath", "vmagent-remotewrite-data", "Path to directory where temporary data for remote write component is stored")
queues = flag.Int("remoteWrite.queues", 1, "The number of concurrent queues to each -remoteWrite.url. Set more queues if a single queue "+
"isn't enough for sending high volume of collected data to remote storage")
showRemoteWriteURL = flag.Bool("remoteWrite.showURL", false, "Whether to show -remoteWrite.url in the exported metrics. "+
"It is hidden by default, since it can contain sensistive auth info")
)
// Init initializes remotewrite.
//
// It must be called after flag.Parse().
//
// Stop must be called for graceful shutdown.
func Init() {
if len(*remoteWriteURLs) == 0 {
logger.Panicf("FATAL: at least one `-remoteWrite.url` must be set")
}
if !*showRemoteWriteURL {
// remoteWrite.url can contain authentication codes, so hide it at `/metrics` output.
httpserver.RegisterSecretFlag("remoteWrite.url")
}
initRelabel()
maxInmemoryBlocks := memory.Allowed() / len(*remoteWriteURLs) / maxRowsPerBlock / 100
if maxInmemoryBlocks > 200 {
// There is no much sense in keeping higher number of blocks in memory,
// since this means that the producer outperforms consumer and the queue
// will continue growing. It is better storing the queue to file.
maxInmemoryBlocks = 200
}
if maxInmemoryBlocks < 2 {
maxInmemoryBlocks = 2
}
for i, remoteWriteURL := range *remoteWriteURLs {
h := xxhash.Sum64([]byte(remoteWriteURL))
path := fmt.Sprintf("%s/persistent-queue/%016X", *tmpDataPath, h)
fq := persistentqueue.MustOpenFastQueue(path, remoteWriteURL, maxInmemoryBlocks)
urlLabelValue := fmt.Sprintf("secret-url-%d", i+1)
if *showRemoteWriteURL {
urlLabelValue = remoteWriteURL
}
_ = metrics.NewGauge(fmt.Sprintf(`vmagent_remotewrite_pending_data_bytes{url=%q, hash="%016X"}`, urlLabelValue, h), func() float64 {
return float64(fq.GetPendingBytes())
})
_ = metrics.NewGauge(fmt.Sprintf(`vmagent_remotewrite_pending_inmemory_blocks{url=%q}`, urlLabelValue), func() float64 {
return float64(fq.GetInmemoryQueueLen())
})
c := newClient(remoteWriteURL, urlLabelValue, fq)
fqs = append(fqs, fq)
cs = append(cs, c)
}
pss = make([]*pendingSeries, *queues)
for i := range pss {
pss[i] = newPendingSeries(pushBlockToPersistentQueues)
}
}
// Stop stops remotewrite.
//
// It is expected that nobody calls Push during and after the call to this func.
func Stop() {
for _, ps := range pss {
ps.MustStop()
}
// Close all the persistent queues. This should unblock clients waiting in MustReadBlock.
for _, fq := range fqs {
fq.MustClose()
}
fqs = nil
// Stop all the clients
for _, c := range cs {
c.MustStop()
}
cs = nil
resetRelabel()
}
// Push sends wr to remote storage systems set via `-remoteWrite.url`.
//
// Each timeseries in wr.Timeseries must contain one sample.
func Push(wr *prompbmarshal.WriteRequest) {
rctx := relabelCtxPool.Get().(*relabelCtx)
rctx.applyRelabeling(wr)
idx := atomic.AddUint64(&pssNextIdx, 1) % uint64(len(pss))
pss[idx].Push(wr.Timeseries)
rctx.reset()
relabelCtxPool.Put(rctx)
}
func pushBlockToPersistentQueues(block []byte) {
for _, fq := range fqs {
fq.MustWriteBlock(block)
}
}
var fqs []*persistentqueue.FastQueue
var cs []*client
var pssNextIdx uint64
var pss []*pendingSeries

View File

@ -0,0 +1,71 @@
package remotewrite
import (
"net"
"sync/atomic"
"github.com/VictoriaMetrics/metrics"
"github.com/valyala/fasthttp"
)
func statDial(addr string) (net.Conn, error) {
conn, err := fasthttp.Dial(addr)
dialsTotal.Inc()
if err != nil {
dialErrors.Inc()
return nil, err
}
conns.Inc()
sc := &statConn{
Conn: conn,
}
return sc, nil
}
var (
dialsTotal = metrics.NewCounter(`vmagent_remotewrite_dials_total`)
dialErrors = metrics.NewCounter(`vmagent_remotewrite_dial_errors_total`)
conns = metrics.NewCounter(`vmagent_remotewrite_conns`)
)
type statConn struct {
closed uint64
net.Conn
}
func (sc *statConn) Read(p []byte) (int, error) {
n, err := sc.Conn.Read(p)
connReadsTotal.Inc()
if err != nil {
connReadErrors.Inc()
}
connBytesRead.Add(n)
return n, err
}
func (sc *statConn) Write(p []byte) (int, error) {
n, err := sc.Conn.Write(p)
connWritesTotal.Inc()
if err != nil {
connWriteErrors.Inc()
}
connBytesWritten.Add(n)
return n, err
}
func (sc *statConn) Close() error {
err := sc.Conn.Close()
if atomic.AddUint64(&sc.closed, 1) == 1 {
conns.Dec()
}
return err
}
var (
connReadsTotal = metrics.NewCounter(`vmagent_remotewrite_conn_reads_total`)
connWritesTotal = metrics.NewCounter(`vmagent_remotewrite_conn_writes_total`)
connReadErrors = metrics.NewCounter(`vmagent_remotewrite_conn_read_errors_total`)
connWriteErrors = metrics.NewCounter(`vmagent_remotewrite_conn_write_errors_total`)
connBytesRead = metrics.NewCounter(`vmagent_remotewrite_conn_bytes_read_total`)
connBytesWritten = metrics.NewCounter(`vmagent_remotewrite_conn_bytes_written_total`)
)

View File

@ -0,0 +1,70 @@
package vmimport
import (
"net/http"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="vmimport"}`)
rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="vmimport"}`)
)
// InsertHandler processes `/api/v1/import` request.
//
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6
func InsertHandler(req *http.Request) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertRows(rows []parser.Row) error {
ctx := common.GetPushCtx()
defer common.PutPushCtx(ctx)
rowsTotal := 0
tssDst := ctx.WriteRequest.Timeseries[:0]
labels := ctx.Labels[:0]
samples := ctx.Samples[:0]
for i := range rows {
r := &rows[i]
labelsLen := len(labels)
for j := range r.Tags {
tag := &r.Tags[j]
labels = append(labels, prompbmarshal.Label{
Name: bytesutil.ToUnsafeString(tag.Key),
Value: bytesutil.ToUnsafeString(tag.Value),
})
}
values := r.Values
timestamps := r.Timestamps
_ = timestamps[len(values)-1]
samplesLen := len(samples)
for j, value := range values {
samples = append(samples, prompbmarshal.Sample{
Value: value,
Timestamp: timestamps[j],
})
}
tssDst = append(tssDst, prompbmarshal.TimeSeries{
Labels: labels[labelsLen:],
Samples: samples[samplesLen:],
})
rowsTotal += len(values)
}
ctx.WriteRequest.Timeseries = tssDst
ctx.Labels = labels
ctx.Samples = samples
remotewrite.Push(&ctx.WriteRequest)
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
return nil
}

View File

@ -0,0 +1,36 @@
package common
import (
"runtime"
"sync"
)
// GetInsertCtx returns InsertCtx from the pool.
//
// Call PutInsertCtx for returning it to the pool.
func GetInsertCtx() *InsertCtx {
select {
case ctx := <-insertCtxPoolCh:
return ctx
default:
if v := insertCtxPool.Get(); v != nil {
return v.(*InsertCtx)
}
return &InsertCtx{}
}
}
// PutInsertCtx returns ctx to the pool.
//
// ctx cannot be used after the call.
func PutInsertCtx(ctx *InsertCtx) {
ctx.Reset(0)
select {
case insertCtxPoolCh <- ctx:
default:
insertCtxPool.Put(ctx)
}
}
var insertCtxPool sync.Pool
var insertCtxPoolCh = make(chan *InsertCtx, runtime.GOMAXPROCS(-1))

View File

@ -1,162 +1,44 @@
package graphite
import (
"fmt"
"io"
"net"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/graphite"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/graphite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="graphite"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="graphite"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="graphite"}`)
)
// insertHandler processes remote write for graphite plaintext protocol.
// InsertHandler processes remote write for graphite plaintext protocol.
//
// See https://graphite.readthedocs.io/en/latest/feeding-carbon.html#the-plaintext-protocol
func insertHandler(r io.Reader) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(r)
func InsertHandler(r io.Reader) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(r, insertRows)
})
}
func insertHandlerInternal(r io.Reader) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
for ctx.Read(r) {
if err := ctx.InsertRows(); err != nil {
return err
}
}
return ctx.Error()
}
func insertRows(rows []parser.Row) error {
ctx := common.GetInsertCtx()
defer common.PutInsertCtx(ctx)
func (ctx *pushCtx) InsertRows() error {
rows := ctx.Rows.Rows
ic := &ctx.Common
ic.Reset(len(rows))
ctx.Reset(len(rows))
for i := range rows {
r := &rows[i]
ic.Labels = ic.Labels[:0]
ic.AddLabel("", r.Metric)
ctx.Labels = ctx.Labels[:0]
ctx.AddLabel("", r.Metric)
for j := range r.Tags {
tag := &r.Tags[j]
ic.AddLabel(tag.Key, tag.Value)
ctx.AddLabel(tag.Key, tag.Value)
}
ic.WriteDataPoint(nil, ic.Labels, r.Timestamp, r.Value)
ctx.WriteDataPoint(nil, ctx.Labels, r.Timestamp, r.Value)
}
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return ic.FlushBufs()
return ctx.FlushBufs()
}
const flushTimeout = 3 * time.Second
func (ctx *pushCtx) Read(r io.Reader) bool {
readCalls.Inc()
if ctx.err != nil {
return false
}
if c, ok := r.(net.Conn); ok {
if err := c.SetReadDeadline(time.Now().Add(flushTimeout)); err != nil {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot set read deadline: %s", err)
return false
}
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ne, ok := ctx.err.(net.Error); ok && ne.Timeout() {
// Flush the read data on timeout and try reading again.
ctx.err = nil
} else {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read graphite plaintext protocol data: %s", ctx.err)
}
return false
}
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
// Fill missing timestamps with the current timestamp rounded to seconds.
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps from seconds to milliseconds.
for i := range rows {
rows[i].Timestamp *= 1e3
}
return true
}
type pushCtx struct {
Rows graphite.Rows
Common common.InsertCtx
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *pushCtx) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *pushCtx) reset() {
ctx.Rows.Reset()
ctx.Common.Reset(0)
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
var (
readCalls = metrics.NewCounter(`vm_read_calls_total{name="graphite"}`)
readErrors = metrics.NewCounter(`vm_read_errors_total{name="graphite"}`)
)
func getPushCtx() *pushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*pushCtx)
}
return &pushCtx{}
}
}
func putPushCtx(ctx *pushCtx) {
ctx.reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *pushCtx, runtime.GOMAXPROCS(-1))

View File

@ -2,18 +2,15 @@ package influx
import (
"flag"
"fmt"
"io"
"net/http"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/influx"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/influx"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
@ -24,63 +21,22 @@ var (
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="influx"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="influx"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="influx"}`)
)
// InsertHandler processes remote write for influx line protocol.
//
// See https://github.com/influxdata/influxdb/blob/4cbdc197b8117fee648d62e2e5be75c6575352f0/tsdb/README.md
func InsertHandler(req *http.Request) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(req)
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertHandlerInternal(req *http.Request) error {
readCalls.Inc()
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
return fmt.Errorf("cannot read gzipped influx line protocol data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
q := req.URL.Query()
tsMultiplier := int64(1e6)
switch q.Get("precision") {
case "ns":
tsMultiplier = 1e6
case "u":
tsMultiplier = 1e3
case "ms":
tsMultiplier = 1
case "s":
tsMultiplier = -1e3
case "m":
tsMultiplier = -1e3 * 60
case "h":
tsMultiplier = -1e3 * 3600
}
// Read db tag from https://docs.influxdata.com/influxdb/v1.7/tools/api/#write-http-endpoint
db := q.Get("db")
func insertRows(db string, rows []parser.Row) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
for ctx.Read(r, tsMultiplier) {
if err := ctx.InsertRows(db); err != nil {
return err
}
}
return ctx.Error()
}
func (ctx *pushCtx) InsertRows(db string) error {
rows := ctx.Rows.Rows
rowsLen := 0
for i := range rows {
rowsLen += len(rows[i].Fields)
@ -126,80 +82,16 @@ func (ctx *pushCtx) InsertRows(db string) error {
return ic.FlushBufs()
}
func (ctx *pushCtx) Read(r io.Reader, tsMultiplier int64) bool {
if ctx.err != nil {
return false
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read influx line protocol data: %s", ctx.err)
}
return false
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
// Adjust timestamps according to tsMultiplier
currentTs := time.Now().UnixNano() / 1e6
if tsMultiplier >= 1 {
for i := range ctx.Rows.Rows {
row := &ctx.Rows.Rows[i]
if row.Timestamp == 0 {
row.Timestamp = currentTs
} else {
row.Timestamp /= tsMultiplier
}
}
} else if tsMultiplier < 0 {
tsMultiplier = -tsMultiplier
currentTs -= currentTs % tsMultiplier
for i := range ctx.Rows.Rows {
row := &ctx.Rows.Rows[i]
if row.Timestamp == 0 {
row.Timestamp = currentTs
} else {
row.Timestamp *= tsMultiplier
}
}
}
return true
}
var (
readCalls = metrics.NewCounter(`vm_read_calls_total{name="influx"}`)
readErrors = metrics.NewCounter(`vm_read_errors_total{name="influx"}`)
)
type pushCtx struct {
Rows influx.Rows
Common common.InsertCtx
reqBuf []byte
tailBuf []byte
Common common.InsertCtx
metricNameBuf []byte
metricGroupBuf []byte
err error
}
func (ctx *pushCtx) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *pushCtx) reset() {
ctx.Rows.Reset()
ctx.Common.Reset(0)
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.metricNameBuf = ctx.metricNameBuf[:0]
ctx.metricGroupBuf = ctx.metricGroupBuf[:0]
ctx.err = nil
}
func getPushCtx() *pushCtx {

View File

@ -6,15 +6,20 @@ import (
"net/http"
"strings"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/graphite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/influx"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/opentsdb"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/prometheus"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/prompush"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/httpserver"
graphiteserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/graphite"
opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb"
opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
@ -28,29 +33,31 @@ var (
)
var (
graphiteServer *graphite.Server
opentsdbServer *opentsdb.Server
opentsdbhttpServer *opentsdbhttp.Server
graphiteServer *graphiteserver.Server
opentsdbServer *opentsdbserver.Server
opentsdbhttpServer *opentsdbhttpserver.Server
)
// Init initializes vminsert.
func Init() {
storage.SetMaxLabelsPerTimeseries(*maxLabelsPerTimeseries)
concurrencylimiter.Init()
writeconcurrencylimiter.Init()
if len(*graphiteListenAddr) > 0 {
graphiteServer = graphite.MustStart(*graphiteListenAddr)
graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, graphite.InsertHandler)
}
if len(*opentsdbListenAddr) > 0 {
opentsdbServer = opentsdb.MustStart(*opentsdbListenAddr)
opentsdbServer = opentsdbserver.MustStart(*opentsdbListenAddr, opentsdb.InsertHandler, opentsdbhttp.InsertHandler)
}
if len(*opentsdbHTTPListenAddr) > 0 {
opentsdbhttpServer = opentsdbhttp.MustStart(*opentsdbHTTPListenAddr)
opentsdbhttpServer = opentsdbhttpserver.MustStart(*opentsdbHTTPListenAddr, opentsdbhttp.InsertHandler)
}
promscrape.Init(prompush.Push)
}
// Stop stops vminsert.
func Stop() {
promscrape.Stop()
if len(*graphiteListenAddr) > 0 {
graphiteServer.MustStop()
}
@ -68,7 +75,7 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool {
switch path {
case "/api/v1/write":
prometheusWriteRequests.Inc()
if err := prometheus.InsertHandler(r); err != nil {
if err := promremotewrite.InsertHandler(r); err != nil {
prometheusWriteErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", r.URL.Path, err)
return true

View File

@ -1,161 +1,44 @@
package opentsdb
import (
"fmt"
"io"
"net"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdb"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdb"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="opentsdb"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="opentsdb"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="opentsdb"}`)
)
// insertHandler processes remote write for OpenTSDB put protocol.
// InsertHandler processes remote write for OpenTSDB put protocol.
//
// See http://opentsdb.net/docs/build/html/api_telnet/put.html
func insertHandler(r io.Reader) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(r)
func InsertHandler(r io.Reader) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(r, insertRows)
})
}
func insertHandlerInternal(r io.Reader) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
for ctx.Read(r) {
if err := ctx.InsertRows(); err != nil {
return err
}
}
return ctx.Error()
}
func insertRows(rows []parser.Row) error {
ctx := common.GetInsertCtx()
defer common.PutInsertCtx(ctx)
func (ctx *pushCtx) InsertRows() error {
rows := ctx.Rows.Rows
ic := &ctx.Common
ic.Reset(len(rows))
ctx.Reset(len(rows))
for i := range rows {
r := &rows[i]
ic.Labels = ic.Labels[:0]
ic.AddLabel("", r.Metric)
ctx.Labels = ctx.Labels[:0]
ctx.AddLabel("", r.Metric)
for j := range r.Tags {
tag := &r.Tags[j]
ic.AddLabel(tag.Key, tag.Value)
ctx.AddLabel(tag.Key, tag.Value)
}
ic.WriteDataPoint(nil, ic.Labels, r.Timestamp, r.Value)
ctx.WriteDataPoint(nil, ctx.Labels, r.Timestamp, r.Value)
}
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return ic.FlushBufs()
return ctx.FlushBufs()
}
const flushTimeout = 3 * time.Second
func (ctx *pushCtx) Read(r io.Reader) bool {
readCalls.Inc()
if ctx.err != nil {
return false
}
if c, ok := r.(net.Conn); ok {
if err := c.SetReadDeadline(time.Now().Add(flushTimeout)); err != nil {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot set read deadline: %s", err)
return false
}
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ne, ok := ctx.err.(net.Error); ok && ne.Timeout() {
// Flush the read data on timeout and try reading again.
ctx.err = nil
} else {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read OpenTSDB put protocol data: %s", ctx.err)
}
return false
}
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
// Fill in missing timestamps
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps from seconds to milliseconds
for i := range rows {
rows[i].Timestamp *= 1e3
}
return true
}
type pushCtx struct {
Rows opentsdb.Rows
Common common.InsertCtx
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *pushCtx) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *pushCtx) reset() {
ctx.Rows.Reset()
ctx.Common.Reset(0)
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
var (
readCalls = metrics.NewCounter(`vm_read_calls_total{name="opentsdb"}`)
readErrors = metrics.NewCounter(`vm_read_errors_total{name="opentsdb"}`)
)
func getPushCtx() *pushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*pushCtx)
}
return &pushCtx{}
}
}
func putPushCtx(ctx *pushCtx) {
ctx.reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *pushCtx, runtime.GOMAXPROCS(-1))

View File

@ -1,151 +1,50 @@
package opentsdbhttp
import (
"flag"
"fmt"
"io"
"net/http"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdbhttp"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var maxInsertRequestSize = flag.Int("opentsdbhttp.maxInsertRequestSize", 32*1024*1024, "The maximum size of OpenTSDB HTTP put request")
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="opentsdb-http"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="opentsdb-http"}`)
readCalls = metrics.NewCounter(`vm_read_calls_total{name="opentsdb-http"}`)
readErrors = metrics.NewCounter(`vm_read_errors_total{name="opentsdb-http"}`)
unmarshalErrors = metrics.NewCounter(`vm_unmarshal_errors_total{name="opentsdb-http"}`)
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="opentsdbhttp"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="opentsdbhttp"}`)
)
// insertHandler processes HTTP OpenTSDB put requests.
// InsertHandler processes HTTP OpenTSDB put requests.
// See http://opentsdb.net/docs/build/html/api_http/put.html
func insertHandler(req *http.Request) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(req)
})
func InsertHandler(req *http.Request) error {
path := req.URL.Path
switch path {
case "/api/put":
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
default:
return fmt.Errorf("unexpected path requested on HTTP OpenTSDB server: %q", path)
}
}
func insertHandlerInternal(req *http.Request) error {
readCalls.Inc()
func insertRows(rows []parser.Row) error {
ctx := common.GetInsertCtx()
defer common.PutInsertCtx(ctx)
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
readErrors.Inc()
return fmt.Errorf("cannot read gzipped http protocol data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
ctx := getPushCtx()
defer putPushCtx(ctx)
// Read the request in ctx.reqBuf
lr := io.LimitReader(r, int64(*maxInsertRequestSize)+1)
reqLen, err := ctx.reqBuf.ReadFrom(lr)
if err != nil {
readErrors.Inc()
return fmt.Errorf("cannot read HTTP OpenTSDB request: %s", err)
}
if reqLen > int64(*maxInsertRequestSize) {
readErrors.Inc()
return fmt.Errorf("too big HTTP OpenTSDB request; mustn't exceed `-opentsdbhttp.maxInsertRequestSize=%d` bytes", *maxInsertRequestSize)
}
// Unmarshal the request to ctx.Rows
p := opentsdbhttp.GetParser()
defer opentsdbhttp.PutParser(p)
v, err := p.ParseBytes(ctx.reqBuf.B)
if err != nil {
unmarshalErrors.Inc()
return fmt.Errorf("cannot parse HTTP OpenTSDB json: %s", err)
}
ctx.Rows.Unmarshal(v)
// Fill in missing timestamps
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
ctx.Reset(len(rows))
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps in seconds to milliseconds if needed.
// See http://opentsdb.net/docs/javadoc/net/opentsdb/core/Const.html#SECOND_MASK
for i := range rows {
r := &rows[i]
if r.Timestamp&secondMask == 0 {
r.Timestamp *= 1e3
}
}
// Insert ctx.Rows to db.
ic := &ctx.Common
ic.Reset(len(rows))
for i := range rows {
r := &rows[i]
ic.Labels = ic.Labels[:0]
ic.AddLabel("", r.Metric)
ctx.Labels = ctx.Labels[:0]
ctx.AddLabel("", r.Metric)
for j := range r.Tags {
tag := &r.Tags[j]
ic.AddLabel(tag.Key, tag.Value)
ctx.AddLabel(tag.Key, tag.Value)
}
ic.WriteDataPoint(nil, ic.Labels, r.Timestamp, r.Value)
ctx.WriteDataPoint(nil, ctx.Labels, r.Timestamp, r.Value)
}
rowsInserted.Add(len(rows))
rowsPerInsert.Update(float64(len(rows)))
return ic.FlushBufs()
return ctx.FlushBufs()
}
const secondMask int64 = 0x7FFFFFFF00000000
type pushCtx struct {
Rows opentsdbhttp.Rows
Common common.InsertCtx
reqBuf bytesutil.ByteBuffer
}
func (ctx *pushCtx) reset() {
ctx.Rows.Reset()
ctx.Common.Reset(0)
ctx.reqBuf.Reset()
}
func getPushCtx() *pushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*pushCtx)
}
return &pushCtx{}
}
}
func putPushCtx(ctx *pushCtx) {
ctx.reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *pushCtx, runtime.GOMAXPROCS(-1))

View File

@ -0,0 +1,97 @@
package prompush
import (
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="promscrape"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="promscrape"}`)
)
// Push pushes wr to to storage.
func Push(wr *prompbmarshal.WriteRequest) {
ctx := getPushCtx()
defer putPushCtx(ctx)
timeseries := wr.Timeseries
rowsLen := 0
for i := range timeseries {
rowsLen += len(timeseries[i].Samples)
}
ic := &ctx.Common
ic.Reset(rowsLen)
rowsTotal := 0
labels := ctx.labels[:0]
for i := range timeseries {
ts := &timeseries[i]
labels = labels[:0]
for j := range ts.Labels {
label := &ts.Labels[j]
labels = append(labels, prompb.Label{
Name: bytesutil.ToUnsafeBytes(label.Name),
Value: bytesutil.ToUnsafeBytes(label.Value),
})
}
var metricNameRaw []byte
for i := range ts.Samples {
r := &ts.Samples[i]
metricNameRaw = ic.WriteDataPointExt(metricNameRaw, labels, r.Timestamp, r.Value)
}
rowsTotal += len(ts.Samples)
}
ctx.labels = labels
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
if err := ic.FlushBufs(); err != nil {
logger.Errorf("cannot flush promscrape data to storage: %s", err)
}
}
type pushCtx struct {
Common common.InsertCtx
labels []prompb.Label
}
func (ctx *pushCtx) reset() {
ctx.Common.Reset(0)
for i := range ctx.labels {
label := &ctx.labels[i]
label.Name = nil
label.Value = nil
}
ctx.labels = ctx.labels[:0]
}
func getPushCtx() *pushCtx {
select {
case ctx := <-pushCtxPoolCh:
return ctx
default:
if v := pushCtxPool.Get(); v != nil {
return v.(*pushCtx)
}
return &pushCtx{}
}
}
func putPushCtx(ctx *pushCtx) {
ctx.reset()
select {
case pushCtxPoolCh <- ctx:
default:
pushCtxPool.Put(ctx)
}
}
var pushCtxPool sync.Pool
var pushCtxPoolCh = make(chan *pushCtx, runtime.GOMAXPROCS(-1))

View File

@ -0,0 +1,47 @@
package promremotewrite
import (
"net/http"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/promremotewrite"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="promremotewrite"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="promremotewrite"}`)
)
// InsertHandler processes remote write for prometheus.
func InsertHandler(req *http.Request) error {
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertRows(timeseries []prompb.TimeSeries) error {
ctx := common.GetInsertCtx()
defer common.PutInsertCtx(ctx)
rowsLen := 0
for i := range timeseries {
rowsLen += len(timeseries[i].Samples)
}
ctx.Reset(rowsLen)
rowsTotal := 0
for i := range timeseries {
ts := &timeseries[i]
var metricNameRaw []byte
for i := range ts.Samples {
r := &ts.Samples[i]
metricNameRaw = ctx.WriteDataPointExt(metricNameRaw, ts.Labels, r.Timestamp, r.Value)
}
rowsTotal += len(ts.Samples)
}
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
return ctx.FlushBufs()
}

View File

@ -1,61 +1,35 @@
package vmimport
import (
"flag"
"fmt"
"io"
"net/http"
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/vmimport"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter"
"github.com/VictoriaMetrics/metrics"
)
var maxLineLen = flag.Int("import.maxLineLen", 100*1024*1024, "The maximum length in bytes of a single line accepted by /api/v1/import")
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="vmimport"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="vmimport"}`)
rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="vmimport"}`)
)
// InsertHandler processes `/api/v1/import` request.
//
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6
func InsertHandler(req *http.Request) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(req)
return writeconcurrencylimiter.Do(func() error {
return parser.ParseStream(req, insertRows)
})
}
func insertHandlerInternal(req *http.Request) error {
readCalls.Inc()
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
return fmt.Errorf("cannot read gzipped vmimport data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
func insertRows(rows []parser.Row) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
for ctx.Read(r) {
if err := ctx.InsertRows(); err != nil {
return err
}
}
return ctx.Error()
}
func (ctx *pushCtx) InsertRows() error {
rows := ctx.Rows.Rows
rowsLen := 0
for i := range rows {
rowsLen += len(rows[i].Values)
@ -85,54 +59,14 @@ func (ctx *pushCtx) InsertRows() error {
return ic.FlushBufs()
}
func (ctx *pushCtx) Read(r io.Reader) bool {
if ctx.err != nil {
return false
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlockExt(r, ctx.reqBuf, ctx.tailBuf, *maxLineLen)
if ctx.err != nil {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read vmimport data: %s", ctx.err)
}
return false
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
return true
}
var (
readCalls = metrics.NewCounter(`vm_read_calls_total{name="vmimport"}`)
readErrors = metrics.NewCounter(`vm_read_errors_total{name="vmimport"}`)
)
type pushCtx struct {
Rows Rows
Common common.InsertCtx
reqBuf []byte
tailBuf []byte
Common common.InsertCtx
metricNameBuf []byte
err error
}
func (ctx *pushCtx) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *pushCtx) reset() {
ctx.Rows.Reset()
ctx.Common.Reset(0)
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.metricNameBuf = ctx.metricNameBuf[:0]
ctx.err = nil
}
func getPushCtx() *pushCtx {

2
go.mod
View File

@ -9,6 +9,7 @@ require (
github.com/cespare/xxhash/v2 v2.1.1
github.com/golang/snappy v0.0.1
github.com/klauspost/compress v1.10.0
github.com/valyala/fasthttp v1.9.0
github.com/valyala/fastjson v1.5.0
github.com/valyala/fastrand v1.0.0
github.com/valyala/gozstd v1.6.4
@ -18,6 +19,7 @@ require (
golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4
golang.org/x/tools v0.0.0-20200214225126-5916a50871fb // indirect
google.golang.org/api v0.17.0
gopkg.in/yaml.v2 v2.2.8
)
go 1.12

8
go.sum
View File

@ -93,10 +93,12 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.4.1/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.10.0 h1:92XGj1AcYzA6UrVdd4qIIBrT8OroryvRvdmg/IfmC7Y=
github.com/klauspost/compress v1.10.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs=
github.com/klauspost/cpuid v0.0.0-20180405133222-e7e905edc00e/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
@ -114,6 +116,8 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P
github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw=
github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
github.com/valyala/fasthttp v1.2.0/go.mod h1:4vX61m6KN+xDduDNwXrhIAVZaZaZiQ1luJk8LWSxF3s=
github.com/valyala/fasthttp v1.9.0 h1:hNpmUdy/+ZXYpGy0OBfm7K0UQTzb73W0T0U4iJIVrMw=
github.com/valyala/fasthttp v1.9.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w=
github.com/valyala/fastjson v1.5.0 h1:DGrb4wEYso2HdGLyLmNoyNCQnCWfjd8yhghPv5/5YQg=
github.com/valyala/fastjson v1.5.0/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY=
github.com/valyala/fastrand v1.0.0 h1:LUKT9aKer2dVQNUi3waewTbKV+7H17kvWFNKs2ObdkI=
@ -141,7 +145,6 @@ golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxT
golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek=
golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY=
golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
golang.org/x/exp v0.0.0-20191227195350-da58074b4299 h1:zQpM52jfKHG6II1ISZY1ZcpygvuSFZpLwfluuF89XOg=
golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
golang.org/x/exp v0.0.0-20200207192155-f17229e696bd h1:zkO/Lhoka23X63N9OSzpSeROEUQ5ODw47tM3YWjygbs=
golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM=
@ -181,6 +184,7 @@ golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn
golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200202094626-16171245cfb2 h1:CCH4IOTTfewWjGOlSp+zGcjutRKlBEZQ6wTn8ozI/nI=
golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
@ -293,6 +297,8 @@ gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8
gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10=
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=

View File

@ -212,7 +212,7 @@ func newWriter(f *os.File, nocache bool) *Writer {
// MustClose syncs the underlying file to storage and then closes it.
func (w *Writer) MustClose() {
if err := w.bw.Flush(); err != nil {
logger.Panicf("FATAL: cannot flush data to file %q: %s", w.f.Name(), err)
logger.Panicf("FATAL: cannot flush buffered data to file %q: %s", w.f.Name(), err)
}
putBufioWriter(w.bw)
w.bw = nil
@ -253,6 +253,14 @@ func (w *Writer) Write(p []byte) (int, error) {
return n, nil
}
// MustFlush flushes all the buffered data to file.
func (w *Writer) MustFlush() {
if err := w.bw.Flush(); err != nil {
logger.Panicf("FATAL: cannot flush buffered data to file %q: %s", w.f.Name(), err)
}
// Do not call w.f.Sync() for performance reasons.
}
type statWriter struct {
*os.File
}

28
lib/flagutil/array.go Normal file
View File

@ -0,0 +1,28 @@
package flagutil
import (
"flag"
"strings"
)
// NewArray returns new Array with the given name and descprition.
func NewArray(name, description string) *Array {
var a Array
flag.Var(&a, name, description)
return &a
}
// Array holds an array of flag values
type Array []string
// String implements flag.Value interface
func (a *Array) String() string {
return strings.Join(*a, ",")
}
// Set implements flag.Value interface
func (a *Array) Set(value string) error {
values := strings.Split(value, ",")
*a = append(*a, values...)
return nil
}

View File

@ -0,0 +1,34 @@
package flagutil
import (
"flag"
"os"
"testing"
)
var fooFlag Array
func init() {
os.Args = append(os.Args, "--fooFlag=foo", "--fooFlag=bar")
flag.Var(&fooFlag, "fooFlag", "test")
}
func TestMain(m *testing.M) {
flag.Parse()
os.Exit(m.Run())
}
func TestArray(t *testing.T) {
expected := map[string]struct{}{
"foo": {},
"bar": {},
}
if len(expected) != len(fooFlag) {
t.Errorf("len array flag (%d) is not equal to %d", len(fooFlag), len(expected))
}
for _, i := range fooFlag {
if _, ok := expected[i]; !ok {
t.Errorf("unexpected item in array %v", i)
}
}
}

View File

@ -23,13 +23,11 @@ func WritePrometheusMetrics(w io.Writer) {
fmt.Fprintf(w, "vm_app_start_timestamp %d\n", startTime.Unix())
fmt.Fprintf(w, "vm_app_uptime_seconds %d\n", int(time.Since(startTime).Seconds()))
// TODO: export other interesting stuff.
// Export flags as metrics.
flag.VisitAll(func(f *flag.Flag) {
lname := strings.ToLower(f.Name)
value := f.Value.String()
if strings.Contains(lname, "pass") || strings.Contains(lname, "key") || strings.Contains(lname, "secret") {
if isSecretFlag(lname) {
// Do not expose passwords and keys to prometheus.
value = "secret"
}
@ -38,3 +36,23 @@ func WritePrometheusMetrics(w io.Writer) {
}
var startTime = time.Now()
// RegisterSecretFlag registers flagName as secret.
//
// This function must be called before starting httpserver.
// It cannot be called from concurrent goroutines.
//
// Secret flags aren't exported at `/metrics` page.
func RegisterSecretFlag(flagName string) {
lname := strings.ToLower(flagName)
secretFlags[lname] = true
}
var secretFlags = make(map[string]bool)
func isSecretFlag(s string) bool {
if strings.Contains(s, "pass") || strings.Contains(s, "key") || strings.Contains(s, "secret") || strings.Contains(s, "token") {
return true
}
return secretFlags[s]
}

View File

@ -1,6 +1,7 @@
package graphite
import (
"io"
"net"
"runtime"
"strings"
@ -14,11 +15,11 @@ import (
)
var (
writeRequestsTCP = metrics.NewCounter(`vm_graphite_requests_total{name="write", net="tcp"}`)
writeErrorsTCP = metrics.NewCounter(`vm_graphite_request_errors_total{name="write", net="tcp"}`)
writeRequestsTCP = metrics.NewCounter(`vm_ingestserver_graphite_requests_total{name="write", net="tcp"}`)
writeErrorsTCP = metrics.NewCounter(`vm_ingestserver_graphite_request_errors_total{name="write", net="tcp"}`)
writeRequestsUDP = metrics.NewCounter(`vm_graphite_requests_total{name="write", net="udp"}`)
writeErrorsUDP = metrics.NewCounter(`vm_graphite_request_errors_total{name="write", net="udp"}`)
writeRequestsUDP = metrics.NewCounter(`vm_ingestserver_graphite_requests_total{name="write", net="udp"}`)
writeErrorsUDP = metrics.NewCounter(`vm_ingestserver_graphite_request_errors_total{name="write", net="udp"}`)
)
// Server accepts Graphite plaintext lines over TCP and UDP.
@ -31,8 +32,10 @@ type Server struct {
// MustStart starts graphite server on the given addr.
//
// The incoming connections are processed with insertHandler.
//
// MustStop must be called on the returned server when it is no longer needed.
func MustStart(addr string) *Server {
func MustStart(addr string, insertHandler func(r io.Reader) error) *Server {
logger.Infof("starting TCP Graphite server at %q", addr)
lnTCP, err := netutil.NewTCPListener("graphite", addr)
if err != nil {
@ -53,13 +56,13 @@ func MustStart(addr string) *Server {
s.wg.Add(1)
go func() {
defer s.wg.Done()
serveTCP(lnTCP)
serveTCP(lnTCP, insertHandler)
logger.Infof("stopped TCP Graphite server at %q", addr)
}()
s.wg.Add(1)
go func() {
defer s.wg.Done()
serveUDP(lnUDP)
serveUDP(lnUDP, insertHandler)
logger.Infof("stopped UDP Graphite server at %q", addr)
}()
return s
@ -79,7 +82,7 @@ func (s *Server) MustStop() {
logger.Infof("TCP and UDP Graphite servers at %q have been stopped", s.addr)
}
func serveTCP(ln net.Listener) {
func serveTCP(ln net.Listener, insertHandler func(r io.Reader) error) {
for {
c, err := ln.Accept()
if err != nil {
@ -107,7 +110,7 @@ func serveTCP(ln net.Listener) {
}
}
func serveUDP(ln net.PacketConn) {
func serveUDP(ln net.PacketConn, insertHandler func(r io.Reader) error) {
gomaxprocs := runtime.GOMAXPROCS(-1)
var wg sync.WaitGroup
for i := 0; i < gomaxprocs; i++ {

View File

@ -1,14 +1,16 @@
package opentsdb
import (
"io"
"net"
"net/http"
"runtime"
"strings"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/netutil"
"github.com/VictoriaMetrics/metrics"
@ -36,7 +38,7 @@ type Server struct {
// MustStart starts OpenTSDB collector on the given addr.
//
// MustStop must be called on the returned server when it is no longer needed.
func MustStart(addr string) *Server {
func MustStart(addr string, telnetInsertHandler func(r io.Reader) error, httpInsertHandler func(req *http.Request) error) *Server {
logger.Infof("starting TCP OpenTSDB collector at %q", addr)
lnTCP, err := netutil.NewTCPListener("opentsdb", addr)
if err != nil {
@ -45,7 +47,7 @@ func MustStart(addr string) *Server {
ls := newListenerSwitch(lnTCP)
lnHTTP := ls.newHTTPListener()
lnTelnet := ls.newTelnetListener()
httpServer := opentsdbhttp.MustServe(lnHTTP)
httpServer := opentsdbhttp.MustServe(lnHTTP, httpInsertHandler)
logger.Infof("starting UDP OpenTSDB collector at %q", addr)
lnUDP, err := net.ListenPacket("udp4", addr)
@ -62,7 +64,7 @@ func MustStart(addr string) *Server {
s.wg.Add(1)
go func() {
defer s.wg.Done()
serveTelnet(lnTelnet)
serveTelnet(lnTelnet, telnetInsertHandler)
logger.Infof("stopped TCP telnet OpenTSDB server at %q", addr)
}()
s.wg.Add(1)
@ -74,7 +76,7 @@ func MustStart(addr string) *Server {
s.wg.Add(1)
go func() {
defer s.wg.Done()
serveUDP(lnUDP)
serveUDP(lnUDP, telnetInsertHandler)
logger.Infof("stopped UDP OpenTSDB server at %q", addr)
}()
return s
@ -100,7 +102,7 @@ func (s *Server) MustStop() {
logger.Infof("TCP and UDP OpenTSDB servers at %q have been stopped", s.addr)
}
func serveTelnet(ln net.Listener) {
func serveTelnet(ln net.Listener, insertHandler func(r io.Reader) error) {
for {
c, err := ln.Accept()
if err != nil {
@ -128,7 +130,7 @@ func serveTelnet(ln net.Listener) {
}
}
func serveUDP(ln net.PacketConn) {
func serveUDP(ln net.PacketConn, insertHandler func(r io.Reader) error) {
gomaxprocs := runtime.GOMAXPROCS(-1)
var wg sync.WaitGroup
for i := 0; i < gomaxprocs; i++ {

View File

@ -14,8 +14,8 @@ import (
)
var (
writeRequests = metrics.NewCounter(`vm_http_requests_total{path="/api/put", protocol="opentsdb-http"}`)
writeErrors = metrics.NewCounter(`vm_http_request_errors_total{path="/api/put", protocol="opentsdb-http"}`)
writeRequests = metrics.NewCounter(`vm_opentsdbhttp_requests_total{name="write", net="tcp"}`)
writeErrors = metrics.NewCounter(`vm_opentsdbhttp_request_errors_total{name="write", net="tcp"}`)
)
// Server represents HTTP OpenTSDB server.
@ -28,20 +28,20 @@ type Server struct {
// MustStart starts HTTP OpenTSDB server on the given addr.
//
// MustStop must be called on the returned server when it is no longer needed.
func MustStart(addr string) *Server {
func MustStart(addr string, insertHandler func(req *http.Request) error) *Server {
logger.Infof("starting HTTP OpenTSDB server at %q", addr)
lnTCP, err := netutil.NewTCPListener("opentsdbhttp", addr)
if err != nil {
logger.Fatalf("cannot start HTTP OpenTSDB collector at %q: %s", addr, err)
}
return MustServe(lnTCP)
return MustServe(lnTCP, insertHandler)
}
// MustServe serves OpenTSDB HTTP put requests from ln.
//
// MustStop must be called on the returned server when it is no longer needed.
func MustServe(ln net.Listener) *Server {
h := newRequestHandler()
func MustServe(ln net.Listener, insertHandler func(req *http.Request) error) *Server {
h := newRequestHandler(insertHandler)
hs := &http.Server{
Handler: h,
ReadTimeout: 30 * time.Second,
@ -82,20 +82,15 @@ func (s *Server) MustStop() {
logger.Infof("OpenTSDB HTTP server at %q has been stopped", s.ln.Addr())
}
func newRequestHandler() http.Handler {
rh := func(w http.ResponseWriter, r *http.Request) {
switch r.URL.Path {
case "/api/put":
writeRequests.Inc()
if err := insertHandler(r); err != nil {
writeErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", r.URL.Path, err)
return
}
w.WriteHeader(http.StatusNoContent)
default:
httpserver.Errorf(w, "unexpected path requested on HTTP OpenTSDB server: %q", r.URL.Path)
func newRequestHandler(insertHandler func(req *http.Request) error) http.Handler {
rh := func(w http.ResponseWriter, req *http.Request) {
writeRequests.Inc()
if err := insertHandler(req); err != nil {
writeErrors.Inc()
httpserver.Errorf(w, "error in %q: %s", req.URL.Path, err)
return
}
w.WriteHeader(http.StatusNoContent)
}
return http.HandlerFunc(rh)
}

View File

@ -32,6 +32,11 @@ func NewTCPListener(name, addr string) (*TCPListener, error) {
return tln, err
}
// TCP6Enabled returns true if dialing and listening for IPv4 TCP is enabled.
func TCP6Enabled() bool {
return *enableTCP6
}
func getNetwork() string {
if *enableTCP6 {
// Enable both tcp4 and tcp6

View File

@ -0,0 +1,153 @@
package persistentqueue
import (
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
)
// FastQueue is a wrapper around Queue, which prefers sending data via memory.
//
// It falls back to sending data via file when readers don't catch up with writers.
type FastQueue struct {
// my protects the state of FastQueue.
mu sync.Mutex
// cond is used for notifying blocked readers when new data has been added
// or when MustClose is called.
cond sync.Cond
// q is file-based queue
q *Queue
// ch is in-memory queue
ch chan *bytesutil.ByteBuffer
pendingInmemoryBytes uint64
mustStop bool
}
// MustOpenFastQueue opens persistent queue at the given path.
//
// It holds up to maxInmemoryBlocks in memory before falling back to file-based persistence.
func MustOpenFastQueue(path, name string, maxInmemoryBlocks int) *FastQueue {
q := MustOpen(path, name)
fq := &FastQueue{
q: q,
ch: make(chan *bytesutil.ByteBuffer, maxInmemoryBlocks),
}
fq.cond.L = &fq.mu
logger.Infof("opened fast persistent queue at %q with maxInmemoryBlocks=%d", path, maxInmemoryBlocks)
return fq
}
// MustClose unblocks all the readers.
//
// It is expected no new writers during and after the call.
func (fq *FastQueue) MustClose() {
fq.mu.Lock()
defer fq.mu.Unlock()
// Unblock blocked readers
fq.mustStop = true
fq.cond.Broadcast()
// flush blocks from fq.ch to fq.q, so they can be persisted
fq.flushInmemoryBlocksToFileLocked()
// Close fq.q
fq.q.MustClose()
logger.Infof("closed fast persistent queue at %q", fq.q.dir)
}
func (fq *FastQueue) flushInmemoryBlocksToFileLocked() {
// fq.mu must be locked by the caller.
for len(fq.ch) > 0 {
bb := <-fq.ch
fq.q.MustWriteBlock(bb.B)
fq.pendingInmemoryBytes -= uint64(len(bb.B))
blockBufPool.Put(bb)
}
}
// GetPendingBytes returns the number of pending bytes in the fq.
func (fq *FastQueue) GetPendingBytes() uint64 {
fq.mu.Lock()
defer fq.mu.Unlock()
n := fq.pendingInmemoryBytes
n += fq.q.GetPendingBytes()
return n
}
// GetInmemoryQueueLen returns the length of inmemory queue.
func (fq *FastQueue) GetInmemoryQueueLen() int {
fq.mu.Lock()
defer fq.mu.Unlock()
return len(fq.ch)
}
// MustWriteBlock writes block to fq.
func (fq *FastQueue) MustWriteBlock(block []byte) {
fq.mu.Lock()
defer fq.mu.Unlock()
if n := fq.q.GetPendingBytes(); n > 0 {
// The file-based queue isn't drained yet. This means that in-memory queue cannot be used yet.
// So put the block to file-based queue.
if len(fq.ch) > 0 {
logger.Panicf("BUG: the in-memory queue must be empty when the file-based queue is non-empty; it contains %d pending bytes", n)
}
fq.q.MustWriteBlock(block)
return
}
if len(fq.ch) == cap(fq.ch) {
// There is no space in the in-memory queue. Put the data to file-based queue.
fq.flushInmemoryBlocksToFileLocked()
fq.q.MustWriteBlock(block)
return
}
// There is enough space in the in-memory queue.
bb := blockBufPool.Get()
bb.B = append(bb.B[:0], block...)
fq.ch <- bb
fq.pendingInmemoryBytes += uint64(len(block))
if len(fq.ch) == 1 {
// Notify potentially blocked reader
fq.cond.Signal()
}
}
// MustReadBlock reads the next block from fq to dst and returns it.
func (fq *FastQueue) MustReadBlock(dst []byte) ([]byte, bool) {
fq.mu.Lock()
defer fq.mu.Unlock()
for {
if fq.mustStop {
return dst, false
}
if len(fq.ch) > 0 {
if n := fq.q.GetPendingBytes(); n > 0 {
logger.Panicf("BUG: the file-based queue must be empty when the inmemory queue is empty; it contains %d pending bytes", n)
}
bb := <-fq.ch
fq.pendingInmemoryBytes -= uint64(len(bb.B))
dst = append(dst, bb.B...)
blockBufPool.Put(bb)
return dst, true
}
if n := fq.q.GetPendingBytes(); n > 0 {
return fq.q.MustReadBlock(dst)
}
// There are no blocks. Wait for new block.
fq.cond.Wait()
}
}
var blockBufPool bytesutil.ByteBufferPool

View File

@ -0,0 +1,274 @@
package persistentqueue
import (
"fmt"
"sync"
"testing"
"time"
)
func TestFastQueueOpenClose(t *testing.T) {
path := "fast-queue-open-close"
mustDeleteDir(path)
for i := 0; i < 10; i++ {
fq := MustOpenFastQueue(path, "foobar", 100)
fq.MustClose()
}
mustDeleteDir(path)
}
func TestFastQueueWriteReadInmemory(t *testing.T) {
path := "fast-queue-write-read-inmemory"
mustDeleteDir(path)
capacity := 100
fq := MustOpenFastQueue(path, "foobar", capacity)
var blocks []string
for i := 0; i < capacity; i++ {
block := fmt.Sprintf("block %d", i)
fq.MustWriteBlock([]byte(block))
blocks = append(blocks, block)
}
for _, block := range blocks {
buf, ok := fq.MustReadBlock(nil)
if !ok {
t.Fatalf("unexpected ok=false")
}
if string(buf) != block {
t.Fatalf("unexpected block read; got %q; want %q", buf, block)
}
}
fq.MustClose()
mustDeleteDir(path)
}
func TestFastQueueWriteReadMixed(t *testing.T) {
path := "fast-queue-write-read-mixed"
mustDeleteDir(path)
capacity := 100
fq := MustOpenFastQueue(path, "foobar", capacity)
if n := fq.GetPendingBytes(); n != 0 {
t.Fatalf("the number of pending bytes must be 0; got %d", n)
}
var blocks []string
for i := 0; i < 2*capacity; i++ {
block := fmt.Sprintf("block %d", i)
fq.MustWriteBlock([]byte(block))
blocks = append(blocks, block)
}
if n := fq.GetPendingBytes(); n == 0 {
t.Fatalf("the number of pending bytes must be greater than 0")
}
for _, block := range blocks {
buf, ok := fq.MustReadBlock(nil)
if !ok {
t.Fatalf("unexpected ok=false")
}
if string(buf) != block {
t.Fatalf("unexpected block read; got %q; want %q", buf, block)
}
}
if n := fq.GetPendingBytes(); n != 0 {
t.Fatalf("the number of pending bytes must be 0; got %d", n)
}
fq.MustClose()
mustDeleteDir(path)
}
func TestFastQueueWriteReadWithCloses(t *testing.T) {
path := "fast-queue-write-read-with-closes"
mustDeleteDir(path)
capacity := 100
fq := MustOpenFastQueue(path, "foobar", capacity)
if n := fq.GetPendingBytes(); n != 0 {
t.Fatalf("the number of pending bytes must be 0; got %d", n)
}
var blocks []string
for i := 0; i < 2*capacity; i++ {
block := fmt.Sprintf("block %d", i)
fq.MustWriteBlock([]byte(block))
blocks = append(blocks, block)
fq.MustClose()
fq = MustOpenFastQueue(path, "foobar", capacity)
}
if n := fq.GetPendingBytes(); n == 0 {
t.Fatalf("the number of pending bytes must be greater than 0")
}
for _, block := range blocks {
buf, ok := fq.MustReadBlock(nil)
if !ok {
t.Fatalf("unexpected ok=false")
}
if string(buf) != block {
t.Fatalf("unexpected block read; got %q; want %q", buf, block)
}
fq.MustClose()
fq = MustOpenFastQueue(path, "foobar", capacity)
}
if n := fq.GetPendingBytes(); n != 0 {
t.Fatalf("the number of pending bytes must be 0; got %d", n)
}
fq.MustClose()
mustDeleteDir(path)
}
func TestFastQueueReadUnblockByClose(t *testing.T) {
path := "fast-queue-read-unblock-by-close"
mustDeleteDir(path)
fq := MustOpenFastQueue(path, "foorbar", 123)
resultCh := make(chan error)
go func() {
data, ok := fq.MustReadBlock(nil)
if ok {
resultCh <- fmt.Errorf("unexpected ok=true")
return
}
if len(data) != 0 {
resultCh <- fmt.Errorf("unexpected non-empty data=%q", data)
return
}
resultCh <- nil
}()
fq.MustClose()
select {
case err := <-resultCh:
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
case <-time.After(time.Second):
t.Fatalf("timeout")
}
mustDeleteDir(path)
}
func TestFastQueueReadUnblockByWrite(t *testing.T) {
path := "fast-queue-read-unblock-by-write"
mustDeleteDir(path)
fq := MustOpenFastQueue(path, "foobar", 13)
block := fmt.Sprintf("foodsafdsaf sdf")
resultCh := make(chan error)
go func() {
data, ok := fq.MustReadBlock(nil)
if !ok {
resultCh <- fmt.Errorf("unexpected ok=false")
return
}
if string(data) != block {
resultCh <- fmt.Errorf("unexpected block read; got %q; want %q", data, block)
return
}
resultCh <- nil
}()
fq.MustWriteBlock([]byte(block))
select {
case err := <-resultCh:
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
case <-time.After(time.Second):
t.Fatalf("timeout")
}
fq.MustClose()
mustDeleteDir(path)
}
func TestFastQueueReadWriteConcurrent(t *testing.T) {
path := "fast-queue-read-write-concurrent"
mustDeleteDir(path)
fq := MustOpenFastQueue(path, "foobar", 5)
var blocks []string
blocksMap := make(map[string]bool)
var blocksMapLock sync.Mutex
for i := 0; i < 1000; i++ {
block := fmt.Sprintf("block %d", i)
blocks = append(blocks, block)
blocksMap[block] = true
}
// Start readers
var readersWG sync.WaitGroup
for i := 0; i < 10; i++ {
readersWG.Add(1)
go func() {
defer readersWG.Done()
for {
data, ok := fq.MustReadBlock(nil)
if !ok {
return
}
blocksMapLock.Lock()
if !blocksMap[string(data)] {
panic(fmt.Errorf("unexpected data read from the queue: %q", data))
}
delete(blocksMap, string(data))
blocksMapLock.Unlock()
}
}()
}
// Start writers
blocksCh := make(chan string)
var writersWG sync.WaitGroup
for i := 0; i < 10; i++ {
writersWG.Add(1)
go func() {
defer writersWG.Done()
for block := range blocksCh {
fq.MustWriteBlock([]byte(block))
}
}()
}
// feed writers
for _, block := range blocks {
blocksCh <- block
}
close(blocksCh)
// Wait for writers to finish
writersWG.Wait()
// wait for a while, so readers could catch up
time.Sleep(100 * time.Millisecond)
// Close fq
fq.MustClose()
// Wait for readers to finish
readersWG.Wait()
// Collect the remaining data
fq = MustOpenFastQueue(path, "foobar", 5)
resultCh := make(chan error)
go func() {
for len(blocksMap) > 0 {
data, ok := fq.MustReadBlock(nil)
if !ok {
resultCh <- fmt.Errorf("unexpected ok=false")
return
}
if !blocksMap[string(data)] {
resultCh <- fmt.Errorf("unexpected data read from fq: %q", data)
return
}
delete(blocksMap, string(data))
}
resultCh <- nil
}()
select {
case err := <-resultCh:
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
case <-time.After(time.Second * 5):
t.Fatalf("timeout")
}
fq.MustClose()
mustDeleteDir(path)
}

View File

@ -0,0 +1,66 @@
package persistentqueue
import (
"fmt"
"runtime"
"testing"
)
func BenchmarkFastQueueThroughputSerial(b *testing.B) {
const iterationsCount = 10
for _, blockSize := range []int{1e0, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6} {
block := make([]byte, blockSize)
b.Run(fmt.Sprintf("block-size-%d", blockSize), func(b *testing.B) {
b.ReportAllocs()
b.SetBytes(int64(blockSize) * iterationsCount)
path := fmt.Sprintf("bench-fast-queue-throughput-serial-%d", blockSize)
mustDeleteDir(path)
fq := MustOpenFastQueue(path, "foobar", iterationsCount*2)
defer func() {
fq.MustClose()
mustDeleteDir(path)
}()
for i := 0; i < b.N; i++ {
writeReadIterationFastQueue(fq, block, iterationsCount)
}
})
}
}
func BenchmarkFastQueueThroughputConcurrent(b *testing.B) {
const iterationsCount = 10
for _, blockSize := range []int{1e0, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6} {
block := make([]byte, blockSize)
b.Run(fmt.Sprintf("block-size-%d", blockSize), func(b *testing.B) {
b.ReportAllocs()
b.SetBytes(int64(blockSize) * iterationsCount)
path := fmt.Sprintf("bench-fast-queue-throughput-concurrent-%d", blockSize)
mustDeleteDir(path)
fq := MustOpenFastQueue(path, "foobar", iterationsCount*runtime.GOMAXPROCS(-1)*2)
defer func() {
fq.MustClose()
mustDeleteDir(path)
}()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
writeReadIterationFastQueue(fq, block, iterationsCount)
}
})
})
}
}
func writeReadIterationFastQueue(fq *FastQueue, block []byte, iterationsCount int) {
for i := 0; i < iterationsCount; i++ {
fq.MustWriteBlock(block)
}
var ok bool
bb := bbPool.Get()
for i := 0; i < iterationsCount; i++ {
bb.B, ok = fq.MustReadBlock(bb.B[:0])
if !ok {
panic(fmt.Errorf("unexpected ok=false"))
}
}
bbPool.Put(bb)
}

View File

@ -0,0 +1,489 @@
package persistentqueue
import (
"encoding/json"
"fmt"
"io"
"io/ioutil"
"os"
"regexp"
"strconv"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/filestream"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
)
// MaxBlockSize is the maximum size of the block persistent queue can work with.
const MaxBlockSize = 32 * 1024 * 1024
const defaultChunkFileSize = (MaxBlockSize + 8) * 16
var chunkFileNameRegex = regexp.MustCompile("^[0-9A-F]{16}$")
// Queue represents persistent queue.
type Queue struct {
chunkFileSize uint64
maxBlockSize uint64
dir string
name string
// mu protects all the fields below.
mu sync.Mutex
// cond is used for notifying blocked readers when new data has been added
// or when MustClose is called.
cond sync.Cond
reader *filestream.Reader
readerPath string
readerOffset uint64
readerLocalOffset uint64
writer *filestream.Writer
writerPath string
writerOffset uint64
writerLocalOffset uint64
writerFlushedOffset uint64
mustStop bool
}
// GetPendingBytes returns the number of pending bytes in the queue.
func (q *Queue) GetPendingBytes() uint64 {
q.mu.Lock()
n := q.writerOffset - q.readerOffset
q.mu.Unlock()
return n
}
// MustOpen opens persistent queue from the given path.
func MustOpen(path, name string) *Queue {
return mustOpen(path, name, defaultChunkFileSize, MaxBlockSize)
}
func mustOpen(path, name string, chunkFileSize, maxBlockSize uint64) *Queue {
if chunkFileSize < 8 || chunkFileSize-8 < maxBlockSize {
logger.Panicf("BUG: too small chunkFileSize=%d for maxBlockSize=%d; chunkFileSize must fit at least one block", chunkFileSize, maxBlockSize)
}
if maxBlockSize <= 0 {
logger.Panicf("BUG: maxBlockSize must be greater than 0; got %d", maxBlockSize)
}
q, err := tryOpeningQueue(path, name, chunkFileSize, maxBlockSize)
if err != nil {
logger.Errorf("cannot open persistent queue at %q: %s; cleaning it up and trying again", path, err)
fs.RemoveDirContents(path)
q, err = tryOpeningQueue(path, name, chunkFileSize, maxBlockSize)
if err != nil {
logger.Panicf("FATAL: %s", err)
}
}
return q
}
func tryOpeningQueue(path, name string, chunkFileSize, maxBlockSize uint64) (*Queue, error) {
var q Queue
q.chunkFileSize = chunkFileSize
q.maxBlockSize = maxBlockSize
q.dir = path
q.name = name
q.cond.L = &q.mu
cleanOnError := func() {
if q.reader != nil {
q.reader.MustClose()
}
if q.writer != nil {
q.writer.MustClose()
}
}
if err := fs.MkdirAllIfNotExist(path); err != nil {
return nil, fmt.Errorf("cannot create directory %q: %s", path, err)
}
// Read metainfo.
var mi metainfo
metainfoPath := q.metainfoPath()
if err := mi.ReadFromFile(metainfoPath); err != nil {
if !os.IsNotExist(err) {
logger.Errorf("cannot read metainfo for persistent queue from %q: %s; re-creating %q", metainfoPath, err, path)
}
// path contents is broken or missing. Re-create it from scratch.
fs.RemoveDirContents(path)
mi.Reset()
mi.Name = q.name
if err := mi.WriteToFile(metainfoPath); err != nil {
return nil, fmt.Errorf("cannot create %q: %s", metainfoPath, err)
}
// Create initial chunk file.
filepath := q.chunkFilePath(0)
if err := fs.WriteFileAtomically(filepath, nil); err != nil {
return nil, fmt.Errorf("cannot create %q: %s", filepath, err)
}
}
if mi.Name != q.name {
return nil, fmt.Errorf("unexpected queue name; got %q; want %q", mi.Name, q.name)
}
// Locate reader and writer chunks in the path.
fis, err := ioutil.ReadDir(path)
if err != nil {
return nil, fmt.Errorf("cannot read contents of the directory %q: %s", path, err)
}
for _, fi := range fis {
fname := fi.Name()
filepath := path + "/" + fname
if fi.IsDir() {
logger.Errorf("skipping unknown directory %q", filepath)
continue
}
if fname == "metainfo.json" {
// skip metainfo file
continue
}
if !chunkFileNameRegex.MatchString(fname) {
logger.Errorf("skipping unknown file %q", filepath)
continue
}
offset, err := strconv.ParseUint(fname, 16, 64)
if err != nil {
logger.Panicf("BUG: cannot parse hex %q: %s", fname, err)
}
if offset%q.chunkFileSize != 0 {
logger.Errorf("unexpected offset for chunk file %q: %d; it must divide by %d; removing the file", filepath, offset, q.chunkFileSize)
fs.MustRemoveAll(filepath)
continue
}
if mi.ReaderOffset >= offset+q.chunkFileSize {
logger.Errorf("unexpected chunk file found from the past: %q; removing it", filepath)
fs.MustRemoveAll(filepath)
continue
}
if mi.WriterOffset < offset {
logger.Errorf("unexpected chunk file found from the future: %q; removing it", filepath)
fs.MustRemoveAll(filepath)
continue
}
if mi.ReaderOffset >= offset && mi.ReaderOffset < offset+q.chunkFileSize {
// Found the chunk for reading
if q.reader != nil {
logger.Panicf("BUG: reader is already initialized with readerPath=%q, readerOffset=%d, readerLocalOffset=%d",
q.readerPath, q.readerOffset, q.readerLocalOffset)
}
q.readerPath = filepath
q.readerOffset = mi.ReaderOffset
q.readerLocalOffset = mi.ReaderOffset % q.chunkFileSize
if fileSize := fs.MustFileSize(q.readerPath); fileSize < q.readerLocalOffset {
logger.Errorf("chunk file %q size is too small for the given reader offset; file size %d bytes; reader offset: %d bytes; removing the file",
q.readerPath, fileSize, q.readerLocalOffset)
fs.MustRemoveAll(q.readerPath)
continue
}
r, err := filestream.OpenReaderAt(q.readerPath, int64(q.readerLocalOffset), true)
if err != nil {
logger.Errorf("cannot open %q for reading at offset %d: %s; removing this file", q.readerPath, q.readerLocalOffset, err)
fs.MustRemoveAll(filepath)
continue
}
q.reader = r
}
if mi.WriterOffset >= offset && mi.WriterOffset < offset+q.chunkFileSize {
// Found the chunk file for writing
if q.writer != nil {
logger.Panicf("BUG: writer is already initialized with writerPath=%q, writerOffset=%d, writerLocalOffset=%d",
q.writerPath, q.writerOffset, q.writerLocalOffset)
}
q.writerPath = filepath
q.writerOffset = mi.WriterOffset
q.writerLocalOffset = mi.WriterOffset % q.chunkFileSize
q.writerFlushedOffset = mi.WriterOffset
if fileSize := fs.MustFileSize(q.writerPath); fileSize != q.writerLocalOffset {
logger.Errorf("chunk file %q size doesn't match writer offset; file size %d bytes; writer offset: %d bytes",
q.writerPath, fileSize, q.writerLocalOffset)
fs.MustRemoveAll(q.writerPath)
continue
}
w, err := filestream.OpenWriterAt(q.writerPath, int64(q.writerLocalOffset), false)
if err != nil {
logger.Errorf("cannot open %q for writing at offset %d: %s; removing this file", q.writerPath, q.writerLocalOffset, err)
fs.MustRemoveAll(filepath)
continue
}
q.writer = w
}
}
if q.reader == nil {
cleanOnError()
return nil, fmt.Errorf("couldn't find chunk file for reading in %q", q.dir)
}
if q.writer == nil {
cleanOnError()
return nil, fmt.Errorf("couldn't find chunk file for writing in %q", q.dir)
}
return &q, nil
}
// MustClose closes q.
//
// It unblocks all the MustReadBlock calls.
//
// MustWriteBlock mustn't be called during and after the call to MustClose.
func (q *Queue) MustClose() {
q.mu.Lock()
defer q.mu.Unlock()
// Unblock goroutines blocked on cond in MustReadBlock.
q.mustStop = true
q.cond.Broadcast()
// Close writer.
q.writer.MustClose()
q.writer = nil
// Close reader.
q.reader.MustClose()
q.reader = nil
// Store metainfo
if err := q.flushMetainfo(); err != nil {
logger.Panicf("FATAL: cannot flush chunked queue metainfo: %s", err)
}
}
func (q *Queue) chunkFilePath(offset uint64) string {
return fmt.Sprintf("%s/%016X", q.dir, offset)
}
func (q *Queue) metainfoPath() string {
return q.dir + "/metainfo.json"
}
// MustWriteBlock writes block to q.
//
// The block size cannot exceed MaxBlockSize.
//
// It is safe calling this function from concurrent goroutines.
func (q *Queue) MustWriteBlock(block []byte) {
if uint64(len(block)) > q.maxBlockSize {
logger.Panicf("BUG: too big block to send: %d bytes; it mustn't exceed %d bytes", len(block), q.maxBlockSize)
}
q.mu.Lock()
defer q.mu.Unlock()
if q.mustStop {
logger.Panicf("BUG: MustWriteBlock cannot be called after MustClose")
}
if q.readerOffset > q.writerOffset {
logger.Panicf("BUG: readerOffset=%d shouldn't exceed writerOffset=%d", q.readerOffset, q.writerOffset)
}
mustNotifyReader := q.readerOffset == q.writerOffset
if err := q.writeBlockLocked(block); err != nil {
logger.Panicf("FATAL: %s", err)
}
if mustNotifyReader {
q.cond.Signal()
}
}
func (q *Queue) writeBlockLocked(block []byte) error {
if q.writerLocalOffset+q.maxBlockSize+8 > q.chunkFileSize {
// Finalize the current chunk and start new one.
q.writer.MustClose()
if n := q.writerOffset % q.chunkFileSize; n > 0 {
q.writerOffset += (q.chunkFileSize - n)
}
q.writerFlushedOffset = q.writerOffset
q.writerLocalOffset = 0
q.writerPath = q.chunkFilePath(q.writerOffset)
w, err := filestream.Create(q.writerPath, false)
if err != nil {
return fmt.Errorf("cannot create chunk file %q: %s", q.writerPath, err)
}
q.writer = w
if err := q.flushMetainfo(); err != nil {
return fmt.Errorf("cannot flush metainfo: %s", err)
}
}
// Write block len.
blockLen := uint64(len(block))
header := headerBufPool.Get()
header.B = encoding.MarshalUint64(header.B, blockLen)
err := q.write(header.B)
headerBufPool.Put(header)
if err != nil {
return fmt.Errorf("cannot write header with size 8 bytes to %q: %s", q.writerPath, err)
}
// Write block contents.
if err := q.write(block); err != nil {
return fmt.Errorf("cannot write block contents with size %d bytes to %q: %s", len(block), q.writerPath, err)
}
return nil
}
// MustReadBlock appends the next block from q to dst and returns the result.
//
// false is returned after MustClose call.
//
// It is safe calling this function from concurrent goroutines.
func (q *Queue) MustReadBlock(dst []byte) ([]byte, bool) {
q.mu.Lock()
defer q.mu.Unlock()
for {
if q.mustStop {
return dst, false
}
if q.readerOffset > q.writerOffset {
logger.Panicf("BUG: readerOffset=%d cannot exceed writerOffset=%d", q.readerOffset, q.writerOffset)
}
if q.readerOffset < q.writerOffset {
break
}
q.cond.Wait()
}
data, err := q.readBlockLocked(dst)
if err != nil {
logger.Panicf("FATAL: %s", err)
}
return data, true
}
func (q *Queue) readBlockLocked(dst []byte) ([]byte, error) {
if q.readerLocalOffset+q.maxBlockSize+8 > q.chunkFileSize {
// Remove the current chunk and go to the next chunk.
q.reader.MustClose()
fs.MustRemoveAll(q.readerPath)
if n := q.readerOffset % q.chunkFileSize; n > 0 {
q.readerOffset += (q.chunkFileSize - n)
}
q.readerLocalOffset = 0
q.readerPath = q.chunkFilePath(q.readerOffset)
r, err := filestream.Open(q.readerPath, true)
if err != nil {
return dst, fmt.Errorf("cannot open chunk file %q: %s", q.readerPath, err)
}
q.reader = r
if err := q.flushMetainfo(); err != nil {
return dst, fmt.Errorf("cannot flush metainfo: %s", err)
}
}
// Read block len.
header := headerBufPool.Get()
header.B = bytesutil.Resize(header.B, 8)
err := q.readFull(header.B)
blockLen := encoding.UnmarshalUint64(header.B)
headerBufPool.Put(header)
if err != nil {
return dst, fmt.Errorf("cannot read header with size 8 bytes from %q: %s", q.readerPath, err)
}
if blockLen > q.maxBlockSize {
return dst, fmt.Errorf("too big block size read from %q: %d bytes; cannot exceed %d bytes", q.readerPath, blockLen, q.maxBlockSize)
}
// Read block contents.
dstLen := len(dst)
dst = bytesutil.Resize(dst, dstLen+int(blockLen))
if err := q.readFull(dst[dstLen:]); err != nil {
return dst, fmt.Errorf("cannot read block contents with size %d bytes from %q: %s", blockLen, q.readerPath, err)
}
return dst, nil
}
func (q *Queue) write(buf []byte) error {
bufLen := uint64(len(buf))
n, err := q.writer.Write(buf)
if err != nil {
return err
}
if uint64(n) != bufLen {
return fmt.Errorf("unexpected number of bytes written; got %d bytes; want %d bytes", n, bufLen)
}
q.writerLocalOffset += bufLen
q.writerOffset += bufLen
return nil
}
func (q *Queue) readFull(buf []byte) error {
bufLen := uint64(len(buf))
if q.readerOffset+bufLen > q.writerFlushedOffset {
q.writer.MustFlush()
q.writerFlushedOffset = q.writerOffset
}
n, err := io.ReadFull(q.reader, buf)
if err != nil {
return err
}
if uint64(n) != bufLen {
return fmt.Errorf("unexpected number of bytes read; got %d bytes; want %d bytes", n, bufLen)
}
q.readerLocalOffset += bufLen
q.readerOffset += bufLen
return nil
}
func (q *Queue) flushMetainfo() error {
mi := &metainfo{
Name: q.name,
ReaderOffset: q.readerOffset,
WriterOffset: q.writerOffset,
}
metainfoPath := q.metainfoPath()
if err := mi.WriteToFile(metainfoPath); err != nil {
return fmt.Errorf("cannot write metainfo to %q: %s", metainfoPath, err)
}
return nil
}
var headerBufPool bytesutil.ByteBufferPool
type metainfo struct {
Name string
ReaderOffset uint64
WriterOffset uint64
}
func (mi *metainfo) Reset() {
mi.ReaderOffset = 0
mi.WriterOffset = 0
}
func (mi *metainfo) WriteToFile(path string) error {
data, err := json.Marshal(mi)
if err != nil {
return fmt.Errorf("cannot marshal persistent queue metainfo %#v: %s", mi, err)
}
if err := ioutil.WriteFile(path, data, 0600); err != nil {
return fmt.Errorf("cannot write persistent queue metainfo to %q: %s", path, err)
}
return nil
}
func (mi *metainfo) ReadFromFile(path string) error {
mi.Reset()
data, err := ioutil.ReadFile(path)
if err != nil {
if os.IsNotExist(err) {
return err
}
return fmt.Errorf("cannot read %q: %s", path, err)
}
if err := json.Unmarshal(data, mi); err != nil {
return fmt.Errorf("cannot unmarshal persistent queue metainfo from %q: %s", path, err)
}
if mi.ReaderOffset > mi.WriterOffset {
return fmt.Errorf("invalid data read from %q: readerOffset=%d cannot exceed writerOffset=%d", path, mi.ReaderOffset, mi.WriterOffset)
}
return nil
}

View File

@ -0,0 +1,436 @@
package persistentqueue
import (
"fmt"
"io/ioutil"
"os"
"sync"
"testing"
"time"
)
func TestQueueOpenClose(t *testing.T) {
path := "queue-open-close"
mustDeleteDir(path)
for i := 0; i < 3; i++ {
q := MustOpen(path, "foobar")
if n := q.GetPendingBytes(); n > 0 {
t.Fatalf("pending bytes must be 0; got %d", n)
}
q.MustClose()
}
mustDeleteDir(path)
}
func TestQueueOpen(t *testing.T) {
t.Run("invalid-metainfo", func(t *testing.T) {
path := "queue-open-invalid-metainfo"
mustCreateDir(path)
mustCreateFile(path+"/metainfo.json", "foobarbaz")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("junk-files-and-dirs", func(t *testing.T) {
path := "queue-open-junk-files-and-dir"
mustCreateDir(path)
mustCreateEmptyMetainfo(path, "foobar")
mustCreateFile(path+"/junk-file", "foobar")
mustCreateDir(path + "/junk-dir")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("invalid-chunk-offset", func(t *testing.T) {
path := "queue-open-invalid-chunk-offset"
mustCreateDir(path)
mustCreateEmptyMetainfo(path, "foobar")
mustCreateFile(fmt.Sprintf("%s/%016X", path, 1234), "qwere")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("too-new-chunk", func(t *testing.T) {
path := "queue-open-too-new-chunk"
mustCreateDir(path)
mustCreateEmptyMetainfo(path, "foobar")
mustCreateFile(fmt.Sprintf("%s/%016X", path, 100*uint64(defaultChunkFileSize)), "asdf")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("too-old-chunk", func(t *testing.T) {
path := "queue-open-too-old-chunk"
mustCreateDir(path)
mi := &metainfo{
Name: "foobar",
ReaderOffset: defaultChunkFileSize,
WriterOffset: defaultChunkFileSize,
}
if err := mi.WriteToFile(path + "/metainfo.json"); err != nil {
t.Fatalf("unexpected error: %s", err)
}
mustCreateFile(fmt.Sprintf("%s/%016X", path, 0), "adfsfd")
q := MustOpen(path, mi.Name)
q.MustClose()
mustDeleteDir(path)
})
t.Run("too-big-reader-offset", func(t *testing.T) {
path := "queue-open-too-big-reader-offset"
mustCreateDir(path)
mi := &metainfo{
Name: "foobar",
ReaderOffset: defaultChunkFileSize + 123,
}
if err := mi.WriteToFile(path + "/metainfo.json"); err != nil {
t.Fatalf("unexpected error: %s", err)
}
q := MustOpen(path, mi.Name)
q.MustClose()
mustDeleteDir(path)
})
t.Run("metainfo-dir", func(t *testing.T) {
path := "queue-open-metainfo-dir"
mustCreateDir(path)
mustCreateDir(path + "/metainfo.json")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("too-small-reader-file", func(t *testing.T) {
path := "too-small-reader-file"
mustCreateDir(path)
mi := &metainfo{
Name: "foobar",
ReaderOffset: 123,
WriterOffset: 123,
}
if err := mi.WriteToFile(path + "/metainfo.json"); err != nil {
t.Fatalf("unexpected error: %s", err)
}
mustCreateFile(fmt.Sprintf("%s/%016X", path, 0), "sdf")
q := MustOpen(path, mi.Name)
q.MustClose()
mustDeleteDir(path)
})
t.Run("invalid-writer-file-size", func(t *testing.T) {
path := "too-small-reader-file"
mustCreateDir(path)
mustCreateEmptyMetainfo(path, "foobar")
mustCreateFile(fmt.Sprintf("%s/%016X", path, 0), "sdfdsf")
q := MustOpen(path, "foobar")
q.MustClose()
mustDeleteDir(path)
})
t.Run("invalid-queue-name", func(t *testing.T) {
path := "invalid-queue-name"
mustCreateDir(path)
mi := &metainfo{
Name: "foobar",
}
if err := mi.WriteToFile(path + "/metainfo.json"); err != nil {
t.Fatalf("unexpected error: %s", err)
}
mustCreateFile(fmt.Sprintf("%s/%016X", path, 0), "sdf")
q := MustOpen(path, "baz")
q.MustClose()
mustDeleteDir(path)
})
}
func TestQueueWriteRead(t *testing.T) {
path := "queue-write-read"
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer func() {
q.MustClose()
mustDeleteDir(path)
}()
for j := 0; j < 5; j++ {
var blocks [][]byte
for i := 0; i < 10; i++ {
block := []byte(fmt.Sprintf("block %d+%d", j, i))
q.MustWriteBlock(block)
blocks = append(blocks, block)
}
if n := q.GetPendingBytes(); n <= 0 {
t.Fatalf("pending bytes must be greater than 0; got %d", n)
}
var buf []byte
var ok bool
for _, block := range blocks {
buf, ok = q.MustReadBlock(buf[:0])
if !ok {
t.Fatalf("unexpected ok=%v returned from MustReadBlock; want true", ok)
}
if string(buf) != string(block) {
t.Fatalf("unexpected block read; got %q; want %q", buf, block)
}
}
if n := q.GetPendingBytes(); n > 0 {
t.Fatalf("pending bytes must be 0; got %d", n)
}
}
}
func TestQueueWriteCloseRead(t *testing.T) {
path := "queue-write-close-read"
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer func() {
q.MustClose()
mustDeleteDir(path)
}()
for j := 0; j < 5; j++ {
var blocks [][]byte
for i := 0; i < 10; i++ {
block := []byte(fmt.Sprintf("block %d+%d", j, i))
q.MustWriteBlock(block)
blocks = append(blocks, block)
}
if n := q.GetPendingBytes(); n <= 0 {
t.Fatalf("pending bytes must be greater than 0; got %d", n)
}
q.MustClose()
q = MustOpen(path, "foobar")
if n := q.GetPendingBytes(); n <= 0 {
t.Fatalf("pending bytes must be greater than 0; got %d", n)
}
var buf []byte
var ok bool
for _, block := range blocks {
buf, ok = q.MustReadBlock(buf[:0])
if !ok {
t.Fatalf("unexpected ok=%v returned from MustReadBlock; want true", ok)
}
if string(buf) != string(block) {
t.Fatalf("unexpected block read; got %q; want %q", buf, block)
}
}
if n := q.GetPendingBytes(); n > 0 {
t.Fatalf("pending bytes must be 0; got %d", n)
}
}
}
func TestQueueReadEmpty(t *testing.T) {
path := "queue-read-empty"
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer mustDeleteDir(path)
resultCh := make(chan error)
go func() {
data, ok := q.MustReadBlock(nil)
var err error
if ok {
err = fmt.Errorf("unexpected ok=%v returned from MustReadBlock; want false", ok)
} else if len(data) > 0 {
err = fmt.Errorf("unexpected non-empty data returned from MustReadBlock: %q", data)
}
resultCh <- err
}()
if n := q.GetPendingBytes(); n > 0 {
t.Fatalf("pending bytes must be 0; got %d", n)
}
q.MustClose()
select {
case err := <-resultCh:
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
case <-time.After(time.Second):
t.Fatalf("timeout")
}
}
func TestQueueReadWriteConcurrent(t *testing.T) {
path := "queue-read-write-concurrent"
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer mustDeleteDir(path)
blocksMap := make(map[string]bool, 1000)
var blocksMapLock sync.Mutex
blocks := make([]string, 1000)
for i := 0; i < 1000; i++ {
block := fmt.Sprintf("block #%d", i)
blocksMap[block] = true
blocks[i] = block
}
// Start block readers
var readersWG sync.WaitGroup
for workerID := 0; workerID < 10; workerID++ {
readersWG.Add(1)
go func() {
defer readersWG.Done()
for {
block, ok := q.MustReadBlock(nil)
if !ok {
return
}
blocksMapLock.Lock()
if !blocksMap[string(block)] {
panic(fmt.Errorf("unexpected block read: %q", block))
}
delete(blocksMap, string(block))
blocksMapLock.Unlock()
}
}()
}
// Start block writers
blocksCh := make(chan string)
var writersWG sync.WaitGroup
for workerID := 0; workerID < 10; workerID++ {
writersWG.Add(1)
go func(workerID int) {
defer writersWG.Done()
for block := range blocksCh {
q.MustWriteBlock([]byte(block))
}
}(workerID)
}
for _, block := range blocks {
blocksCh <- block
}
close(blocksCh)
// Wait for block writers to finish
writersWG.Wait()
// Notify readers that the queue is closed
q.MustClose()
// Wait for block readers to finish
readersWG.Wait()
// Read the remaining blocks in q.
q = MustOpen(path, "foobar")
defer q.MustClose()
resultCh := make(chan error)
go func() {
for len(blocksMap) > 0 {
block, ok := q.MustReadBlock(nil)
if !ok {
resultCh <- fmt.Errorf("unexpected ok=false returned from MustReadBlock")
return
}
if !blocksMap[string(block)] {
resultCh <- fmt.Errorf("unexpected block read from the queue: %q", block)
return
}
delete(blocksMap, string(block))
}
resultCh <- nil
}()
select {
case err := <-resultCh:
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
case <-time.After(5 * time.Second):
t.Fatalf("timeout")
}
if n := q.GetPendingBytes(); n > 0 {
t.Fatalf("pending bytes must be 0; got %d", n)
}
}
func TestQueueChunkManagementSimple(t *testing.T) {
path := "queue-chunk-management-simple"
mustDeleteDir(path)
const chunkFileSize = 100
const maxBlockSize = 20
q := mustOpen(path, "foobar", chunkFileSize, maxBlockSize)
defer mustDeleteDir(path)
defer q.MustClose()
var blocks []string
for i := 0; i < 100; i++ {
block := fmt.Sprintf("block %d", i)
q.MustWriteBlock([]byte(block))
blocks = append(blocks, block)
}
if n := q.GetPendingBytes(); n == 0 {
t.Fatalf("unexpected zero number of bytes pending")
}
for _, block := range blocks {
data, ok := q.MustReadBlock(nil)
if !ok {
t.Fatalf("unexpected ok=false")
}
if block != string(data) {
t.Fatalf("unexpected block read; got %q; want %q", data, block)
}
}
if n := q.GetPendingBytes(); n != 0 {
t.Fatalf("unexpected non-zero number of pending bytes: %d", n)
}
}
func TestQueueChunkManagementPeriodicClose(t *testing.T) {
path := "queue-chunk-management-periodic-close"
mustDeleteDir(path)
const chunkFileSize = 100
const maxBlockSize = 20
q := mustOpen(path, "foobar", chunkFileSize, maxBlockSize)
defer func() {
q.MustClose()
mustDeleteDir(path)
}()
var blocks []string
for i := 0; i < 100; i++ {
block := fmt.Sprintf("block %d", i)
q.MustWriteBlock([]byte(block))
blocks = append(blocks, block)
q.MustClose()
q = mustOpen(path, "foobar", chunkFileSize, maxBlockSize)
}
if n := q.GetPendingBytes(); n == 0 {
t.Fatalf("unexpected zero number of bytes pending")
}
for _, block := range blocks {
data, ok := q.MustReadBlock(nil)
if !ok {
t.Fatalf("unexpected ok=false")
}
if block != string(data) {
t.Fatalf("unexpected block read; got %q; want %q", data, block)
}
q.MustClose()
q = mustOpen(path, "foobar", chunkFileSize, maxBlockSize)
}
if n := q.GetPendingBytes(); n != 0 {
t.Fatalf("unexpected non-zero number of pending bytes: %d", n)
}
}
func mustCreateFile(path, contents string) {
if err := ioutil.WriteFile(path, []byte(contents), 0600); err != nil {
panic(fmt.Errorf("cannot create file %q with %d bytes contents: %s", path, len(contents), err))
}
}
func mustCreateDir(path string) {
mustDeleteDir(path)
if err := os.MkdirAll(path, 0700); err != nil {
panic(fmt.Errorf("cannot create dir %q: %s", path, err))
}
}
func mustDeleteDir(path string) {
if err := os.RemoveAll(path); err != nil {
panic(fmt.Errorf("cannot remove dir %q: %s", path, err))
}
}
func mustCreateEmptyMetainfo(path, name string) {
var mi metainfo
mi.Name = name
if err := mi.WriteToFile(path + "/metainfo.json"); err != nil {
panic(fmt.Errorf("cannot create metainfo: %s", err))
}
}

View File

@ -0,0 +1,69 @@
package persistentqueue
import (
"fmt"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
)
func BenchmarkQueueThroughputSerial(b *testing.B) {
const iterationsCount = 10
for _, blockSize := range []int{1e0, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6} {
block := make([]byte, blockSize)
b.Run(fmt.Sprintf("block-size-%d", blockSize), func(b *testing.B) {
b.ReportAllocs()
b.SetBytes(int64(blockSize) * iterationsCount)
path := fmt.Sprintf("bench-queue-throughput-serial-%d", blockSize)
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer func() {
q.MustClose()
mustDeleteDir(path)
}()
for i := 0; i < b.N; i++ {
writeReadIteration(q, block, iterationsCount)
}
})
}
}
func BenchmarkQueueThroughputConcurrent(b *testing.B) {
const iterationsCount = 10
for _, blockSize := range []int{1e0, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6} {
block := make([]byte, blockSize)
b.Run(fmt.Sprintf("block-size-%d", blockSize), func(b *testing.B) {
b.ReportAllocs()
b.SetBytes(int64(blockSize) * iterationsCount)
path := fmt.Sprintf("bench-queue-throughput-concurrent-%d", blockSize)
mustDeleteDir(path)
q := MustOpen(path, "foobar")
defer func() {
q.MustClose()
mustDeleteDir(path)
}()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
writeReadIteration(q, block, iterationsCount)
}
})
})
}
}
func writeReadIteration(q *Queue, block []byte, iterationsCount int) {
for i := 0; i < iterationsCount; i++ {
q.MustWriteBlock(block)
}
var ok bool
bb := bbPool.Get()
for i := 0; i < iterationsCount; i++ {
bb.B, ok = q.MustReadBlock(bb.B[:0])
if !ok {
panic(fmt.Errorf("unexpected ok=false"))
}
}
bbPool.Put(bb)
}
var bbPool bytesutil.ByteBufferPool

View File

@ -0,0 +1,79 @@
// Code generated by protoc-gen-gogo. DO NOT EDIT.
// source: remote.proto
package prompbmarshal
import (
math_bits "math/bits"
)
type WriteRequest struct {
Timeseries []TimeSeries `protobuf:"bytes,1,rep,name=timeseries,proto3" json:"timeseries"`
}
func (m *WriteRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *WriteRequest) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *WriteRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Timeseries) > 0 {
for iNdEx := len(m.Timeseries) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Timeseries[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintRemote(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
}
func encodeVarintRemote(dAtA []byte, offset int, v uint64) int {
offset -= sovRemote(v)
base := offset
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
dAtA[offset] = uint8(v)
return base
}
func (m *WriteRequest) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.Timeseries) > 0 {
for _, e := range m.Timeseries {
l = e.Size()
n += 1 + l + sovRemote(uint64(l))
}
}
return n
}
func sovRemote(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}

View File

@ -0,0 +1,82 @@
// Copyright 2016 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
package prometheus;
option go_package = "prompbmarshal";
import "types.proto";
import "gogoproto/gogo.proto";
message WriteRequest {
repeated prometheus.TimeSeries timeseries = 1 [(gogoproto.nullable) = false];
}
// ReadRequest represents a remote read request.
message ReadRequest {
repeated Query queries = 1;
enum ResponseType {
// Server will return a single ReadResponse message with matched series that includes list of raw samples.
// It's recommended to use streamed response types instead.
//
// Response headers:
// Content-Type: "application/x-protobuf"
// Content-Encoding: "snappy"
SAMPLES = 0;
// Server will stream a delimited ChunkedReadResponse message that contains XOR encoded chunks for a single series.
// Each message is following varint size and fixed size bigendian uint32 for CRC32 Castagnoli checksum.
//
// Response headers:
// Content-Type: "application/x-streamed-protobuf; proto=prometheus.ChunkedReadResponse"
// Content-Encoding: ""
STREAMED_XOR_CHUNKS = 1;
}
// accepted_response_types allows negotiating the content type of the response.
//
// Response types are taken from the list in the FIFO order. If no response type in `accepted_response_types` is
// implemented by server, error is returned.
// For request that do not contain `accepted_response_types` field the SAMPLES response type will be used.
repeated ResponseType accepted_response_types = 2;
}
// ReadResponse is a response when response_type equals SAMPLES.
message ReadResponse {
// In same order as the request's queries.
repeated QueryResult results = 1;
}
message Query {
int64 start_timestamp_ms = 1;
int64 end_timestamp_ms = 2;
repeated prometheus.LabelMatcher matchers = 3;
prometheus.ReadHints hints = 4;
}
message QueryResult {
// Samples within a time series must be ordered by time.
repeated prometheus.TimeSeries timeseries = 1;
}
// ChunkedReadResponse is a response when response_type equals STREAMED_XOR_CHUNKS.
// We strictly stream full series after series, optionally split by time. This means that a single frame can contain
// partition of the single series, but once a new series is started to be streamed it means that no more chunks will
// be sent for previous one. Series are returned sorted in the same way TSDB block are internally.
message ChunkedReadResponse {
repeated prometheus.ChunkedSeries chunked_series = 1;
// query_index represents an index of the query from ReadRequest.queries these chunks relates to.
int64 query_index = 2;
}

View File

@ -0,0 +1,216 @@
// Code generated by protoc-gen-gogo. DO NOT EDIT.
// source: types.proto
package prompbmarshal
import (
encoding_binary "encoding/binary"
math "math"
math_bits "math/bits"
)
type Sample struct {
Value float64 `protobuf:"fixed64,1,opt,name=value,proto3" json:"value,omitempty"`
Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
}
// TimeSeries represents samples and labels for a single time series.
type TimeSeries struct {
Labels []Label `protobuf:"bytes,1,rep,name=labels,proto3" json:"labels"`
Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"`
}
type Label struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
}
func (m *Sample) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *Sample) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if m.Timestamp != 0 {
i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp))
i--
dAtA[i] = 0x10
}
if m.Value != 0 {
i -= 8
encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value))))
i--
dAtA[i] = 0x9
}
return len(dAtA) - i, nil
}
func (m *TimeSeries) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *TimeSeries) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Samples) > 0 {
for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintTypes(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
}
}
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 = encodeVarintTypes(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
}
func (m *Label) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *Label) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *Label) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Value) > 0 {
i -= len(m.Value)
copy(dAtA[i:], m.Value)
i = encodeVarintTypes(dAtA, i, uint64(len(m.Value)))
i--
dAtA[i] = 0x12
}
if len(m.Name) > 0 {
i -= len(m.Name)
copy(dAtA[i:], m.Name)
i = encodeVarintTypes(dAtA, i, uint64(len(m.Name)))
i--
dAtA[i] = 0xa
}
return len(dAtA) - i, nil
}
func encodeVarintTypes(dAtA []byte, offset int, v uint64) int {
offset -= sovTypes(v)
base := offset
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
dAtA[offset] = uint8(v)
return base
}
func (m *Sample) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Value != 0 {
n += 9
}
if m.Timestamp != 0 {
n += 1 + sovTypes(uint64(m.Timestamp))
}
return n
}
func (m *TimeSeries) 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 + sovTypes(uint64(l))
}
}
if len(m.Samples) > 0 {
for _, e := range m.Samples {
l = e.Size()
n += 1 + l + sovTypes(uint64(l))
}
}
return n
}
func (m *Label) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
l = len(m.Name)
if l > 0 {
n += 1 + l + sovTypes(uint64(l))
}
l = len(m.Value)
if l > 0 {
n += 1 + l + sovTypes(uint64(l))
}
return n
}
func sovTypes(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}

View File

@ -0,0 +1,85 @@
// Copyright 2017 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
package prometheus;
option go_package = "prompbmarshal";
import "gogoproto/gogo.proto";
message Sample {
double value = 1;
int64 timestamp = 2;
}
// TimeSeries represents samples and labels for a single time series.
message TimeSeries {
repeated Label labels = 1 [(gogoproto.nullable) = false];
repeated Sample samples = 2 [(gogoproto.nullable) = false];
}
message Label {
string name = 1;
string value = 2;
}
message Labels {
repeated Label labels = 1 [(gogoproto.nullable) = false];
}
// Matcher specifies a rule, which can match or set of labels or not.
message LabelMatcher {
enum Type {
EQ = 0;
NEQ = 1;
RE = 2;
NRE = 3;
}
Type type = 1;
string name = 2;
string value = 3;
}
message ReadHints {
int64 step_ms = 1; // Query step size in milliseconds.
string func = 2; // String representation of surrounding function or aggregation.
int64 start_ms = 3; // Start time in milliseconds.
int64 end_ms = 4; // End time in milliseconds.
repeated string grouping = 5; // List of label names used in aggregation.
bool by = 6; // Indicate whether it is without or by.
int64 range_ms = 7; // Range vector selector range in milliseconds.
}
// Chunk represents a TSDB chunk.
// Time range [min, max] is inclusive.
message Chunk {
int64 min_time_ms = 1;
int64 max_time_ms = 2;
// We require this to match chunkenc.Encoding.
enum Encoding {
UNKNOWN = 0;
XOR = 1;
}
Encoding type = 3;
bytes data = 4;
}
// ChunkedSeries represents single, encoded time series.
message ChunkedSeries {
// Labels should be sorted.
repeated Label labels = 1 [(gogoproto.nullable) = false];
// Chunks will be in start time order and may overlap.
repeated Chunk chunks = 2 [(gogoproto.nullable) = false];
}

30
lib/prompbmarshal/util.go Normal file
View File

@ -0,0 +1,30 @@
package prompbmarshal
import (
"fmt"
)
// MarshalWriteRequest marshals wr to dst and returns the result.
func MarshalWriteRequest(dst []byte, wr *WriteRequest) []byte {
size := wr.Size()
dstLen := len(dst)
if n := size - (cap(dst) - dstLen); n > 0 {
dst = append(dst[:cap(dst)], make([]byte, n)...)
}
dst = dst[:dstLen+size]
n, err := wr.MarshalToSizedBuffer(dst[dstLen:])
if err != nil {
panic(fmt.Errorf("BUG: unexpected error when marshaling WriteRequest: %s", err))
}
return dst[:dstLen+n]
}
// ResetWriteRequest resets wr.
func ResetWriteRequest(wr *WriteRequest) {
for i := range wr.Timeseries {
ts := wr.Timeseries[i]
ts.Labels = nil
ts.Samples = nil
}
wr.Timeseries = wr.Timeseries[:0]
}

129
lib/promrelabel/config.go Normal file
View File

@ -0,0 +1,129 @@
package promrelabel
import (
"fmt"
"io/ioutil"
"regexp"
"gopkg.in/yaml.v2"
)
// RelabelConfig represents relabel config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
type RelabelConfig struct {
SourceLabels []string `yaml:"source_labels"`
Separator *string `yaml:"separator"`
TargetLabel string `yaml:"target_label"`
Regex *string `yaml:"regex"`
Modulus uint64 `yaml:"modulus"`
Replacement *string `yaml:"replacement"`
Action string `yaml:"action"`
}
// LoadRelabelConfigs loads relabel configs from the given path.
func LoadRelabelConfigs(path string) ([]ParsedRelabelConfig, error) {
data, err := ioutil.ReadFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read `relabel_configs` from %q: %s", path, err)
}
var rcs []RelabelConfig
if err := yaml.Unmarshal(data, &rcs); err != nil {
return nil, fmt.Errorf("cannot unmarshal `relabel_configs` from %q: %s", path, err)
}
return ParseRelabelConfigs(nil, rcs)
}
// ParseRelabelConfigs parses rcs to dst.
func ParseRelabelConfigs(dst []ParsedRelabelConfig, rcs []RelabelConfig) ([]ParsedRelabelConfig, error) {
if len(rcs) == 0 {
return dst, nil
}
for i := range rcs {
var err error
dst, err = parseRelabelConfig(dst, &rcs[i])
if err != nil {
return dst, fmt.Errorf("error when parsing `relabel_config` #%d: %s", i+1, err)
}
}
return dst, nil
}
var defaultRegexForRelabelConfig = regexp.MustCompile("^(.*)$")
func parseRelabelConfig(dst []ParsedRelabelConfig, rc *RelabelConfig) ([]ParsedRelabelConfig, error) {
sourceLabels := rc.SourceLabels
separator := ";"
if rc.Separator != nil {
separator = *rc.Separator
}
targetLabel := rc.TargetLabel
regexCompiled := defaultRegexForRelabelConfig
if rc.Regex != nil {
regex := *rc.Regex
if rc.Action != "replace_all" && rc.Action != "labelmap_all" {
regex = "^" + *rc.Regex + "$"
}
re, err := regexp.Compile(regex)
if err != nil {
return dst, fmt.Errorf("cannot parse `regex` %q: %s", regex, err)
}
regexCompiled = re
}
modulus := rc.Modulus
replacement := "$1"
if rc.Replacement != nil {
replacement = *rc.Replacement
}
action := rc.Action
if action == "" {
action = "replace"
}
switch action {
case "replace":
if targetLabel == "" {
return dst, fmt.Errorf("missing `target_label` for `action=replace`")
}
case "replace_all":
if len(sourceLabels) == 0 {
return dst, fmt.Errorf("missing `source_labels` for `action=replace_all`")
}
if targetLabel == "" {
return dst, fmt.Errorf("missing `target_label` for `action=replace`")
}
case "keep":
if len(sourceLabels) == 0 {
return dst, fmt.Errorf("missing `source_labels` for `action=keep`")
}
case "drop":
if len(sourceLabels) == 0 {
return dst, fmt.Errorf("missing `source_labels` for `action=drop`")
}
case "hashmod":
if len(sourceLabels) == 0 {
return dst, fmt.Errorf("missing `source_labels` for `action=hashmod`")
}
if targetLabel == "" {
return dst, fmt.Errorf("missing `target_label` for `action=hashmod`")
}
if modulus < 1 {
return dst, fmt.Errorf("unexpected `modulus` for `action=hashmod`: %d; must be greater than 0", modulus)
}
case "labelmap":
case "labelmap_all":
case "labeldrop":
case "labelkeep":
default:
return dst, fmt.Errorf("unknown `action` %q", action)
}
dst = append(dst, ParsedRelabelConfig{
SourceLabels: sourceLabels,
Separator: separator,
TargetLabel: targetLabel,
Regex: regexCompiled,
Modulus: modulus,
Replacement: replacement,
Action: action,
})
return dst, nil
}

View File

@ -0,0 +1,163 @@
package promrelabel
import (
"reflect"
"testing"
)
func TestLoadRelabelConfigsSuccess(t *testing.T) {
path := "testdata/relabel_configs_valid.yml"
prcs, err := LoadRelabelConfigs(path)
if err != nil {
t.Fatalf("cannot load relabel configs from %q: %s", path, err)
}
if len(prcs) != 7 {
t.Fatalf("unexpected number of relabel configs loaded from %q; got %d; want %d", path, len(prcs), 7)
}
}
func TestLoadRelabelConfigsFailure(t *testing.T) {
f := func(path string) {
t.Helper()
rcs, err := LoadRelabelConfigs(path)
if err == nil {
t.Fatalf("expecting non-nil error")
}
if len(rcs) != 0 {
t.Fatalf("unexpected non-empty rcs: %#v", rcs)
}
}
t.Run("non-existing-file", func(t *testing.T) {
f("testdata/non-exsiting-file")
})
t.Run("invalid-file", func(t *testing.T) {
f("testdata/invalid_config.yml")
})
}
func TestParseRelabelConfigsSuccess(t *testing.T) {
f := func(rcs []RelabelConfig, prcsExpected []ParsedRelabelConfig) {
t.Helper()
prcs, err := ParseRelabelConfigs(nil, rcs)
if err != nil {
t.Fatalf("unexected error: %s", err)
}
if !reflect.DeepEqual(prcs, prcsExpected) {
t.Fatalf("unexpected prcs; got\n%#v\nwant\n%#v", prcs, prcsExpected)
}
}
f(nil, nil)
f([]RelabelConfig{
{
SourceLabels: []string{"foo", "bar"},
TargetLabel: "xxx",
},
}, []ParsedRelabelConfig{
{
SourceLabels: []string{"foo", "bar"},
Separator: ";",
TargetLabel: "xxx",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
Action: "replace",
},
})
}
func TestParseRelabelConfigsFailure(t *testing.T) {
f := func(rcs []RelabelConfig) {
t.Helper()
prcs, err := ParseRelabelConfigs(nil, rcs)
if err == nil {
t.Fatalf("expecting non-nil error")
}
if len(prcs) > 0 {
t.Fatalf("unexpected non-empty prcs: %#v", prcs)
}
}
t.Run("invalid-regex", func(t *testing.T) {
f([]RelabelConfig{
{
SourceLabels: []string{"aaa"},
TargetLabel: "xxx",
Regex: strPtr("foo[bar"),
},
})
})
t.Run("replace-missing-target-label", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "replace",
SourceLabels: []string{"foo"},
},
})
})
t.Run("replace_all-missing-source-labels", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "replace_all",
TargetLabel: "xxx",
},
})
})
t.Run("replace_all-missing-target-label", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "replace_all",
SourceLabels: []string{"foo"},
},
})
})
t.Run("keep-missing-source-labels", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "keep",
},
})
})
t.Run("drop-missing-source-labels", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "drop",
},
})
})
t.Run("hashmod-missing-source-labels", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "hashmod",
TargetLabel: "aaa",
Modulus: 123,
},
})
})
t.Run("hashmod-missing-target-label", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "hashmod",
SourceLabels: []string{"aaa"},
Modulus: 123,
},
})
})
t.Run("hashmod-missing-modulus", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "hashmod",
SourceLabels: []string{"aaa"},
TargetLabel: "xxx",
},
})
})
t.Run("invalid-action", func(t *testing.T) {
f([]RelabelConfig{
{
Action: "invalid-action",
},
})
})
}
func strPtr(s string) *string {
return &s
}

268
lib/promrelabel/relabel.go Normal file
View File

@ -0,0 +1,268 @@
package promrelabel
import (
"regexp"
"strconv"
"strings"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
xxhash "github.com/cespare/xxhash/v2"
)
// ParsedRelabelConfig contains parsed `relabel_config`.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
type ParsedRelabelConfig struct {
SourceLabels []string
Separator string
TargetLabel string
Regex *regexp.Regexp
Modulus uint64
Replacement string
Action string
}
// ApplyRelabelConfigs applies prcs to labels starting from the labelsOffset.
//
// If isFinalize is set, then FinalizeLabels is called on the labels[labelsOffset:].
//
// The returned labels at labels[labelsOffset:] are sorted.
func ApplyRelabelConfigs(labels []prompbmarshal.Label, labelsOffset int, prcs []ParsedRelabelConfig, isFinalize bool) []prompbmarshal.Label {
for i := range prcs {
tmp := applyRelabelConfig(labels, labelsOffset, &prcs[i])
if len(tmp) == labelsOffset {
// All the labels have been removed.
return tmp
}
labels = tmp
}
labels = removeEmptyLabels(labels, labelsOffset)
if isFinalize {
labels = FinalizeLabels(labels[:labelsOffset], labels[labelsOffset:])
}
SortLabels(labels[labelsOffset:])
return labels
}
func removeEmptyLabels(labels []prompbmarshal.Label, labelsOffset int) []prompbmarshal.Label {
src := labels[labelsOffset:]
needsRemoval := false
for i := range src {
label := &src[i]
if label.Name == "" || label.Value == "" {
needsRemoval = true
break
}
}
if !needsRemoval {
return labels
}
dst := labels[:labelsOffset]
for i := range src {
label := &src[i]
if label.Name != "" && label.Value != "" {
dst = append(dst, *label)
}
}
return dst
}
// FinalizeLabels finalizes labels according to relabel_config rules.
//
// It renames `__address__` to `instance` and removes labels with "__" in the beginning.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
func FinalizeLabels(dst, src []prompbmarshal.Label) []prompbmarshal.Label {
for i := range src {
label := &src[i]
name := label.Name
if !strings.HasPrefix(name, "__") || name == "__name__" {
dst = append(dst, *label)
continue
}
if name == "__address__" {
if GetLabelByName(src, "instance") != nil {
// The `instance` label is already set. Skip `__address__` label.
continue
}
// Rename `__address__` label to `instance`.
labelCopy := *label
labelCopy.Name = "instance"
dst = append(dst, labelCopy)
}
}
return dst
}
// applyRelabelConfig applies relabeling according to cfg.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
func applyRelabelConfig(labels []prompbmarshal.Label, labelsOffset int, cfg *ParsedRelabelConfig) []prompbmarshal.Label {
src := labels[labelsOffset:]
switch cfg.Action {
case "replace":
bb := relabelBufPool.Get()
bb.B = concatLabelValues(bb.B[:0], src, cfg.SourceLabels, cfg.Separator)
if len(bb.B) == 0 && cfg.Regex == defaultRegexForRelabelConfig && !strings.Contains(cfg.Replacement, "$") {
// Fast path for the following rule that just sets label value:
// - target_label: foobar
// replacement: something-here
relabelBufPool.Put(bb)
return setLabelValue(labels, labelsOffset, cfg.TargetLabel, cfg.Replacement)
}
match := cfg.Regex.FindSubmatchIndex(bb.B)
if match == nil {
// Fast path - nothing to replace.
relabelBufPool.Put(bb)
return labels
}
sourceStr := bytesutil.ToUnsafeString(bb.B)
value := relabelBufPool.Get()
value.B = cfg.Regex.ExpandString(value.B[:0], cfg.Replacement, sourceStr, match)
relabelBufPool.Put(bb)
valueStr := string(value.B)
relabelBufPool.Put(value)
return setLabelValue(labels, labelsOffset, cfg.TargetLabel, valueStr)
case "replace_all":
bb := relabelBufPool.Get()
bb.B = concatLabelValues(bb.B[:0], src, cfg.SourceLabels, cfg.Separator)
if !cfg.Regex.Match(bb.B) {
// Fast path - nothing to replace.
relabelBufPool.Put(bb)
return labels
}
sourceStr := string(bb.B) // Make a copy of bb, since it can be returned from ReplaceAllString
relabelBufPool.Put(bb)
valueStr := cfg.Regex.ReplaceAllString(sourceStr, cfg.Replacement)
return setLabelValue(labels, labelsOffset, cfg.TargetLabel, valueStr)
case "keep":
bb := relabelBufPool.Get()
bb.B = concatLabelValues(bb.B[:0], src, cfg.SourceLabels, cfg.Separator)
keep := cfg.Regex.Match(bb.B)
relabelBufPool.Put(bb)
if !keep {
return labels[:labelsOffset]
}
return labels
case "drop":
bb := relabelBufPool.Get()
bb.B = concatLabelValues(bb.B[:0], src, cfg.SourceLabels, cfg.Separator)
drop := cfg.Regex.Match(bb.B)
relabelBufPool.Put(bb)
if drop {
return labels[:labelsOffset]
}
return labels
case "hashmod":
bb := relabelBufPool.Get()
bb.B = concatLabelValues(bb.B[:0], src, cfg.SourceLabels, cfg.Separator)
h := xxhash.Sum64(bb.B) % cfg.Modulus
value := strconv.Itoa(int(h))
relabelBufPool.Put(bb)
return setLabelValue(labels, labelsOffset, cfg.TargetLabel, value)
case "labelmap":
for i := range src {
label := &src[i]
match := cfg.Regex.FindStringSubmatchIndex(label.Name)
if match == nil {
continue
}
value := relabelBufPool.Get()
value.B = cfg.Regex.ExpandString(value.B[:0], cfg.Replacement, label.Name, match)
label.Name = string(value.B)
relabelBufPool.Put(value)
}
return labels
case "labelmap_all":
for i := range src {
label := &src[i]
if !cfg.Regex.MatchString(label.Name) {
continue
}
label.Name = cfg.Regex.ReplaceAllString(label.Name, cfg.Replacement)
}
return labels
case "labeldrop":
keepSrc := true
for i := range src {
if cfg.Regex.MatchString(src[i].Name) {
keepSrc = false
break
}
}
if keepSrc {
return labels
}
dst := labels[:labelsOffset]
for i := range src {
label := &src[i]
if !cfg.Regex.MatchString(label.Name) {
dst = append(dst, *label)
}
}
return dst
case "labelkeep":
keepSrc := true
for i := range src {
if !cfg.Regex.MatchString(src[i].Name) {
keepSrc = false
break
}
}
if keepSrc {
return labels
}
dst := labels[:labelsOffset]
for i := range src {
label := &src[i]
if cfg.Regex.MatchString(label.Name) {
dst = append(dst, *label)
}
}
return dst
default:
logger.Panicf("BUG: unknown `action`: %q", cfg.Action)
return labels
}
}
var relabelBufPool bytesutil.ByteBufferPool
func concatLabelValues(dst []byte, labels []prompbmarshal.Label, labelNames []string, separator string) []byte {
if len(labelNames) == 0 {
return dst
}
for _, labelName := range labelNames {
label := GetLabelByName(labels, labelName)
if label != nil {
dst = append(dst, label.Value...)
}
dst = append(dst, separator...)
}
return dst[:len(dst)-len(separator)]
}
func setLabelValue(labels []prompbmarshal.Label, labelsOffset int, name, value string) []prompbmarshal.Label {
if label := GetLabelByName(labels[labelsOffset:], name); label != nil {
label.Value = value
return labels
}
labels = append(labels, prompbmarshal.Label{
Name: name,
Value: value,
})
return labels
}
// GetLabelByName returns label with the given name from labels.
func GetLabelByName(labels []prompbmarshal.Label, name string) *prompbmarshal.Label {
for i := range labels {
label := &labels[i]
if label.Name == name {
return label
}
}
return nil
}

View File

@ -0,0 +1,630 @@
package promrelabel
import (
"reflect"
"regexp"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
func TestApplyRelabelConfigs(t *testing.T) {
f := func(prcs []ParsedRelabelConfig, labels []prompbmarshal.Label, isFinalize bool, resultExpected []prompbmarshal.Label) {
t.Helper()
result := ApplyRelabelConfigs(labels, 0, prcs, isFinalize)
if !reflect.DeepEqual(result, resultExpected) {
t.Fatalf("unexpected result; got\n%v\nwant\n%v", result, resultExpected)
}
}
t.Run("empty_replabel_configs", func(t *testing.T) {
f(nil, nil, false, nil)
f(nil, nil, true, nil)
f(nil, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
}, false, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
})
f(nil, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "__name__",
Value: "xxx",
},
{
Name: "__aaa",
Value: "yyy",
},
}, true, []prompbmarshal.Label{
{
Name: "__name__",
Value: "xxx",
},
{
Name: "foo",
Value: "bar",
},
})
})
t.Run("replace-miss", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace",
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, nil, false, []prompbmarshal.Label{})
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, nil, false, []prompbmarshal.Label{})
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
})
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: regexp.MustCompile(".+"),
Replacement: "$1",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("replace-hit", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"xxx", "foo"},
Separator: ";",
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "a-$1-b",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "bar",
Value: "a-yyy;-b",
},
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("replace_all-miss", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace_all",
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, nil, false, []prompbmarshal.Label{})
f([]ParsedRelabelConfig{
{
Action: "replace_all",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, nil, false, []prompbmarshal.Label{})
f([]ParsedRelabelConfig{
{
Action: "replace_all",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
})
f([]ParsedRelabelConfig{
{
Action: "replace_all",
SourceLabels: []string{"foo"},
TargetLabel: "bar",
Regex: regexp.MustCompile(".+"),
Replacement: "$1",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("replace_all-hit", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace_all",
SourceLabels: []string{"xxx", "foo"},
Separator: ";",
TargetLabel: "xxx",
Regex: regexp.MustCompile("(;)"),
Replacement: "-$1-",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "y;y",
},
}, false, []prompbmarshal.Label{
{
Name: "xxx",
Value: "y-;-y-;-",
},
})
})
t.Run("replace-add-multi-labels", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"xxx"},
TargetLabel: "bar",
Regex: defaultRegexForRelabelConfig,
Replacement: "a-$1",
},
{
Action: "replace",
SourceLabels: []string{"bar"},
TargetLabel: "zar",
Regex: defaultRegexForRelabelConfig,
Replacement: "b-$1",
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
{
Name: "__address__",
Value: "a.bc",
},
}, true, []prompbmarshal.Label{
{
Name: "bar",
Value: "a-yyy",
},
{
Name: "instance",
Value: "a.bc",
},
{
Name: "xxx",
Value: "yyy",
},
{
Name: "zar",
Value: "b-a-yyy",
},
})
})
t.Run("replace-self", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"foo"},
TargetLabel: "foo",
Regex: defaultRegexForRelabelConfig,
Replacement: "a-$1",
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "aaxx",
},
}, true, []prompbmarshal.Label{
{
Name: "foo",
Value: "a-aaxx",
},
})
})
t.Run("replace-missing-source", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "replace",
TargetLabel: "foo",
Regex: defaultRegexForRelabelConfig,
Replacement: "foobar",
},
}, []prompbmarshal.Label{}, true, []prompbmarshal.Label{
{
Name: "foo",
Value: "foobar",
},
})
})
t.Run("keep-miss", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "keep",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, nil, true, nil)
f([]ParsedRelabelConfig{
{
Action: "keep",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, true, []prompbmarshal.Label{})
})
t.Run("keep-hit", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "keep",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "foo",
Value: "yyy",
},
})
})
t.Run("drop-miss", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, nil, false, nil)
f([]ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, true, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("drop-hit", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"foo"},
Regex: regexp.MustCompile(".+"),
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "yyy",
},
}, true, []prompbmarshal.Label{})
})
t.Run("hashmod-miss", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "hashmod",
SourceLabels: []string{"foo"},
TargetLabel: "aaa",
Modulus: 123,
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
}, false, []prompbmarshal.Label{
{
Name: "aaa",
Value: "81",
},
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("hashmod-hit", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "hashmod",
SourceLabels: []string{"foo"},
TargetLabel: "aaa",
Modulus: 123,
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "yyy",
},
}, true, []prompbmarshal.Label{
{
Name: "aaa",
Value: "73",
},
{
Name: "foo",
Value: "yyy",
},
})
})
t.Run("labelmap", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "labelmap",
Regex: regexp.MustCompile("foo(.+)"),
Replacement: "$1-x",
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "yyy",
},
{
Name: "foobar",
Value: "aaa",
},
}, true, []prompbmarshal.Label{
{
Name: "bar-x",
Value: "aaa",
},
{
Name: "foo",
Value: "yyy",
},
})
})
t.Run("labelmap_all", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "labelmap_all",
Regex: regexp.MustCompile(`\.`),
Replacement: "-",
},
}, []prompbmarshal.Label{
{
Name: "foo.bar.baz",
Value: "yyy",
},
{
Name: "foobar",
Value: "aaa",
},
}, true, []prompbmarshal.Label{
{
Name: "foo-bar-baz",
Value: "yyy",
},
{
Name: "foobar",
Value: "aaa",
},
})
})
t.Run("labeldrop", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "labeldrop",
Regex: regexp.MustCompile("dropme.*"),
},
}, []prompbmarshal.Label{
{
Name: "aaa",
Value: "bbb",
},
}, true, []prompbmarshal.Label{
{
Name: "aaa",
Value: "bbb",
},
})
f([]ParsedRelabelConfig{
{
Action: "labeldrop",
Regex: regexp.MustCompile("dropme.*"),
},
}, []prompbmarshal.Label{
{
Name: "xxx",
Value: "yyy",
},
{
Name: "dropme-please",
Value: "aaa",
},
{
Name: "foo",
Value: "bar",
},
}, false, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "xxx",
Value: "yyy",
},
})
})
t.Run("labelkeep", func(t *testing.T) {
f([]ParsedRelabelConfig{
{
Action: "labelkeep",
Regex: regexp.MustCompile("keepme.*"),
},
}, []prompbmarshal.Label{
{
Name: "keepme",
Value: "aaa",
},
}, true, []prompbmarshal.Label{
{
Name: "keepme",
Value: "aaa",
},
})
f([]ParsedRelabelConfig{
{
Action: "labelkeep",
Regex: regexp.MustCompile("keepme.*"),
},
}, []prompbmarshal.Label{
{
Name: "keepme",
Value: "aaa",
},
{
Name: "aaaa",
Value: "awef",
},
{
Name: "keepme-aaa",
Value: "234",
},
}, false, []prompbmarshal.Label{
{
Name: "keepme",
Value: "aaa",
},
{
Name: "keepme-aaa",
Value: "234",
},
})
})
}
func TestFinalizeLabels(t *testing.T) {
f := func(labels, resultExpected []prompbmarshal.Label) {
t.Helper()
result := FinalizeLabels(nil, labels)
if !reflect.DeepEqual(result, resultExpected) {
t.Fatalf("unexpected result; got\n%v\nwant\n%v", result, resultExpected)
}
}
f(nil, nil)
f([]prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "__aaa",
Value: "ass",
},
{
Name: "__address__",
Value: "foo.com",
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "instance",
Value: "foo.com",
},
})
f([]prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "instance",
Value: "ass",
},
{
Name: "__address__",
Value: "foo.com",
},
}, []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "instance",
Value: "ass",
},
})
}

View File

@ -0,0 +1,567 @@
package promrelabel
import (
"fmt"
"regexp"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
func BenchmarkApplyRelabelConfigs(b *testing.B) {
b.Run("replace-label-copy", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"id"},
TargetLabel: "__name__",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "foobar-random-string-here"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("replace-set-label", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "replace",
TargetLabel: "__name__",
Regex: defaultRegexForRelabelConfig,
Replacement: "foobar",
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "foobar" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "foobar"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("replace-add-label", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "replace",
TargetLabel: "aaa",
Regex: defaultRegexForRelabelConfig,
Replacement: "foobar",
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 2 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 2, labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "aaa" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "aaa"))
}
if labels[1].Value != "foobar" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar"))
}
}
})
})
b.Run("replace-mismatch", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"non-existing-label"},
TargetLabel: "id",
Regex: regexp.MustCompile("(foobar)-.*"),
Replacement: "$1",
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("replace-match", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "replace",
SourceLabels: []string{"id"},
TargetLabel: "id",
Regex: regexp.MustCompile("(foobar)-.*"),
Replacement: "$1",
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar"))
}
}
})
})
b.Run("drop-mismatch", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"non-existing-label"},
Regex: regexp.MustCompile("(foobar)-.*"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("drop-match", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"id"},
Regex: regexp.MustCompile("(foobar)-.*"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 0 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 0, labels))
}
}
})
})
b.Run("keep-mismatch", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "keep",
SourceLabels: []string{"non-existing-label"},
Regex: regexp.MustCompile("(foobar)-.*"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 0 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 0, labels))
}
}
})
})
b.Run("keep-match", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "keep",
SourceLabels: []string{"id"},
Regex: regexp.MustCompile("(foobar)-.*"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("labeldrop-mismatch", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "labeldrop",
Regex: regexp.MustCompile("non-existing-label"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("labeldrop-match", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "labeldrop",
Regex: regexp.MustCompile("id"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 1 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 1, labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
}
})
})
b.Run("labelkeep-mismatch", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "labelkeep",
Regex: regexp.MustCompile("non-existing-label"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 0 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 0, labels))
}
}
})
})
b.Run("labelkeep-match", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "labelkeep",
Regex: regexp.MustCompile("id"),
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != 1 {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), 1, labels))
}
if labels[0].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "id"))
}
if labels[0].Value != "foobar-random-string-here" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "foobar-random-string-here"))
}
}
})
})
b.Run("hashmod", func(b *testing.B) {
prcs := []ParsedRelabelConfig{
{
Action: "hashmod",
SourceLabels: []string{"id"},
TargetLabel: "id",
Modulus: 23,
},
}
labelsOrig := []prompbmarshal.Label{
{
Name: "__name__",
Value: "metric",
},
{
Name: "id",
Value: "foobar-random-string-here",
},
}
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
var labels []prompbmarshal.Label
for pb.Next() {
labels = append(labels[:0], labelsOrig...)
labels = ApplyRelabelConfigs(labels, 0, prcs, true)
if len(labels) != len(labelsOrig) {
panic(fmt.Errorf("unexpected number of labels; got %d; want %d; labels:\n%#v", len(labels), len(labelsOrig), labels))
}
if labels[0].Name != "__name__" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[0].Name, "__name__"))
}
if labels[0].Value != "metric" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[0].Value, "metric"))
}
if labels[1].Name != "id" {
panic(fmt.Errorf("unexpected label name; got %q; want %q", labels[1].Name, "id"))
}
if labels[1].Value != "11" {
panic(fmt.Errorf("unexpected label value; got %q; want %q", labels[1].Value, "11"))
}
}
})
})
}

37
lib/promrelabel/sort.go Normal file
View File

@ -0,0 +1,37 @@
package promrelabel
import (
"sort"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
// SortLabels sorts labels.
func SortLabels(labels []prompbmarshal.Label) {
ls := labelsSorterPool.Get().(*labelsSorter)
*ls = labels
if !sort.IsSorted(ls) {
sort.Sort(ls)
}
*ls = nil
labelsSorterPool.Put(ls)
}
var labelsSorterPool = &sync.Pool{
New: func() interface{} {
return &labelsSorter{}
},
}
type labelsSorter []prompbmarshal.Label
func (ls *labelsSorter) Len() int { return len(*ls) }
func (ls *labelsSorter) Swap(i, j int) {
a := *ls
a[i], a[j] = a[j], a[i]
}
func (ls *labelsSorter) Less(i, j int) bool {
a := *ls
return a[i].Name < a[j].Name
}

View File

@ -0,0 +1,43 @@
package promrelabel
import (
"reflect"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
func TestSortLabels(t *testing.T) {
labels := []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "aa",
Value: "bb",
},
{
Name: "ba",
Value: "zz",
},
}
labelsExpected := []prompbmarshal.Label{
{
Name: "aa",
Value: "bb",
},
{
Name: "ba",
Value: "zz",
},
{
Name: "foo",
Value: "bar",
},
}
SortLabels(labels)
if !reflect.DeepEqual(labels, labelsExpected) {
t.Fatalf("unexpected sorted labels; got\n%v\nwant\n%v", labels, labelsExpected)
}
}

View File

@ -0,0 +1,3 @@
fodsofdsf
f dsfdds
fdsfsdsfd

View File

@ -0,0 +1,20 @@
- target_label: bar
- source_labels: [aa]
separator: "foobar"
regex: "foo.+bar"
target_label: aaa
replacement: "xxx"
- action: keep
source_labels: [aaa]
- action: drop
source_labels: [aaa]
- action: hashmod
source_labels: [aaa]
target_label: aaa
modulus: 234
- action: replace_all
source_labels: [aa]
target_label: bb
- action: labelmap_all
regex: "\\."
replacement: ":"

134
lib/promscrape/client.go Normal file
View File

@ -0,0 +1,134 @@
package promscrape
import (
"crypto/tls"
"flag"
"fmt"
"strings"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/netutil"
"github.com/VictoriaMetrics/metrics"
"github.com/valyala/fasthttp"
)
var (
maxScrapeSize = flag.Int("promscrape.maxScrapeSize", 16*1024*1024, "The maximum size of scrape response in bytes to process from Prometheus targets. "+
"Bigger responses are rejected")
disableCompression = flag.Bool("promscrape.disableCompression", false, "Whether to disable sending 'Accept-Encoding: gzip' request headers to scrape targets. "+
"This may reduce CPU usage on scrape targets at the cost of higher network bandwidth utilization")
)
type client struct {
hc *fasthttp.HostClient
scrapeURL string
host string
requestURI string
authHeader string
}
func newClient(sw *ScrapeWork) *client {
var u fasthttp.URI
u.Update(sw.ScrapeURL)
host := string(u.Host())
requestURI := string(u.RequestURI())
isTLS := string(u.Scheme()) == "https"
var tlsCfg *tls.Config
if isTLS {
tlsCfg = getTLSConfig(sw)
}
if !strings.Contains(host, ":") {
if !isTLS {
host += ":80"
} else {
host += ":443"
}
}
hc := &fasthttp.HostClient{
Addr: host,
Name: "vm_promscrape",
Dial: statDial,
DialDualStack: netutil.TCP6Enabled(),
IsTLS: isTLS,
TLSConfig: tlsCfg,
MaxIdleConnDuration: 2 * sw.ScrapeInterval,
ReadTimeout: sw.ScrapeTimeout,
WriteTimeout: 10 * time.Second,
MaxResponseBodySize: *maxScrapeSize,
}
return &client{
hc: hc,
scrapeURL: sw.ScrapeURL,
host: host,
requestURI: requestURI,
authHeader: sw.Authorization,
}
}
func (c *client) ReadData(dst []byte) ([]byte, error) {
req := fasthttp.AcquireRequest()
req.SetRequestURI(c.requestURI)
req.SetHost(c.host)
if !*disableCompression {
req.Header.Set("Accept-Encoding", "gzip")
}
if c.authHeader != "" {
req.Header.Set("Authorization", c.authHeader)
}
resp := fasthttp.AcquireResponse()
// There is no need in calling DoTimeout, since the timeout is already set in c.hc.ReadTimeout.
err := c.hc.Do(req, resp)
fasthttp.ReleaseRequest(req)
if err != nil {
fasthttp.ReleaseResponse(resp)
if err == fasthttp.ErrTimeout {
scrapesTimedout.Inc()
return dst, fmt.Errorf("error when scraping %q with timeout %s: %s", c.scrapeURL, c.hc.ReadTimeout, err)
}
return dst, fmt.Errorf("error when scraping %q: %s", c.scrapeURL, err)
}
dstLen := len(dst)
if ce := resp.Header.Peek("Content-Encoding"); string(ce) == "gzip" {
var err error
dst, err = fasthttp.AppendGunzipBytes(dst, resp.Body())
if err != nil {
fasthttp.ReleaseResponse(resp)
scrapesGunzipFailed.Inc()
return dst, fmt.Errorf("cannot ungzip response from %q: %s", c.scrapeURL, err)
}
scrapesGunzipped.Inc()
} else {
dst = append(dst, resp.Body()...)
}
statusCode := resp.StatusCode()
if statusCode != fasthttp.StatusOK {
metrics.GetOrCreateCounter(fmt.Sprintf(`vm_promscrape_scrapes_total{status_code="%d"}`, statusCode)).Inc()
return dst, fmt.Errorf("unexpected status code returned when scraping %q: %d; expecting %d; response body: %q",
c.scrapeURL, statusCode, fasthttp.StatusOK, dst[dstLen:])
}
scrapesOK.Inc()
fasthttp.ReleaseResponse(resp)
return dst, nil
}
var (
scrapesTimedout = metrics.NewCounter(`vm_promscrape_scrapes_timed_out_total`)
scrapesOK = metrics.NewCounter(`vm_promscrape_scrapes_total{status_code="200"}`)
scrapesGunzipped = metrics.NewCounter(`vm_promscrape_scrapes_gunziped_total`)
scrapesGunzipFailed = metrics.NewCounter(`vm_promscrape_scrapes_gunzip_failed_total`)
)
func getTLSConfig(sw *ScrapeWork) *tls.Config {
tlsCfg := &tls.Config{
RootCAs: sw.TLSRootCA,
ClientSessionCache: tls.NewLRUClientSessionCache(0),
}
if sw.TLSCertificate != nil {
tlsCfg.Certificates = []tls.Certificate{*sw.TLSCertificate}
}
tlsCfg.ServerName = sw.TLSServerName
tlsCfg.InsecureSkipVerify = sw.TLSInsecureSkipVerify
return tlsCfg
}

588
lib/promscrape/config.go Normal file
View File

@ -0,0 +1,588 @@
package promscrape
import (
"crypto/tls"
"crypto/x509"
"encoding/base64"
"fmt"
"io/ioutil"
"net/url"
"path/filepath"
"strings"
"time"
"unicode"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"gopkg.in/yaml.v2"
)
// Config represents essential parts from Prometheus config defined at https://prometheus.io/docs/prometheus/latest/configuration/configuration/
type Config struct {
Global GlobalConfig `yaml:"global"`
ScrapeConfigs []ScrapeConfig `yaml:"scrape_configs"`
// This is set to the directory from where the config has been loaded.
baseDir string
}
// GlobalConfig represents essential parts for `global` section of Prometheus config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/
type GlobalConfig struct {
ScrapeInterval time.Duration `yaml:"scrape_interval"`
ScrapeTimeout time.Duration `yaml:"scrape_timeout"`
ExternalLabels map[string]string `yaml:"external_labels"`
}
// ScrapeConfig represents essential parts for `scrape_config` section of Prometheus config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config
type ScrapeConfig struct {
JobName string `yaml:"job_name"`
ScrapeInterval time.Duration `yaml:"scrape_interval"`
ScrapeTimeout time.Duration `yaml:"scrape_timeout"`
MetricsPath string `yaml:"metrics_path"`
HonorLabels bool `yaml:"honor_labels"`
HonorTimestamps bool `yaml:"honor_timestamps"`
Scheme string `yaml:"scheme"`
Params map[string][]string `yaml:"params"`
BasicAuth *BasicAuthConfig `yaml:"basic_auth"`
BearerToken string `yaml:"bearer_token"`
BearerTokenFile string `yaml:"bearer_token_file"`
TLSConfig *TLSConfig `yaml:"tls_config"`
StaticConfigs []StaticConfig `yaml:"static_configs"`
FileSDConfigs []FileSDConfig `yaml:"file_sd_configs"`
RelabelConfigs []promrelabel.RelabelConfig `yaml:"relabel_configs"`
MetricRelabelConfigs []promrelabel.RelabelConfig `yaml:"metric_relabel_configs"`
ScrapeLimit int `yaml:"scrape_limit"`
// This is set in loadConfig
swc *scrapeWorkConfig
}
// FileSDConfig represents file-based service discovery config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#file_sd_config
type FileSDConfig struct {
Files []string `yaml:"files"`
// `refresh_interval` is ignored. See `-prometheus.fileSDCheckInterval`
}
// TLSConfig represents TLS config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#tls_config
type TLSConfig struct {
CAFile string `yaml:"ca_file"`
CertFile string `yaml:"cert_file"`
KeyFile string `yaml:"key_file"`
ServerName string `yaml:"server_name"`
InsecureSkipVerify bool `yaml:"insecure_skip_verify"`
}
// BasicAuthConfig represents basic auth config.
type BasicAuthConfig struct {
Username string `yaml:"username"`
Password string `yaml:"password"`
PasswordFile string `yaml:"password_file"`
}
// StaticConfig represents essential parts for `static_config` section of Prometheus config.
//
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#static_config
type StaticConfig struct {
Targets []string `yaml:"targets"`
Labels map[string]string `yaml:"labels"`
}
func loadStaticConfigs(path string) ([]StaticConfig, error) {
data, err := ioutil.ReadFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read `static_configs` from %q: %s", path, err)
}
var stcs []StaticConfig
if err := yaml.Unmarshal(data, &stcs); err != nil {
return nil, fmt.Errorf("cannot unmarshal `static_configs` from %q: %s", path, err)
}
return stcs, nil
}
// loadConfig loads Prometheus config from the given path.
func loadConfig(path string) (cfg *Config, err error) {
data, err := ioutil.ReadFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read Prometheus config from %q: %s", path, err)
}
var cfgObj Config
if err := cfgObj.parse(data, path); err != nil {
return nil, fmt.Errorf("cannot parse Prometheus config from %q: %s", path, err)
}
return &cfgObj, nil
}
func (cfg *Config) parse(data []byte, path string) error {
if err := yaml.Unmarshal(data, cfg); err != nil {
return fmt.Errorf("cannot unmarshal data: %s", err)
}
absPath, err := filepath.Abs(path)
if err != nil {
return fmt.Errorf("cannot obtain abs path for %q: %s", path, err)
}
cfg.baseDir = filepath.Dir(absPath)
for i := range cfg.ScrapeConfigs {
sc := &cfg.ScrapeConfigs[i]
swc, err := getScrapeWorkConfig(sc, cfg.baseDir, &cfg.Global)
if err != nil {
return fmt.Errorf("cannot parse `scrape_config` #%d: %s", i+1, err)
}
sc.swc = swc
}
return nil
}
func (cfg *Config) fileSDConfigsCount() int {
n := 0
for i := range cfg.ScrapeConfigs {
n += len(cfg.ScrapeConfigs[i].FileSDConfigs)
}
return n
}
// getFileSDScrapeWork returns `file_sd_configs` ScrapeWork from cfg.
func (cfg *Config) getFileSDScrapeWork(prev []ScrapeWork) ([]ScrapeWork, error) {
var sws []ScrapeWork
for i := range cfg.ScrapeConfigs {
var err error
sws, err = cfg.ScrapeConfigs[i].appendFileSDScrapeWork(sws, prev, cfg.baseDir)
if err != nil {
return nil, fmt.Errorf("error when parsing `scrape_config` #%d: %s", i+1, err)
}
}
return sws, nil
}
// getStaticScrapeWork returns `static_configs` ScrapeWork from from cfg.
func (cfg *Config) getStaticScrapeWork() ([]ScrapeWork, error) {
var sws []ScrapeWork
for i := range cfg.ScrapeConfigs {
var err error
sws, err = cfg.ScrapeConfigs[i].appendStaticScrapeWork(sws)
if err != nil {
return nil, fmt.Errorf("error when parsing `scrape_config` #%d: %s", i+1, err)
}
}
return sws, nil
}
func (sc *ScrapeConfig) appendFileSDScrapeWork(dst, prev []ScrapeWork, baseDir string) ([]ScrapeWork, error) {
if len(sc.FileSDConfigs) == 0 {
// Fast path - no `file_sd_configs`
return dst, nil
}
// Create a map for the previous scrape work.
swPrev := make(map[string][]ScrapeWork)
for i := range prev {
sw := &prev[i]
label := promrelabel.GetLabelByName(sw.Labels, "__meta_filepath")
if label == nil {
logger.Panicf("BUG: missing `__meta_filepath` label")
}
swPrev[label.Value] = append(swPrev[label.Value], *sw)
}
for i := range sc.FileSDConfigs {
var err error
dst, err = sc.FileSDConfigs[i].appendScrapeWork(dst, swPrev, baseDir, sc.swc)
if err != nil {
return nil, fmt.Errorf("error when parsing `file_sd_config` #%d: %s", i+1, err)
}
}
return dst, nil
}
func (sc *ScrapeConfig) appendStaticScrapeWork(dst []ScrapeWork) ([]ScrapeWork, error) {
for i := range sc.StaticConfigs {
var err error
dst, err = sc.StaticConfigs[i].appendScrapeWork(dst, sc.swc)
if err != nil {
return nil, fmt.Errorf("error when parsing `static_config` #%d: %s", i+1, err)
}
}
return dst, nil
}
func getScrapeWorkConfig(sc *ScrapeConfig, baseDir string, globalCfg *GlobalConfig) (*scrapeWorkConfig, error) {
jobName := sc.JobName
if jobName == "" {
return nil, fmt.Errorf("missing `job_name` field in `scrape_config`")
}
scrapeInterval := sc.ScrapeInterval
if scrapeInterval <= 0 {
scrapeInterval = globalCfg.ScrapeInterval
if scrapeInterval <= 0 {
scrapeInterval = defaultScrapeInterval
}
}
scrapeTimeout := sc.ScrapeTimeout
if scrapeTimeout <= 0 {
scrapeTimeout = globalCfg.ScrapeTimeout
if scrapeTimeout <= 0 {
scrapeTimeout = defaultScrapeTimeout
}
}
honorLabels := sc.HonorLabels
honorTimestamps := sc.HonorTimestamps
metricsPath := sc.MetricsPath
if metricsPath == "" {
metricsPath = "/metrics"
}
scheme := sc.Scheme
if scheme == "" {
scheme = "http"
}
if scheme != "http" && scheme != "https" {
return nil, fmt.Errorf("unexpected `scheme` for `job_name` %q: %q; supported values: http or https", jobName, scheme)
}
params := sc.Params
var authorization string
if sc.BasicAuth != nil {
if sc.BasicAuth.Username == "" {
return nil, fmt.Errorf("missing `username` in `basic_auth` section for `job_name` %q", jobName)
}
username := sc.BasicAuth.Username
password := sc.BasicAuth.Password
if sc.BasicAuth.PasswordFile != "" {
if sc.BasicAuth.Password != "" {
return nil, fmt.Errorf("both `password`=%q and `password_file`=%q are set in `basic_auth` section for `job_name` %q",
sc.BasicAuth.Password, sc.BasicAuth.PasswordFile, jobName)
}
path := getFilepath(baseDir, sc.BasicAuth.PasswordFile)
pass, err := readPasswordFromFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read password from `password_file`=%q set in `basic_auth` section for `job_name` %q: %s",
sc.BasicAuth.PasswordFile, jobName, err)
}
password = pass
}
// See https://en.wikipedia.org/wiki/Basic_access_authentication
token := username + ":" + password
token64 := base64.StdEncoding.EncodeToString([]byte(token))
authorization = "Basic " + token64
}
bearerToken := sc.BearerToken
if sc.BearerTokenFile != "" {
if sc.BearerToken != "" {
return nil, fmt.Errorf("both `bearer_token`=%q and `bearer_token_file`=%q are set for `job_name` %q", sc.BearerToken, sc.BearerTokenFile, jobName)
}
path := getFilepath(baseDir, sc.BearerTokenFile)
token, err := readPasswordFromFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read bearer token from `bearer_token_file`=%q for `job_name` %q: %s", sc.BearerTokenFile, jobName, err)
}
bearerToken = token
}
if bearerToken != "" {
if authorization != "" {
return nil, fmt.Errorf("cannot use both `basic_auth` and `bearer_token` for `job_name` %q", jobName)
}
authorization = "Bearer " + bearerToken
}
var tlsRootCA *x509.CertPool
var tlsCertificate *tls.Certificate
tlsServerName := ""
tlsInsecureSkipVerify := false
if sc.TLSConfig != nil {
tlsServerName = sc.TLSConfig.ServerName
tlsInsecureSkipVerify = sc.TLSConfig.InsecureSkipVerify
if sc.TLSConfig.CertFile != "" || sc.TLSConfig.KeyFile != "" {
certPath := getFilepath(baseDir, sc.TLSConfig.CertFile)
keyPath := getFilepath(baseDir, sc.TLSConfig.KeyFile)
cert, err := tls.LoadX509KeyPair(certPath, keyPath)
if err != nil {
return nil, fmt.Errorf("cannot load TLS certificate for `job_name` %q from `cert_file`=%q, `key_file`=%q: %s",
jobName, sc.TLSConfig.CertFile, sc.TLSConfig.KeyFile, err)
}
tlsCertificate = &cert
}
if sc.TLSConfig.CAFile != "" {
path := getFilepath(baseDir, sc.TLSConfig.CAFile)
data, err := ioutil.ReadFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read `ca_file` %q for `job_name` %q: %s", sc.TLSConfig.CAFile, jobName, err)
}
tlsRootCA = x509.NewCertPool()
if !tlsRootCA.AppendCertsFromPEM(data) {
return nil, fmt.Errorf("cannot parse data from `ca_file` %q for `job_name` %q", sc.TLSConfig.CAFile, jobName)
}
}
}
var err error
var relabelConfigs []promrelabel.ParsedRelabelConfig
relabelConfigs, err = promrelabel.ParseRelabelConfigs(relabelConfigs[:0], sc.RelabelConfigs)
if err != nil {
return nil, fmt.Errorf("cannot parse `relabel_configs` for `job_name` %q: %s", jobName, err)
}
var metricRelabelConfigs []promrelabel.ParsedRelabelConfig
metricRelabelConfigs, err = promrelabel.ParseRelabelConfigs(metricRelabelConfigs[:0], sc.MetricRelabelConfigs)
if err != nil {
return nil, fmt.Errorf("cannot parse `metric_relabel_configs` for `job_name` %q: %s", jobName, err)
}
scrapeLimit := sc.ScrapeLimit
swc := &scrapeWorkConfig{
scrapeInterval: scrapeInterval,
scrapeTimeout: scrapeTimeout,
jobName: jobName,
metricsPath: metricsPath,
scheme: scheme,
params: params,
authorization: authorization,
honorLabels: honorLabels,
honorTimestamps: honorTimestamps,
externalLabels: globalCfg.ExternalLabels,
tlsRootCA: tlsRootCA,
tlsCertificate: tlsCertificate,
tlsServerName: tlsServerName,
tlsInsecureSkipVerify: tlsInsecureSkipVerify,
relabelConfigs: relabelConfigs,
metricRelabelConfigs: metricRelabelConfigs,
scrapeLimit: scrapeLimit,
}
return swc, nil
}
type scrapeWorkConfig struct {
scrapeInterval time.Duration
scrapeTimeout time.Duration
jobName string
metricsPath string
scheme string
params map[string][]string
authorization string
honorLabels bool
honorTimestamps bool
externalLabels map[string]string
tlsRootCA *x509.CertPool
tlsCertificate *tls.Certificate
tlsServerName string
tlsInsecureSkipVerify bool
relabelConfigs []promrelabel.ParsedRelabelConfig
metricRelabelConfigs []promrelabel.ParsedRelabelConfig
scrapeLimit int
metaLabels map[string]string
}
func (sdc *FileSDConfig) appendScrapeWork(dst []ScrapeWork, swPrev map[string][]ScrapeWork, baseDir string, swc *scrapeWorkConfig) ([]ScrapeWork, error) {
for _, file := range sdc.Files {
pathPattern := getFilepath(baseDir, file)
paths := []string{pathPattern}
if strings.Contains(pathPattern, "*") {
var err error
paths, err = filepath.Glob(pathPattern)
if err != nil {
return nil, fmt.Errorf("invalid pattern %q in `files` section: %s", file, err)
}
}
for _, path := range paths {
stcs, err := loadStaticConfigs(path)
if err != nil {
// Do not return this error, since other paths may contain valid scrape configs.
if sws := swPrev[path]; sws != nil {
// Re-use the previous valid scrape work for this path.
logger.Errorf("keeping the previously loaded `static_configs` from %q because of error when re-loading the file: %s", path, err)
dst = append(dst, sws...)
} else {
logger.Errorf("skipping loading `static_configs` from %q because of error: %s", path, err)
}
continue
}
swcCopy := *swc
pathShort := path
if strings.HasPrefix(pathShort, baseDir) {
pathShort = path[len(baseDir):]
if len(pathShort) > 0 && pathShort[0] == filepath.Separator {
pathShort = pathShort[1:]
}
}
swcCopy.metaLabels = map[string]string{
"__meta_filepath": pathShort,
}
for i := range stcs {
dst, err = stcs[i].appendScrapeWork(dst, &swcCopy)
if err != nil {
// Do not return this error, since other paths may contain valid scrape configs.
logger.Errorf("error when parsing `static_config` #%d from %q: %s", i+1, path, err)
continue
}
}
}
}
return dst, nil
}
func (stc *StaticConfig) appendScrapeWork(dst []ScrapeWork, swc *scrapeWorkConfig) ([]ScrapeWork, error) {
for _, target := range stc.Targets {
if target == "" {
return nil, fmt.Errorf("`static_configs` target for `job_name` %q cannot be empty", swc.jobName)
}
target = addMissingPort(swc.scheme, target)
labels, err := mergeLabels(swc.jobName, swc.scheme, target, swc.metricsPath, stc.Labels, swc.externalLabels, swc.metaLabels, swc.params)
if err != nil {
return nil, fmt.Errorf("cannot merge labels for `static_configs` target for `job_name` %q: %s", swc.jobName, err)
}
labels = promrelabel.ApplyRelabelConfigs(labels, 0, swc.relabelConfigs, false)
if len(labels) == 0 {
// Drop target without labels.
continue
}
// See https://www.robustperception.io/life-of-a-label
schemeRelabeled := ""
if schemeLabel := promrelabel.GetLabelByName(labels, "__scheme__"); schemeLabel != nil {
schemeRelabeled = schemeLabel.Value
}
if schemeRelabeled == "" {
schemeRelabeled = "http"
}
addressLabel := promrelabel.GetLabelByName(labels, "__address__")
if addressLabel == nil || addressLabel.Name == "" {
// Drop target without scrape address.
continue
}
targetRelabeled := addMissingPort(schemeRelabeled, addressLabel.Value)
if strings.Contains(targetRelabeled, "/") {
// Drop target with '/'
continue
}
metricsPathRelabeled := ""
if metricsPathLabel := promrelabel.GetLabelByName(labels, "__metrics_path__"); metricsPathLabel != nil {
metricsPathRelabeled = metricsPathLabel.Value
}
if metricsPathRelabeled == "" {
metricsPathRelabeled = "/metrics"
}
paramsRelabeled := getParamsFromLabels(labels, swc.params)
optionalQuestion := "?"
if len(paramsRelabeled) == 0 || strings.Contains(metricsPathRelabeled, "?") {
optionalQuestion = ""
}
paramsStr := url.Values(paramsRelabeled).Encode()
scrapeURL := fmt.Sprintf("%s://%s%s%s%s", schemeRelabeled, targetRelabeled, metricsPathRelabeled, optionalQuestion, paramsStr)
if _, err := url.Parse(scrapeURL); err != nil {
return nil, fmt.Errorf("invalid url %q for scheme=%q (%q), target=%q (%q), metrics_path=%q (%q) for `job_name` %q: %s",
scrapeURL, swc.scheme, schemeRelabeled, target, targetRelabeled, swc.metricsPath, metricsPathRelabeled, swc.jobName, err)
}
dst = append(dst, ScrapeWork{
ScrapeURL: scrapeURL,
ScrapeInterval: swc.scrapeInterval,
ScrapeTimeout: swc.scrapeTimeout,
HonorLabels: swc.honorLabels,
HonorTimestamps: swc.honorTimestamps,
Labels: labels,
Authorization: swc.authorization,
TLSRootCA: swc.tlsRootCA,
TLSCertificate: swc.tlsCertificate,
TLSServerName: swc.tlsServerName,
TLSInsecureSkipVerify: swc.tlsInsecureSkipVerify,
MetricRelabelConfigs: swc.metricRelabelConfigs,
ScrapeLimit: swc.scrapeLimit,
})
}
return dst, nil
}
func getParamsFromLabels(labels []prompbmarshal.Label, paramsOrig map[string][]string) map[string][]string {
// See https://www.robustperception.io/life-of-a-label
m := make(map[string][]string)
for i := range labels {
label := &labels[i]
if !strings.HasPrefix(label.Name, "__param_") {
continue
}
name := label.Name[len("__param_"):]
values := []string{label.Value}
if p := paramsOrig[name]; len(p) > 1 {
values = append(values, p[1:]...)
}
m[name] = values
}
return m
}
func mergeLabels(job, scheme, target, metricsPath string, labels, externalLabels, metaLabels map[string]string, params map[string][]string) ([]prompbmarshal.Label, error) {
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
m := map[string]string{
"job": job,
"__address__": target,
"__scheme__": scheme,
"__metrics_path__": metricsPath,
}
for k, v := range externalLabels {
if vOrig, ok := m[k]; ok {
return nil, fmt.Errorf("external label `%q: %q` clashes with the previously set label with value %q", k, v, vOrig)
}
m[k] = v
}
for k, v := range metaLabels {
if vOrig, ok := m[k]; ok {
return nil, fmt.Errorf("meta label `%q: %q` clashes with the previously set label with value %q", k, v, vOrig)
}
m[k] = v
}
for k, v := range labels {
if vOrig, ok := m[k]; ok {
return nil, fmt.Errorf("label `%q: %q` clashes with the previously set label with value %q", k, v, vOrig)
}
m[k] = v
}
for k, args := range params {
if len(args) == 0 {
continue
}
k = "__param_" + k
v := args[0]
if vOrig, ok := m[k]; ok {
return nil, fmt.Errorf("param `%q: %q` claches with the previously set label with value %q", k, v, vOrig)
}
m[k] = v
}
result := make([]prompbmarshal.Label, 0, len(m))
for k, v := range m {
result = append(result, prompbmarshal.Label{
Name: k,
Value: v,
})
}
return result, nil
}
func getFilepath(baseDir, path string) string {
if filepath.IsAbs(path) {
return path
}
return filepath.Join(baseDir, path)
}
func readPasswordFromFile(path string) (string, error) {
data, err := ioutil.ReadFile(path)
if err != nil {
return "", err
}
pass := strings.TrimRightFunc(string(data), unicode.IsSpace)
return pass, nil
}
func addMissingPort(scheme, target string) string {
if strings.Contains(target, ":") {
return target
}
if scheme == "https" {
target += ":443"
} else {
target += ":80"
}
return target
}
const (
defaultScrapeInterval = time.Minute
defaultScrapeTimeout = 10 * time.Second
)

File diff suppressed because it is too large Load Diff

243
lib/promscrape/scraper.go Normal file
View File

@ -0,0 +1,243 @@
package promscrape
import (
"flag"
"os"
"os/signal"
"sync"
"syscall"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/metrics"
)
var (
fileSDCheckInterval = flag.Duration("promscrape.fileSDCheckInterval", time.Minute, "Interval for checking for changes in 'file_sd_config'. "+
"See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#file_sd_config")
promscrapeConfigFile = flag.String("promscrape.config", "", "Optional path to Prometheus config file with 'scrape_configs' section containing targets to scrape. "+
"See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config for details")
)
// Init initializes Prometheus scraper with config from the `-promscrape.config`.
//
// Scraped data is passed to pushData.
func Init(pushData func(wr *prompbmarshal.WriteRequest)) {
stopCh = make(chan struct{})
scraperWG.Add(1)
go func() {
defer scraperWG.Done()
runScraper(*promscrapeConfigFile, pushData, stopCh)
}()
}
// Stop stops Prometheus scraper.
func Stop() {
close(stopCh)
scraperWG.Wait()
}
var (
stopCh chan struct{}
scraperWG sync.WaitGroup
)
func runScraper(configFile string, pushData func(wr *prompbmarshal.WriteRequest), globalStopCh <-chan struct{}) {
if configFile == "" {
// Nothing to scrape.
return
}
sighupCh := make(chan os.Signal, 1)
signal.Notify(sighupCh, syscall.SIGHUP)
logger.Infof("reading Prometheus configs from %q", configFile)
cfg, err := loadConfig(configFile)
if err != nil {
logger.Fatalf("cannot read %q: %s", configFile, err)
}
swsStatic, err := cfg.getStaticScrapeWork()
if err != nil {
logger.Fatalf("cannot parse `static_configs` from %q: %s", configFile, err)
}
swsFileSD, err := cfg.getFileSDScrapeWork(nil)
if err != nil {
logger.Fatalf("cannot parse `file_sd_config` from %q: %s", configFile, err)
}
mustStop := false
for !mustStop {
stopCh := make(chan struct{})
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
runStaticScrapers(swsStatic, pushData, stopCh)
}()
wg.Add(1)
go func() {
defer wg.Done()
runFileSDScrapers(swsFileSD, cfg, pushData, stopCh)
}()
waitForChans:
select {
case <-sighupCh:
logger.Infof("SIGHUP received; reloading Prometheus configs from %q", configFile)
cfgNew, err := loadConfig(configFile)
if err != nil {
logger.Errorf("cannot read %q: %s; continuing with the previous config", configFile, err)
goto waitForChans
}
swsStaticNew, err := cfg.getStaticScrapeWork()
if err != nil {
logger.Errorf("cannot parse `static_configs` from %q: %s; continuing with the previous config", configFile, err)
goto waitForChans
}
swsFileSDNew, err := cfg.getFileSDScrapeWork(swsFileSD)
if err != nil {
logger.Errorf("cannot parse `file_sd_config` from %q: %s; continuing with the previous config", configFile, err)
}
cfg = cfgNew
swsStatic = swsStaticNew
swsFileSD = swsFileSDNew
case <-globalStopCh:
mustStop = true
}
logger.Infof("stopping Prometheus scrapers")
startTime := time.Now()
close(stopCh)
wg.Wait()
logger.Infof("stopped Prometheus scrapers in %.3f seconds", time.Since(startTime).Seconds())
configReloads.Inc()
}
}
var configReloads = metrics.NewCounter(`vm_promscrape_config_reloads_total`)
func runStaticScrapers(sws []ScrapeWork, pushData func(wr *prompbmarshal.WriteRequest), stopCh <-chan struct{}) {
if len(sws) == 0 {
return
}
logger.Infof("starting %d scrapers for `static_config` targets", len(sws))
staticTargets.Set(uint64(len(sws)))
runScrapeWorkers(sws, pushData, stopCh)
staticTargets.Set(0)
logger.Infof("stopped all the %d scrapers for `static_config` targets", len(sws))
}
var staticTargets = metrics.NewCounter(`vm_promscrape_targets{type="static"}`)
func runFileSDScrapers(sws []ScrapeWork, cfg *Config, pushData func(wr *prompbmarshal.WriteRequest), stopCh <-chan struct{}) {
if cfg.fileSDConfigsCount() == 0 {
return
}
ticker := time.NewTicker(*fileSDCheckInterval)
defer ticker.Stop()
mustStop := false
for !mustStop {
localStopCh := make(chan struct{})
var wg sync.WaitGroup
wg.Add(1)
go func(sws []ScrapeWork) {
defer wg.Done()
logger.Infof("starting %d scrapers for `file_sd_config` targets", len(sws))
fileSDTargets.Set(uint64(len(sws)))
runScrapeWorkers(sws, pushData, localStopCh)
fileSDTargets.Set(0)
logger.Infof("stopped all the %d scrapers for `file_sd_config` targets", len(sws))
}(sws)
waitForChans:
select {
case <-ticker.C:
swsNew, err := cfg.getFileSDScrapeWork(sws)
if err != nil {
logger.Panicf("BUG: error when re-reading `file_sd_config` targets the second time: %s", err)
}
if equalStaticConfigForScrapeWorks(swsNew, sws) {
// Nothing changed, continue waiting for updated scrape work
goto waitForChans
}
logger.Infof("restarting scrapers for changed `file_sd_config` targets")
sws = swsNew
case <-stopCh:
mustStop = true
}
close(localStopCh)
wg.Wait()
fileSDReloads.Inc()
}
}
var (
fileSDTargets = metrics.NewCounter(`vm_promscrape_targets{type="file_sd"}`)
fileSDReloads = metrics.NewCounter(`vm_promscrape_file_sd_reloads_total`)
)
func equalStaticConfigForScrapeWorks(as, bs []ScrapeWork) bool {
if len(as) != len(bs) {
return false
}
for i := range as {
if !equalStaticConfigForScrapeWork(&as[i], &bs[i]) {
return false
}
}
return true
}
func equalStaticConfigForScrapeWork(a, b *ScrapeWork) bool {
// `static_config` can change only ScrapeURL and Labels. So compare only them.
if a.ScrapeURL != b.ScrapeURL {
return false
}
if !equalLabels(a.Labels, b.Labels) {
return false
}
return true
}
func equalLabels(as, bs []prompbmarshal.Label) bool {
if len(as) != len(bs) {
return false
}
for i := range as {
if !equalLabel(&as[i], &bs[i]) {
return false
}
}
return true
}
func equalLabel(a, b *prompbmarshal.Label) bool {
if a.Name != b.Name {
return false
}
if a.Value != b.Value {
return false
}
return true
}
// runScrapeWorkers runs sws.
//
// This function returns after closing stopCh.
func runScrapeWorkers(sws []ScrapeWork, pushData func(wr *prompbmarshal.WriteRequest), stopCh <-chan struct{}) {
var wg sync.WaitGroup
for i := range sws {
cfg := &sws[i]
c := newClient(cfg)
var sw scrapeWork
sw.Config = *cfg
sw.ReadData = c.ReadData
sw.PushData = pushData
wg.Add(1)
go func() {
defer wg.Done()
sw.run(stopCh)
}()
}
wg.Wait()
}

View File

@ -0,0 +1,261 @@
package promscrape
import (
"crypto/tls"
"crypto/x509"
"math/rand"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
"github.com/VictoriaMetrics/metrics"
)
// ScrapeWork represents a unit of work for scraping Prometheus metrics.
type ScrapeWork struct {
// Full URL (including query args) for the scrape.
ScrapeURL string
// Interval for scraping the ScrapeURL.
ScrapeInterval time.Duration
// Timeout for scraping the ScrapeURL.
ScrapeTimeout time.Duration
// How to deal with conflicting labels.
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config
HonorLabels bool
// How to deal with scraped timestamps.
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config
HonorTimestamps bool
// Labels to add to the scraped metrics.
//
// The list contains at least the following labels according to https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
//
// * job
// * __address__
// * __scheme__
// * __metrics_path__
// * __param_<name>
// * __meta_*
// * user-defined labels set via `relabel_configs` section in `scrape_config`
//
// See also https://prometheus.io/docs/concepts/jobs_instances/
Labels []prompbmarshal.Label
// Optional `Authorization` header.
//
// It may contain `Basic ....` or `Bearer ....` string.
Authorization string
// Optional TLS config
TLSRootCA *x509.CertPool
TLSCertificate *tls.Certificate
TLSServerName string
TLSInsecureSkipVerify bool
// Optional `metric_relabel_configs`.
MetricRelabelConfigs []promrelabel.ParsedRelabelConfig
// The maximum number of metrics to scrape after relabeling.
ScrapeLimit int
}
type scrapeWork struct {
// Config for the scrape.
Config ScrapeWork
// ReadData is called for reading the data.
ReadData func(dst []byte) ([]byte, error)
// PushData is called for pushing collected data.
PushData func(wr *prompbmarshal.WriteRequest)
bodyBuf []byte
rows parser.Rows
tmpRow parser.Row
writeRequest prompbmarshal.WriteRequest
labels []prompbmarshal.Label
samples []prompbmarshal.Sample
}
func (sw *scrapeWork) run(stopCh <-chan struct{}) {
// Randomize start time for the first scrape in order to spread load
// when scraping many targets.
randSleep := time.Duration(float64(sw.Config.ScrapeInterval) * rand.Float64())
timer := time.NewTimer(randSleep)
var ticker *time.Ticker
select {
case <-stopCh:
timer.Stop()
return
case t := <-timer.C:
ticker = time.NewTicker(sw.Config.ScrapeInterval)
timestamp := t.UnixNano() / 1e6
sw.scrapeAndLogError(timestamp)
}
defer ticker.Stop()
for {
startTime := time.Now()
select {
case <-stopCh:
return
case t := <-ticker.C:
// Adjust t if it is from the past (i.e. stale tick)
// This can be the case if the previous scrape took longer than the scrape interval.
if t.Sub(startTime) < 0 {
t = startTime
}
timestamp := t.UnixNano() / 1e6
sw.scrapeAndLogError(timestamp)
}
}
}
func (sw *scrapeWork) logError(s string) {
logger.ErrorfSkipframes(1, "error when scraping %q: %s", sw.Config.ScrapeURL, s)
}
func (sw *scrapeWork) scrapeAndLogError(timestamp int64) {
if err := sw.scrapeInternal(timestamp); err != nil {
logger.Errorf("error when scraping %q: %s", sw.Config.ScrapeURL, err)
}
}
var (
scrapeDuration = metrics.NewHistogram("vm_promscrape_scrape_duration_seconds")
scrapeResponseSize = metrics.NewHistogram("vm_promscrape_scrape_response_size_bytes")
scrapedSamples = metrics.NewHistogram("vm_promscrape_scraped_samples")
scrapesSkippedByScrapeLimit = metrics.NewCounter("vm_promscrape_scrapes_skipped_by_scrape_limit_total")
scrapesFailed = metrics.NewCounter("vm_promscrape_scrapes_failed_total")
pushDataDuration = metrics.NewHistogram("vm_promscrape_push_data_duration_seconds")
)
func (sw *scrapeWork) scrapeInternal(timestamp int64) error {
var err error
sw.bodyBuf, err = sw.ReadData(sw.bodyBuf[:0])
endTimestamp := time.Now().UnixNano() / 1e6
duration := float64(endTimestamp-timestamp) / 1e3
scrapeDuration.Update(duration)
scrapeResponseSize.Update(float64(len(sw.bodyBuf)))
up := 1
if err != nil {
up = 0
scrapesFailed.Inc()
} else {
bodyString := bytesutil.ToUnsafeString(sw.bodyBuf)
sw.rows.UnmarshalWithErrLogger(bodyString, sw.logError)
}
srcRows := sw.rows.Rows
samplesScraped := len(srcRows)
scrapedSamples.Update(float64(samplesScraped))
for i := range srcRows {
sw.addRowToTimeseries(&srcRows[i], timestamp)
}
sw.rows.Reset()
if sw.Config.ScrapeLimit > 0 && len(sw.writeRequest.Timeseries) > sw.Config.ScrapeLimit {
prompbmarshal.ResetWriteRequest(&sw.writeRequest)
up = 0
scrapesSkippedByScrapeLimit.Inc()
}
samplesPostRelabeling := len(sw.writeRequest.Timeseries)
sw.addAutoTimeseries("up", float64(up), timestamp)
sw.addAutoTimeseries("scrape_duration_seconds", duration, timestamp)
sw.addAutoTimeseries("scrape_samples_scraped", float64(samplesScraped), timestamp)
sw.addAutoTimeseries("scrape_samples_post_metric_relabeling", float64(samplesPostRelabeling), timestamp)
startTime := time.Now()
sw.PushData(&sw.writeRequest)
pushDataDuration.UpdateDuration(startTime)
prompbmarshal.ResetWriteRequest(&sw.writeRequest)
sw.labels = sw.labels[:0]
sw.samples = sw.samples[:0]
tsmGlobal.Update(&sw.Config, up == 1, timestamp, int64(duration*1000), err)
return err
}
// addAutoTimeseries adds automatically generated time series with the given name, value and timestamp.
//
// See https://prometheus.io/docs/concepts/jobs_instances/#automatically-generated-labels-and-time-series
func (sw *scrapeWork) addAutoTimeseries(name string, value float64, timestamp int64) {
sw.tmpRow.Metric = name
sw.tmpRow.Tags = nil
sw.tmpRow.Value = value
sw.tmpRow.Timestamp = timestamp
sw.addRowToTimeseries(&sw.tmpRow, timestamp)
}
func (sw *scrapeWork) addRowToTimeseries(r *parser.Row, timestamp int64) {
labelsLen := len(sw.labels)
sw.labels = appendLabels(sw.labels, r.Metric, r.Tags, sw.Config.Labels, sw.Config.HonorLabels)
sw.labels = promrelabel.ApplyRelabelConfigs(sw.labels, labelsLen, sw.Config.MetricRelabelConfigs, true)
if len(sw.labels) == labelsLen {
// Skip row without labels.
return
}
labels := sw.labels[labelsLen:]
sw.samples = append(sw.samples, prompbmarshal.Sample{})
sample := &sw.samples[len(sw.samples)-1]
sample.Value = r.Value
sample.Timestamp = r.Timestamp
if !sw.Config.HonorTimestamps || sample.Timestamp == 0 {
sample.Timestamp = timestamp
}
wr := &sw.writeRequest
wr.Timeseries = append(wr.Timeseries, prompbmarshal.TimeSeries{})
ts := &wr.Timeseries[len(wr.Timeseries)-1]
ts.Labels = labels
ts.Samples = sw.samples[len(sw.samples)-1:]
}
func appendLabels(dst []prompbmarshal.Label, metric string, src []parser.Tag, extraLabels []prompbmarshal.Label, honorLabels bool) []prompbmarshal.Label {
dstLen := len(dst)
dst = append(dst, prompbmarshal.Label{
Name: "__name__",
Value: metric,
})
for i := range src {
tag := &src[i]
dst = append(dst, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
dst = append(dst, extraLabels...)
labels := dst[dstLen:]
if len(labels) <= 1 {
// Fast path - only a single label.
return dst
}
// de-duplicate labels
dstLabels := labels[:0]
for i := range labels {
label := &labels[i]
prevLabel := promrelabel.GetLabelByName(dstLabels, label.Name)
if prevLabel == nil {
dstLabels = append(dstLabels, *label)
continue
}
if honorLabels {
// Skip the extra label with the same name.
continue
}
// Rename the prevLabel to "exported_" + label.Name.
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#scrape_config
exportedName := "exported_" + label.Name
if promrelabel.GetLabelByName(dstLabels, exportedName) != nil {
// Override duplicate with the current label.
*prevLabel = *label
continue
}
prevLabel.Name = exportedName
dstLabels = append(dstLabels, *label)
}
return dst[:dstLen+len(dstLabels)]
}

View File

@ -0,0 +1,348 @@
package promscrape
import (
"fmt"
"regexp"
"strings"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
)
func TestScrapeWorkScrapeInternalFailure(t *testing.T) {
dataExpected := `
up 0 123
scrape_samples_scraped 0 123
scrape_duration_seconds 0 123
scrape_samples_post_metric_relabeling 0 123
`
timeseriesExpected := parseData(dataExpected)
var sw scrapeWork
readDataCalls := 0
sw.ReadData = func(dst []byte) ([]byte, error) {
readDataCalls++
return dst, fmt.Errorf("error when reading data")
}
pushDataCalls := 0
var pushDataErr error
sw.PushData = func(wr *prompbmarshal.WriteRequest) {
if err := expectEqualTimeseries(wr.Timeseries, timeseriesExpected); err != nil {
pushDataErr = fmt.Errorf("unexpected data pushed: %s\ngot\n%#v\nwant\n%#v", err, wr.Timeseries, timeseriesExpected)
}
pushDataCalls++
}
timestamp := int64(123)
if err := sw.scrapeInternal(timestamp); err == nil {
t.Fatalf("expecting non-nil error")
}
if pushDataErr != nil {
t.Fatalf("unexpected error: %s", pushDataErr)
}
if readDataCalls != 1 {
t.Fatalf("unexpected number of readData calls; got %d; want %d", readDataCalls, 1)
}
if pushDataCalls != 1 {
t.Fatalf("unexpected number of pushData calls; got %d; want %d", pushDataCalls, 1)
}
}
func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
f := func(data string, cfg *ScrapeWork, dataExpected string) {
t.Helper()
timeseriesExpected := parseData(dataExpected)
var sw scrapeWork
sw.Config = *cfg
readDataCalls := 0
sw.ReadData = func(dst []byte) ([]byte, error) {
readDataCalls++
dst = append(dst, data...)
return dst, nil
}
pushDataCalls := 0
var pushDataErr error
sw.PushData = func(wr *prompbmarshal.WriteRequest) {
if err := expectEqualTimeseries(wr.Timeseries, timeseriesExpected); err != nil {
pushDataErr = fmt.Errorf("unexpected data pushed: %s\ngot\n%#v\nwant\n%#v", err, wr.Timeseries, timeseriesExpected)
}
pushDataCalls++
}
timestamp := int64(123)
if err := sw.scrapeInternal(timestamp); err != nil {
t.Fatalf("unexpected error: %s", err)
}
if pushDataErr != nil {
t.Fatalf("unexpected error: %s", pushDataErr)
}
if readDataCalls != 1 {
t.Fatalf("unexpected number of readData calls; got %d; want %d", readDataCalls, 1)
}
if pushDataCalls != 1 {
t.Fatalf("unexpected number of pushData calls; got %d; want %d", pushDataCalls, 1)
}
}
f(``, &ScrapeWork{}, `
up 1 123
scrape_samples_scraped 0 123
scrape_duration_seconds 0 123
scrape_samples_post_metric_relabeling 0 123
`)
f(`
foo{bar="baz"} 34.45 3
abc -2
`, &ScrapeWork{}, `
foo{bar="baz"} 34.45 123
abc -2 123
up 1 123
scrape_samples_scraped 2 123
scrape_duration_seconds 0 123
scrape_samples_post_metric_relabeling 2 123
`)
f(`
foo{bar="baz"} 34.45 3
abc -2
`, &ScrapeWork{
HonorTimestamps: true,
Labels: []prompbmarshal.Label{
{
Name: "foo",
Value: "x",
},
},
}, `
foo{bar="baz",foo="x"} 34.45 3
abc{foo="x"} -2 123
up{foo="x"} 1 123
scrape_samples_scraped{foo="x"} 2 123
scrape_duration_seconds{foo="x"} 0 123
scrape_samples_post_metric_relabeling{foo="x"} 2 123
`)
f(`
foo{job="orig",bar="baz"} 34.45
bar{y="2",job="aa",a="b",job="bb",x="1"} -3e4 2345
`, &ScrapeWork{
HonorLabels: false,
Labels: []prompbmarshal.Label{
{
Name: "job",
Value: "override",
},
},
}, `
foo{exported_job="orig",job="override",bar="baz"} 34.45 123
bar{exported_job="aa",job="override",x="1",a="b",y="2"} -3e4 123
up{job="override"} 1 123
scrape_samples_scraped{job="override"} 2 123
scrape_duration_seconds{job="override"} 0 123
scrape_samples_post_metric_relabeling{job="override"} 2 123
`)
f(`
foo{job="orig",bar="baz"} 34.45
bar{job="aa",a="b",job="bb"} -3e4 2345
`, &ScrapeWork{
HonorLabels: true,
Labels: []prompbmarshal.Label{
{
Name: "job",
Value: "override",
},
},
}, `
foo{job="orig",bar="baz"} 34.45 123
bar{job="aa",a="b"} -3e4 123
up{job="override"} 1 123
scrape_samples_scraped{job="override"} 2 123
scrape_duration_seconds{job="override"} 0 123
scrape_samples_post_metric_relabeling{job="override"} 2 123
`)
f(`
foo{bar="baz"} 34.44
bar{a="b",c="d"} -3e4
`, &ScrapeWork{
HonorLabels: true,
Labels: []prompbmarshal.Label{
{
Name: "job",
Value: "xx",
},
{
Name: "__address__",
Value: "foo.com",
},
},
MetricRelabelConfigs: []promrelabel.ParsedRelabelConfig{
{
SourceLabels: []string{"__address__", "job"},
Separator: "/",
TargetLabel: "instance",
Regex: defaultRegexForRelabelConfig,
Replacement: "$1",
Action: "replace",
},
{
Action: "labeldrop",
Regex: regexp.MustCompile("^c$"),
},
},
}, `
foo{bar="baz",job="xx",instance="foo.com/xx"} 34.44 123
bar{a="b",job="xx",instance="foo.com/xx"} -3e4 123
up{job="xx",instance="foo.com/xx"} 1 123
scrape_samples_scraped{job="xx",instance="foo.com/xx"} 2 123
scrape_duration_seconds{job="xx",instance="foo.com/xx"} 0 123
scrape_samples_post_metric_relabeling{job="xx",instance="foo.com/xx"} 2 123
`)
f(`
foo{bar="baz"} 34.44
bar{a="b",c="d"} -3e4
dropme{foo="bar"} 334
dropme{xxx="yy",ss="dsf"} 843
`, &ScrapeWork{
HonorLabels: true,
Labels: []prompbmarshal.Label{
{
Name: "job",
Value: "xx",
},
{
Name: "__address__",
Value: "foo.com",
},
},
MetricRelabelConfigs: []promrelabel.ParsedRelabelConfig{
{
Action: "drop",
SourceLabels: []string{"a", "c"},
Regex: regexp.MustCompile("^bd$"),
},
{
Action: "drop",
SourceLabels: []string{"__name__"},
Regex: regexp.MustCompile("^(dropme|up)$"),
},
},
}, `
foo{bar="baz",job="xx",instance="foo.com"} 34.44 123
scrape_samples_scraped{job="xx",instance="foo.com"} 4 123
scrape_duration_seconds{job="xx",instance="foo.com"} 0 123
scrape_samples_post_metric_relabeling{job="xx",instance="foo.com"} 1 123
`)
f(`
foo{bar="baz"} 34.44
bar{a="b",c="d"} -3e4
`, &ScrapeWork{
HonorLabels: true,
ScrapeLimit: 1,
}, `
up 0 123
scrape_samples_scraped 2 123
scrape_duration_seconds 0 123
scrape_samples_post_metric_relabeling 0 123
`)
}
func parseData(data string) []prompbmarshal.TimeSeries {
var rows parser.Rows
errLogger := func(s string) {
panic(fmt.Errorf("unexpected error when unmarshaling Prometheus rows: %s", s))
}
rows.UnmarshalWithErrLogger(data, errLogger)
var tss []prompbmarshal.TimeSeries
for _, r := range rows.Rows {
labels := []prompbmarshal.Label{
{
Name: "__name__",
Value: r.Metric,
},
}
for _, tag := range r.Tags {
labels = append(labels, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
var ts prompbmarshal.TimeSeries
ts.Labels = labels
ts.Samples = []prompbmarshal.Sample{
{
Value: r.Value,
Timestamp: r.Timestamp,
},
}
tss = append(tss, ts)
}
return tss
}
func expectEqualTimeseries(tss, tssExpected []prompbmarshal.TimeSeries) error {
m, err := timeseriesToMap(tss)
if err != nil {
return fmt.Errorf("invalid generated timeseries: %s", err)
}
mExpected, err := timeseriesToMap(tssExpected)
if err != nil {
return fmt.Errorf("invalid expected timeseries: %s", err)
}
if len(m) != len(mExpected) {
return fmt.Errorf("unexpected time series len; got %d; want %d", len(m), len(mExpected))
}
for k, tsExpected := range mExpected {
ts := m[k]
if ts != tsExpected {
return fmt.Errorf("unexpected timeseries %q; got\n%s\nwant\n%s", k, ts, tsExpected)
}
}
return nil
}
func timeseriesToMap(tss []prompbmarshal.TimeSeries) (map[string]string, error) {
m := make(map[string]string, len(tss))
for i := range tss {
ts := &tss[i]
if len(ts.Labels) == 0 {
return nil, fmt.Errorf("unexpected empty labels for timeseries #%d; timeseries: %#v", i, ts)
}
if len(ts.Samples) != 1 {
return nil, fmt.Errorf("unexpected number of samples for timeseries #%d; got %d; want %d", i, len(ts.Samples), 1)
}
if ts.Labels[0].Name != "__name__" {
return nil, fmt.Errorf("unexpected first name for timeseries #%d; got %q; want %q", i, ts.Labels[0].Name, "__name__")
}
if ts.Labels[0].Value == "scrape_duration_seconds" {
// Reset scrape_duration_seconds value to 0, since it is non-deterministic
ts.Samples[0].Value = 0
}
m[ts.Labels[0].Value] = timeseriesToString(ts)
}
return m, nil
}
func timeseriesToString(ts *prompbmarshal.TimeSeries) string {
promrelabel.SortLabels(ts.Labels)
var sb strings.Builder
fmt.Fprintf(&sb, "{")
for i, label := range ts.Labels {
fmt.Fprintf(&sb, "%s=%q", label.Name, label.Value)
if i+1 < len(ts.Labels) {
fmt.Fprintf(&sb, ",")
}
}
fmt.Fprintf(&sb, "} ")
if len(ts.Samples) != 1 {
panic(fmt.Errorf("expecting a single sample; got %d samples", len(ts.Samples)))
}
s := ts.Samples[0]
fmt.Fprintf(&sb, "%g %d", s.Value, s.Timestamp)
return sb.String()
}

View File

@ -0,0 +1,50 @@
package promscrape
import (
"fmt"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
func BenchmarkScrapeWorkScrapeInternal(b *testing.B) {
data := `
vm_tcplistener_accepts_total{name="http", addr=":80"} 1443
vm_tcplistener_accepts_total{name="https", addr=":443"} 12801
vm_tcplistener_conns{name="http", addr=":80"} 0
vm_tcplistener_conns{name="https", addr=":443"} 2
vm_tcplistener_errors_total{name="http", addr=":80", type="accept"} 0
vm_tcplistener_errors_total{name="http", addr=":80", type="close"} 0
vm_tcplistener_errors_total{name="http", addr=":80", type="read"} 97
vm_tcplistener_errors_total{name="http", addr=":80", type="write"} 2
vm_tcplistener_errors_total{name="https", addr=":443", type="accept"} 0
vm_tcplistener_errors_total{name="https", addr=":443", type="close"} 0
vm_tcplistener_errors_total{name="https", addr=":443", type="read"} 243
vm_tcplistener_errors_total{name="https", addr=":443", type="write"} 285
vm_tcplistener_read_bytes_total{name="http", addr=":80"} 879339
vm_tcplistener_read_bytes_total{name="https", addr=":443"} 19453340
vm_tcplistener_read_calls_total{name="http", addr=":80"} 7780
vm_tcplistener_read_calls_total{name="https", addr=":443"} 70323
vm_tcplistener_read_timeouts_total{name="http", addr=":80"} 673
vm_tcplistener_read_timeouts_total{name="https", addr=":443"} 12353
vm_tcplistener_write_calls_total{name="http", addr=":80"} 3996
vm_tcplistener_write_calls_total{name="https", addr=":443"} 132356
`
readDataFunc := func(dst []byte) ([]byte, error) {
return append(dst, data...), nil
}
b.ReportAllocs()
b.SetBytes(int64(len(data)))
b.RunParallel(func(pb *testing.PB) {
var sw scrapeWork
sw.ReadData = readDataFunc
sw.PushData = func(wr *prompbmarshal.WriteRequest) {}
timestamp := int64(0)
for pb.Next() {
if err := sw.scrapeInternal(timestamp); err != nil {
panic(fmt.Errorf("unexpected error: %s", err))
}
timestamp++
}
})
}

View File

@ -0,0 +1,71 @@
package promscrape
import (
"net"
"sync/atomic"
"github.com/VictoriaMetrics/metrics"
"github.com/valyala/fasthttp"
)
func statDial(addr string) (net.Conn, error) {
conn, err := fasthttp.Dial(addr)
dialsTotal.Inc()
if err != nil {
dialErrors.Inc()
return nil, err
}
conns.Inc()
sc := &statConn{
Conn: conn,
}
return sc, nil
}
var (
dialsTotal = metrics.NewCounter(`vm_promscrape_dials_total`)
dialErrors = metrics.NewCounter(`vm_promscrape_dial_errors_total`)
conns = metrics.NewCounter(`vm_promscrape_conns`)
)
type statConn struct {
closed uint64
net.Conn
}
func (sc *statConn) Read(p []byte) (int, error) {
n, err := sc.Conn.Read(p)
connReadsTotal.Inc()
if err != nil {
connReadErrors.Inc()
}
connBytesRead.Add(n)
return n, err
}
func (sc *statConn) Write(p []byte) (int, error) {
n, err := sc.Conn.Write(p)
connWritesTotal.Inc()
if err != nil {
connWriteErrors.Inc()
}
connBytesWritten.Add(n)
return n, err
}
func (sc *statConn) Close() error {
err := sc.Conn.Close()
if atomic.AddUint64(&sc.closed, 1) == 1 {
conns.Dec()
}
return err
}
var (
connReadsTotal = metrics.NewCounter(`vm_promscrape_conn_reads_total`)
connWritesTotal = metrics.NewCounter(`vm_promscrape_conn_writes_total`)
connReadErrors = metrics.NewCounter(`vm_promscrape_conn_read_errors_total`)
connWriteErrors = metrics.NewCounter(`vm_promscrape_conn_write_errors_total`)
connBytesRead = metrics.NewCounter(`vm_promscrape_conn_bytes_read_total`)
connBytesWritten = metrics.NewCounter(`vm_promscrape_conn_bytes_written_total`)
)

View File

@ -0,0 +1,127 @@
package promscrape
import (
"fmt"
"io"
"sort"
"strings"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
)
var tsmGlobal = newTargetStatusMap()
// WriteHumanReadableTargetsStatus writes human-readable status for all the scrape targets to w.
func WriteHumanReadableTargetsStatus(w io.Writer) {
tsmGlobal.WriteHumanReadable(w)
}
type targetStatusMap struct {
mu sync.Mutex
m map[string]targetStatus
}
func newTargetStatusMap() *targetStatusMap {
return &targetStatusMap{
m: make(map[string]targetStatus),
}
}
func (tsm *targetStatusMap) Reset() {
tsm.mu.Lock()
tsm.m = make(map[string]targetStatus)
tsm.mu.Unlock()
}
func (tsm *targetStatusMap) Update(sw *ScrapeWork, up bool, scrapeTime, scrapeDuration int64, err error) {
tsm.mu.Lock()
tsm.m[sw.ScrapeURL] = targetStatus{
sw: sw,
up: up,
scrapeTime: scrapeTime,
scrapeDuration: scrapeDuration,
err: err,
}
tsm.mu.Unlock()
}
func (tsm *targetStatusMap) WriteHumanReadable(w io.Writer) {
byJob := make(map[string][]targetStatus)
tsm.mu.Lock()
for k, st := range tsm.m {
if st.getDurationFromLastScrape() > 10*st.sw.ScrapeInterval {
// Remove obsolete targets
delete(tsm.m, k)
continue
}
job := ""
label := promrelabel.GetLabelByName(st.sw.Labels, "job")
if label != nil {
job = label.Value
}
byJob[job] = append(byJob[job], st)
}
tsm.mu.Unlock()
var jss []jobStatus
for job, statuses := range byJob {
jss = append(jss, jobStatus{
job: job,
statuses: statuses,
})
}
sort.Slice(jss, func(i, j int) bool {
return jss[i].job < jss[j].job
})
for _, js := range jss {
sts := js.statuses
sort.Slice(sts, func(i, j int) bool {
return sts[i].sw.ScrapeURL < sts[j].sw.ScrapeURL
})
ups := 0
for _, st := range sts {
if st.up {
ups++
}
}
fmt.Fprintf(w, "job=%q (%d/%d up)\n", js.job, ups, len(sts))
for _, st := range sts {
state := "up"
if !st.up {
state = "down"
}
var labels []string
for _, label := range promrelabel.FinalizeLabels(nil, st.sw.Labels) {
labels = append(labels, fmt.Sprintf("%s=%q", label.Name, label.Value))
}
labelsStr := "{" + strings.Join(labels, ", ") + "}"
lastScrape := st.getDurationFromLastScrape()
errMsg := ""
if st.err != nil {
errMsg = st.err.Error()
}
fmt.Fprintf(w, "\tstate=%s, endpoint=%s, labels=%s, last_scrape=%.3fs ago, scrape_duration=%.3fs, error=%q\n",
state, st.sw.ScrapeURL, labelsStr, lastScrape.Seconds(), float64(st.scrapeDuration)/1000, errMsg)
}
}
}
type jobStatus struct {
job string
statuses []targetStatus
}
type targetStatus struct {
sw *ScrapeWork
up bool
scrapeTime int64
scrapeDuration int64
err error
}
func (st *targetStatus) getDurationFromLastScrape() time.Duration {
return time.Since(time.Unix(st.scrapeTime/1000, (st.scrapeTime%1000)*1e6))
}

View File

@ -0,0 +1 @@
- targets: ["foo", ""]

8
lib/promscrape/testdata/file_sd.json vendored Normal file
View File

@ -0,0 +1,8 @@
[
{
"targets": ["host1", "host2"],
"labels": {
"qwe": "rty"
}
}
]

3
lib/promscrape/testdata/file_sd_1.yml vendored Normal file
View File

@ -0,0 +1,3 @@
- targets: [localhost:9090]
labels:
yml: test

1
lib/promscrape/testdata/password.txt vendored Normal file
View File

@ -0,0 +1 @@
secret-pass

View File

@ -0,0 +1,6 @@
scrape_configs:
- job_name: foo
file_sd_configs:
- files: ["file_sd_*.yml"]
refresh_interval: 10s
- fules: ["file_sd.json"]

View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQD4IQusAs8PJdnG
3mURt/AXtgC+ceqLOatJ49JJE1VPTkMAy+oE1f1XvkMrYsHqmDf6GWVzgVXryL4U
wq2/nJSm56ddhN55nI8oSN3dtywUB8/ShelEN73nlN77PeD9tl6NksPwWaKrqxq0
FlabRPZSQCfmgZbhDV8Sa8mfCkFU0G0lit6kLGceCKMvmW+9Bz7ebsYmVdmVMxmf
IJStFD44lWFTdUc65WISKEdW2ELcUefb0zOLw+0PCbXFGJH5x5ktksW8+BBk2Hkg
GeQRL/qPCccthbScO0VgNj3zJ3ZZL0ObSDAbvNDG85joeNjDNq5DT/BAZ0bOSbEF
sh+f9BAzAgMBAAECggEBAJWv2cq7Jw6MVwSRxYca38xuD6TUNBopgBvjREixURW2
sNUaLuMb9Omp7fuOaE2N5rcJ+xnjPGIxh/oeN5MQctz9gwn3zf6vY+15h97pUb4D
uGvYPRDaT8YVGS+X9NMZ4ZCmqW2lpWzKnCFoGHcy8yZLbcaxBsRdvKzwOYGoPiFb
K2QuhXZ/1UPmqK9i2DFKtj40X6vBszTNboFxOVpXrPu0FJwLVSDf2hSZ4fMM0DH3
YqwKcYf5te+hxGKgrqRA3tn0NCWii0in6QIwXMC+kMw1ebg/tZKqyDLMNptAK8J+
DVw9m5X1seUHS5ehU/g2jrQrtK5WYn7MrFK4lBzlRwECgYEA/d1TeANYECDWRRDk
B0aaRZs87Rwl/J9PsvbsKvtU/bX+OfSOUjOa9iQBqn0LmU8GqusEET/QVUfocVwV
Bggf/5qDLxz100Rj0ags/yE/kNr0Bb31kkkKHFMnCT06YasR7qKllwrAlPJvQv9x
IzBKq+T/Dx08Wep9bCRSFhzRCnsCgYEA+jdeZXTDr/Vz+D2B3nAw1frqYFfGnEVY
wqmoK3VXMDkGuxsloO2rN+SyiUo3JNiQNPDub/t7175GH5pmKtZOlftePANsUjBj
wZ1D0rI5Bxu/71ibIUYIRVmXsTEQkh/ozoh3jXCZ9+bLgYiYx7789IUZZSokFQ3D
FICUT9KJ36kCgYAGoq9Y1rWJjmIrYfqj2guUQC+CfxbbGIrrwZqAsRsSmpwvhZ3m
tiSZxG0quKQB+NfSxdvQW5ulbwC7Xc3K35F+i9pb8+TVBdeaFkw+yu6vaZmxQLrX
fQM/pEjD7A7HmMIaO7QaU5SfEAsqdCTP56Y8AftMuNXn/8IRfo2KuGwaWwKBgFpU
ILzJoVdlad9E/Rw7LjYhZfkv1uBVXIyxyKcfrkEXZSmozDXDdxsvcZCEfVHM6Ipk
K/+7LuMcqp4AFEAEq8wTOdq6daFaHLkpt/FZK6M4TlruhtpFOPkoNc3e45eM83OT
6mziKINJC1CQ6m65sQHpBtjxlKMRG8rL/D6wx9s5AoGBAMRlqNPMwglT3hvDmsAt
9Lf9pdmhERUlHhD8bj8mDaBj2Aqv7f6VRJaYZqP403pKKQexuqcn80mtjkSAPFkN
Cj7BVt/RXm5uoxDTnfi26RF9F6yNDEJ7UU9+peBr99aazF/fTgW/1GcMkQnum8uV
c257YgaWmjK9uB0Y2r2VxS0G
-----END PRIVATE KEY-----

View File

@ -0,0 +1,17 @@
-----BEGIN CERTIFICATE-----
MIICujCCAaKgAwIBAgIJAMbXnKZ/cikUMA0GCSqGSIb3DQEBCwUAMBUxEzARBgNV
BAMTCnVidW50dS5uYW4wHhcNMTUwMjA0MDgwMTM5WhcNMjUwMjAxMDgwMTM5WjAV
MRMwEQYDVQQDEwp1YnVudHUubmFuMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
CgKCAQEA+CELrALPDyXZxt5lEbfwF7YAvnHqizmrSePSSRNVT05DAMvqBNX9V75D
K2LB6pg3+hllc4FV68i+FMKtv5yUpuenXYTeeZyPKEjd3bcsFAfP0oXpRDe955Te
+z3g/bZejZLD8Fmiq6satBZWm0T2UkAn5oGW4Q1fEmvJnwpBVNBtJYrepCxnHgij
L5lvvQc+3m7GJlXZlTMZnyCUrRQ+OJVhU3VHOuViEihHVthC3FHn29Mzi8PtDwm1
xRiR+ceZLZLFvPgQZNh5IBnkES/6jwnHLYW0nDtFYDY98yd2WS9Dm0gwG7zQxvOY
6HjYwzauQ0/wQGdGzkmxBbIfn/QQMwIDAQABow0wCzAJBgNVHRMEAjAAMA0GCSqG
SIb3DQEBCwUAA4IBAQBQjKm/4KN/iTgXbLTL3i7zaxYXFLXsnT1tF+ay4VA8aj98
L3JwRTciZ3A5iy/W4VSCt3eASwOaPWHKqDBB5RTtL73LoAqsWmO3APOGQAbixcQ2
45GXi05OKeyiYRi1Nvq7Unv9jUkRDHUYVPZVSAjCpsXzPhFkmZoTRxmx5l0ZF7Li
K91lI5h+eFq0dwZwrmlPambyh1vQUi70VHv8DNToVU29kel7YLbxGbuqETfhrcy6
X+Mha6RYITkAn5FqsZcKMsc9eYGEF4l3XV+oS7q6xfTxktYJMFTI18J0lQ2Lv/CI
whdMnYGntDQBE/iFCrJEGNsKGc38796GBOb5j+zd
-----END CERTIFICATE-----

View File

@ -0,0 +1,130 @@
package graphite
import (
"fmt"
"io"
"net"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common"
"github.com/VictoriaMetrics/metrics"
)
// ParseStream parses Graphite lines from r and calls callback for the parsed rows.
//
// The callback can be called multiple times for streamed data from r.
//
// callback shouldn't hold rows after returning.
func ParseStream(r io.Reader, callback func(rows []Row) error) error {
ctx := getStreamContext()
defer putStreamContext(ctx)
for ctx.Read(r) {
if err := callback(ctx.Rows.Rows); err != nil {
return err
}
}
return ctx.Error()
}
const flushTimeout = 3 * time.Second
func (ctx *streamContext) Read(r io.Reader) bool {
readCalls.Inc()
if ctx.err != nil {
return false
}
if c, ok := r.(net.Conn); ok {
if err := c.SetReadDeadline(time.Now().Add(flushTimeout)); err != nil {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot set read deadline: %s", err)
return false
}
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ne, ok := ctx.err.(net.Error); ok && ne.Timeout() {
// Flush the read data on timeout and try reading again.
ctx.err = nil
} else {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read graphite plaintext protocol data: %s", ctx.err)
}
return false
}
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
rowsRead.Add(len(ctx.Rows.Rows))
// Fill missing timestamps with the current timestamp rounded to seconds.
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps from seconds to milliseconds.
for i := range rows {
rows[i].Timestamp *= 1e3
}
return true
}
type streamContext struct {
Rows Rows
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *streamContext) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *streamContext) reset() {
ctx.Rows.Reset()
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
var (
readCalls = metrics.NewCounter(`vm_protoparser_graphite_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_graphite_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_graphite_rows_read_total`)
)
func getStreamContext() *streamContext {
select {
case ctx := <-streamContextPoolCh:
return ctx
default:
if v := streamContextPool.Get(); v != nil {
return v.(*streamContext)
}
return &streamContext{}
}
}
func putStreamContext(ctx *streamContext) {
ctx.reset()
select {
case streamContextPoolCh <- ctx:
default:
streamContextPool.Put(ctx)
}
}
var streamContextPool sync.Pool
var streamContextPoolCh = make(chan *streamContext, runtime.GOMAXPROCS(-1))

View File

@ -0,0 +1,153 @@
package influx
import (
"fmt"
"io"
"net/http"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common"
"github.com/VictoriaMetrics/metrics"
)
// ParseStream parses req and calls callback for the parsed rows.
//
// The callback can be called multiple times for streamed data from req.
//
// callback shouldn't hold rows after returning.
func ParseStream(req *http.Request, callback func(db string, rows []Row) error) error {
readCalls.Inc()
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
return fmt.Errorf("cannot read gzipped influx line protocol data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
q := req.URL.Query()
tsMultiplier := int64(1e6)
switch q.Get("precision") {
case "ns":
tsMultiplier = 1e6
case "u":
tsMultiplier = 1e3
case "ms":
tsMultiplier = 1
case "s":
tsMultiplier = -1e3
case "m":
tsMultiplier = -1e3 * 60
case "h":
tsMultiplier = -1e3 * 3600
}
// Read db tag from https://docs.influxdata.com/influxdb/v1.7/tools/api/#write-http-endpoint
db := q.Get("db")
ctx := getStreamContext()
defer putStreamContext(ctx)
for ctx.Read(r, tsMultiplier) {
if err := callback(db, ctx.Rows.Rows); err != nil {
return err
}
}
return ctx.Error()
}
func (ctx *streamContext) Read(r io.Reader, tsMultiplier int64) bool {
if ctx.err != nil {
return false
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read influx line protocol data: %s", ctx.err)
}
return false
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
rowsRead.Add(len(ctx.Rows.Rows))
// Adjust timestamps according to tsMultiplier
currentTs := time.Now().UnixNano() / 1e6
if tsMultiplier >= 1 {
for i := range ctx.Rows.Rows {
row := &ctx.Rows.Rows[i]
if row.Timestamp == 0 {
row.Timestamp = currentTs
} else {
row.Timestamp /= tsMultiplier
}
}
} else if tsMultiplier < 0 {
tsMultiplier = -tsMultiplier
currentTs -= currentTs % tsMultiplier
for i := range ctx.Rows.Rows {
row := &ctx.Rows.Rows[i]
if row.Timestamp == 0 {
row.Timestamp = currentTs
} else {
row.Timestamp *= tsMultiplier
}
}
}
return true
}
var (
readCalls = metrics.NewCounter(`vm_protoparser_influx_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_influx_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_influx_rows_read_total`)
)
type streamContext struct {
Rows Rows
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *streamContext) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *streamContext) reset() {
ctx.Rows.Reset()
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
func getStreamContext() *streamContext {
select {
case ctx := <-streamContextPoolCh:
return ctx
default:
if v := streamContextPool.Get(); v != nil {
return v.(*streamContext)
}
return &streamContext{}
}
}
func putStreamContext(ctx *streamContext) {
ctx.reset()
select {
case streamContextPoolCh <- ctx:
default:
streamContextPool.Put(ctx)
}
}
var streamContextPool sync.Pool
var streamContextPoolCh = make(chan *streamContext, runtime.GOMAXPROCS(-1))

View File

@ -0,0 +1,128 @@
package opentsdb
import (
"fmt"
"io"
"net"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common"
"github.com/VictoriaMetrics/metrics"
)
// ParseStream parses OpenTSDB lines from r and calls callback for the parsed rows.
//
// The callback can be called multiple times for streamed data from r.
//
// callback shouldn't hold rows after returning.
func ParseStream(r io.Reader, callback func(rows []Row) error) error {
ctx := getStreamContext()
defer putStreamContext(ctx)
for ctx.Read(r) {
if err := callback(ctx.Rows.Rows); err != nil {
return err
}
}
return ctx.Error()
}
const flushTimeout = 3 * time.Second
func (ctx *streamContext) Read(r io.Reader) bool {
readCalls.Inc()
if ctx.err != nil {
return false
}
if c, ok := r.(net.Conn); ok {
if err := c.SetReadDeadline(time.Now().Add(flushTimeout)); err != nil {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot set read deadline: %s", err)
return false
}
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(r, ctx.reqBuf, ctx.tailBuf)
if ctx.err != nil {
if ne, ok := ctx.err.(net.Error); ok && ne.Timeout() {
// Flush the read data on timeout and try reading again.
ctx.err = nil
} else {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read OpenTSDB put protocol data: %s", ctx.err)
}
return false
}
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
rowsRead.Add(len(ctx.Rows.Rows))
// Fill in missing timestamps
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps from seconds to milliseconds
for i := range rows {
rows[i].Timestamp *= 1e3
}
return true
}
type streamContext struct {
Rows Rows
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *streamContext) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *streamContext) reset() {
ctx.Rows.Reset()
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
var (
readCalls = metrics.NewCounter(`vm_protoparser_opentsdb_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_opentsdb_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_opentsdb_rows_read_total`)
)
func getStreamContext() *streamContext {
select {
case ctx := <-streamContextPoolCh:
return ctx
default:
if v := streamContextPool.Get(); v != nil {
return v.(*streamContext)
}
return &streamContext{}
}
}
func putStreamContext(ctx *streamContext) {
ctx.reset()
select {
case streamContextPoolCh <- ctx:
default:
streamContextPool.Put(ctx)
}
}
var streamContextPool sync.Pool
var streamContextPoolCh = make(chan *streamContext, runtime.GOMAXPROCS(-1))

View File

@ -156,7 +156,7 @@ func unmarshalRow(dst []Row, o *fastjson.Value, tagsPool []Tag) ([]Row, []Tag) {
return dst, tagsPool
}
var invalidLines = metrics.NewCounter(`vm_rows_invalid_total{type="opentsdb-http"}`)
var invalidLines = metrics.NewCounter(`vm_rows_invalid_total{type="opentsdbhttp"}`)
func unmarshalTags(dst []Tag, o *fastjson.Object) ([]Tag, error) {
var err error

View File

@ -0,0 +1,127 @@
package opentsdbhttp
import (
"flag"
"fmt"
"io"
"net/http"
"runtime"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common"
"github.com/VictoriaMetrics/metrics"
)
var maxInsertRequestSize = flag.Int("opentsdbhttp.maxInsertRequestSize", 32*1024*1024, "The maximum size of OpenTSDB HTTP put request")
// ParseStream parses OpenTSDB http lines from req and calls callback for the parsed rows.
//
// The callback can be called multiple times for streamed data from req.
//
// callback shouldn't hold rows after returning.
func ParseStream(req *http.Request, callback func(rows []Row) error) error {
readCalls.Inc()
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
readErrors.Inc()
return fmt.Errorf("cannot read gzipped http protocol data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
ctx := getStreamContext()
defer putStreamContext(ctx)
// Read the request in ctx.reqBuf
lr := io.LimitReader(r, int64(*maxInsertRequestSize)+1)
reqLen, err := ctx.reqBuf.ReadFrom(lr)
if err != nil {
readErrors.Inc()
return fmt.Errorf("cannot read HTTP OpenTSDB request: %s", err)
}
if reqLen > int64(*maxInsertRequestSize) {
readErrors.Inc()
return fmt.Errorf("too big HTTP OpenTSDB request; mustn't exceed `-opentsdbhttp.maxInsertRequestSize=%d` bytes", *maxInsertRequestSize)
}
// Unmarshal the request to ctx.Rows
p := GetParser()
defer PutParser(p)
v, err := p.ParseBytes(ctx.reqBuf.B)
if err != nil {
unmarshalErrors.Inc()
return fmt.Errorf("cannot parse HTTP OpenTSDB json: %s", err)
}
ctx.Rows.Unmarshal(v)
rowsRead.Add(len(ctx.Rows.Rows))
// Fill in missing timestamps
currentTimestamp := time.Now().Unix()
rows := ctx.Rows.Rows
for i := range rows {
r := &rows[i]
if r.Timestamp == 0 {
r.Timestamp = currentTimestamp
}
}
// Convert timestamps in seconds to milliseconds if needed.
// See http://opentsdb.net/docs/javadoc/net/opentsdb/core/Const.html#SECOND_MASK
for i := range rows {
r := &rows[i]
if r.Timestamp&secondMask == 0 {
r.Timestamp *= 1e3
}
}
// Insert ctx.Rows to db.
return callback(ctx.Rows.Rows)
}
const secondMask int64 = 0x7FFFFFFF00000000
type streamContext struct {
Rows Rows
reqBuf bytesutil.ByteBuffer
}
func (ctx *streamContext) reset() {
ctx.Rows.Reset()
ctx.reqBuf.Reset()
}
var (
readCalls = metrics.NewCounter(`vm_protoparser_opentsdbhttp_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_opentsdbhttp_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_opentsdbhttp_rows_read_total`)
unmarshalErrors = metrics.NewCounter(`vm_protoparser_opentsdbhttp_unmarshal_errors_total`)
)
func getStreamContext() *streamContext {
select {
case ctx := <-streamContextPoolCh:
return ctx
default:
if v := streamContextPool.Get(); v != nil {
return v.(*streamContext)
}
return &streamContext{}
}
}
func putStreamContext(ctx *streamContext) {
ctx.reset()
select {
case streamContextPoolCh <- ctx:
default:
streamContextPool.Put(ctx)
}
}
var streamContextPool sync.Pool
var streamContextPoolCh = make(chan *streamContext, runtime.GOMAXPROCS(-1))

View File

@ -38,8 +38,19 @@ func (rs *Rows) Reset() {
//
// s must be unchanged until rs is in use.
func (rs *Rows) Unmarshal(s string) {
rs.UnmarshalWithErrLogger(s, stdErrLogger)
}
func stdErrLogger(s string) {
logger.ErrorfSkipframes(1, "%s", s)
}
// UnmarshalWithErrLogger unmarshal Prometheus exposition text rows from s.
//
// It calls errLogger for logging parsing errors.
func (rs *Rows) UnmarshalWithErrLogger(s string, errLogger func(s string)) {
noEscapes := strings.IndexByte(s, '\\') < 0
rs.Rows, rs.tagsPool = unmarshalRows(rs.Rows[:0], s, rs.tagsPool[:0], noEscapes)
rs.Rows, rs.tagsPool = unmarshalRows(rs.Rows[:0], s, rs.tagsPool[:0], noEscapes, errLogger)
}
// Row is a single Prometheus row.
@ -136,20 +147,20 @@ func (r *Row) unmarshal(s string, tagsPool []Tag, noEscapes bool) ([]Tag, error)
return tagsPool, nil
}
func unmarshalRows(dst []Row, s string, tagsPool []Tag, noEscapes bool) ([]Row, []Tag) {
func unmarshalRows(dst []Row, s string, tagsPool []Tag, noEscapes bool, errLogger func(s string)) ([]Row, []Tag) {
for len(s) > 0 {
n := strings.IndexByte(s, '\n')
if n < 0 {
// The last line.
return unmarshalRow(dst, s, tagsPool, noEscapes)
return unmarshalRow(dst, s, tagsPool, noEscapes, errLogger)
}
dst, tagsPool = unmarshalRow(dst, s[:n], tagsPool, noEscapes)
dst, tagsPool = unmarshalRow(dst, s[:n], tagsPool, noEscapes, errLogger)
s = s[n+1:]
}
return dst, tagsPool
}
func unmarshalRow(dst []Row, s string, tagsPool []Tag, noEscapes bool) ([]Row, []Tag) {
func unmarshalRow(dst []Row, s string, tagsPool []Tag, noEscapes bool, errLogger func(s string)) ([]Row, []Tag) {
if len(s) > 0 && s[len(s)-1] == '\r' {
s = s[:len(s)-1]
}
@ -172,7 +183,8 @@ func unmarshalRow(dst []Row, s string, tagsPool []Tag, noEscapes bool) ([]Row, [
tagsPool, err = r.unmarshal(s, tagsPool, noEscapes)
if err != nil {
dst = dst[:len(dst)-1]
logger.Errorf("cannot unmarshal Prometheus line %q: %s", s, err)
msg := fmt.Sprintf("cannot unmarshal Prometheus line %q: %s", s, err)
errLogger(msg)
invalidLines.Inc()
}
return dst, tagsPool

View File

@ -1,4 +1,4 @@
package prometheus
package promremotewrite
import (
"flag"
@ -8,8 +8,6 @@ import (
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/concurrencylimiter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
"github.com/VictoriaMetrics/metrics"
@ -18,79 +16,56 @@ import (
var maxInsertRequestSize = flag.Int("maxInsertRequestSize", 32*1024*1024, "The maximum size in bytes of a single Prometheus remote_write API request")
var (
rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="prometheus"}`)
rowsPerInsert = metrics.NewSummary(`vm_rows_per_insert{type="prometheus"}`)
)
// InsertHandler processes remote write for prometheus.
func InsertHandler(r *http.Request) error {
return concurrencylimiter.Do(func() error {
return insertHandlerInternal(r)
})
}
func insertHandlerInternal(r *http.Request) error {
// ParseStream parses Prometheus remote_write message req and calls callback for the parsed timeseries.
//
// callback shouldn't hold timeseries after returning.
func ParseStream(req *http.Request, callback func(timeseries []prompb.TimeSeries) error) error {
ctx := getPushCtx()
defer putPushCtx(ctx)
if err := ctx.Read(r); err != nil {
if err := ctx.Read(req); err != nil {
return err
}
timeseries := ctx.req.Timeseries
rowsLen := 0
for i := range timeseries {
rowsLen += len(timeseries[i].Samples)
}
ic := &ctx.Common
ic.Reset(rowsLen)
rowsTotal := 0
for i := range timeseries {
ts := &timeseries[i]
var metricNameRaw []byte
for i := range ts.Samples {
r := &ts.Samples[i]
metricNameRaw = ic.WriteDataPointExt(metricNameRaw, ts.Labels, r.Timestamp, r.Value)
}
rowsTotal += len(ts.Samples)
}
rowsInserted.Add(rowsTotal)
rowsPerInsert.Update(float64(rowsTotal))
return ic.FlushBufs()
return callback(ctx.wr.Timeseries)
}
type pushCtx struct {
Common common.InsertCtx
req prompb.WriteRequest
wr prompb.WriteRequest
reqBuf []byte
}
func (ctx *pushCtx) reset() {
ctx.Common.Reset(0)
ctx.req.Reset()
ctx.wr.Reset()
ctx.reqBuf = ctx.reqBuf[:0]
}
func (ctx *pushCtx) Read(r *http.Request) error {
prometheusReadCalls.Inc()
readCalls.Inc()
var err error
ctx.reqBuf, err = readSnappy(ctx.reqBuf[:0], r.Body)
if err != nil {
prometheusReadErrors.Inc()
readErrors.Inc()
return fmt.Errorf("cannot read prompb.WriteRequest: %s", err)
}
if err = ctx.req.Unmarshal(ctx.reqBuf); err != nil {
prometheusUnmarshalErrors.Inc()
if err = ctx.wr.Unmarshal(ctx.reqBuf); err != nil {
unmarshalErrors.Inc()
return fmt.Errorf("cannot unmarshal prompb.WriteRequest with size %d bytes: %s", len(ctx.reqBuf), err)
}
rows := 0
tss := ctx.wr.Timeseries
for i := range tss {
rows += len(tss[i].Samples)
}
rowsRead.Add(rows)
return nil
}
var (
prometheusReadCalls = metrics.NewCounter(`vm_read_calls_total{name="prometheus"}`)
prometheusReadErrors = metrics.NewCounter(`vm_read_errors_total{name="prometheus"}`)
prometheusUnmarshalErrors = metrics.NewCounter(`vm_unmarshal_errors_total{name="prometheus"}`)
readCalls = metrics.NewCounter(`vm_protoparser_promremotewrite_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_promremotewrite_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_promremotewrite_rows_read_total`)
unmarshalErrors = metrics.NewCounter(`vm_protoparser_promremotewrite_unmarshal_errors`)
)
func getPushCtx() *pushCtx {

View File

@ -0,0 +1,111 @@
package vmimport
import (
"flag"
"fmt"
"io"
"net/http"
"runtime"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common"
"github.com/VictoriaMetrics/metrics"
)
var maxLineLen = flag.Int("import.maxLineLen", 100*1024*1024, "The maximum length in bytes of a single line accepted by /api/v1/import")
// ParseStream parses /api/v1/import lines from req and calls callback for the parsed rows.
//
// The callback can be called multiple times for streamed data from req.
//
// callback shouldn't hold rows after returning.
func ParseStream(req *http.Request, callback func(rows []Row) error) error {
readCalls.Inc()
r := req.Body
if req.Header.Get("Content-Encoding") == "gzip" {
zr, err := common.GetGzipReader(r)
if err != nil {
return fmt.Errorf("cannot read gzipped vmimport data: %s", err)
}
defer common.PutGzipReader(zr)
r = zr
}
ctx := getStreamContext()
defer putStreamContext(ctx)
for ctx.Read(r) {
if err := callback(ctx.Rows.Rows); err != nil {
return err
}
}
return ctx.Error()
}
func (ctx *streamContext) Read(r io.Reader) bool {
if ctx.err != nil {
return false
}
ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlockExt(r, ctx.reqBuf, ctx.tailBuf, *maxLineLen)
if ctx.err != nil {
if ctx.err != io.EOF {
readErrors.Inc()
ctx.err = fmt.Errorf("cannot read vmimport data: %s", ctx.err)
}
return false
}
ctx.Rows.Unmarshal(bytesutil.ToUnsafeString(ctx.reqBuf))
rowsRead.Add(len(ctx.Rows.Rows))
return true
}
var (
readCalls = metrics.NewCounter(`vm_protoparser_vmimport_read_calls_total`)
readErrors = metrics.NewCounter(`vm_protoparser_vmimport_read_errors_total`)
rowsRead = metrics.NewCounter(`vm_protoparser_vmimport_rows_read_total`)
)
type streamContext struct {
Rows Rows
reqBuf []byte
tailBuf []byte
err error
}
func (ctx *streamContext) Error() error {
if ctx.err == io.EOF {
return nil
}
return ctx.err
}
func (ctx *streamContext) reset() {
ctx.Rows.Reset()
ctx.reqBuf = ctx.reqBuf[:0]
ctx.tailBuf = ctx.tailBuf[:0]
ctx.err = nil
}
func getStreamContext() *streamContext {
select {
case ctx := <-streamContextPoolCh:
return ctx
default:
if v := streamContextPool.Get(); v != nil {
return v.(*streamContext)
}
return &streamContext{}
}
}
func putStreamContext(ctx *streamContext) {
ctx.reset()
select {
case streamContextPoolCh <- ctx:
default:
streamContextPool.Put(ctx)
}
}
var streamContextPool sync.Pool
var streamContextPoolCh = make(chan *streamContext, runtime.GOMAXPROCS(-1))

View File

@ -1,4 +1,4 @@
package concurrencylimiter
package writeconcurrencylimiter
import (
"flag"
@ -54,8 +54,7 @@ func Do(f func() error) error {
concurrencyLimitTimeout.Inc()
return &httpserver.ErrorWithStatusCode{
Err: fmt.Errorf("cannot handle more than %d concurrent inserts during %s; possible solutions: "+
"increase `-insert.maxQueueDuration`, increase `-maxConcurrentInserts`, "+
"decrease `-search.maxConcurrentRequests`, increase server capacity", *maxConcurrentInserts, *maxQueueDuration),
"increase `-insert.maxQueueDuration`, increase `-maxConcurrentInserts`, increase server capacity", *maxConcurrentInserts, *maxQueueDuration),
StatusCode: http.StatusServiceUnavailable,
}
}

183
vendor/github.com/klauspost/compress/zlib/reader.go generated vendored Normal file
View File

@ -0,0 +1,183 @@
// Copyright 2009 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Package zlib implements reading and writing of zlib format compressed data,
as specified in RFC 1950.
The implementation provides filters that uncompress during reading
and compress during writing. For example, to write compressed data
to a buffer:
var b bytes.Buffer
w := zlib.NewWriter(&b)
w.Write([]byte("hello, world\n"))
w.Close()
and to read that data back:
r, err := zlib.NewReader(&b)
io.Copy(os.Stdout, r)
r.Close()
*/
package zlib
import (
"bufio"
"errors"
"hash"
"hash/adler32"
"io"
"github.com/klauspost/compress/flate"
)
const zlibDeflate = 8
var (
// ErrChecksum is returned when reading ZLIB data that has an invalid checksum.
ErrChecksum = errors.New("zlib: invalid checksum")
// ErrDictionary is returned when reading ZLIB data that has an invalid dictionary.
ErrDictionary = errors.New("zlib: invalid dictionary")
// ErrHeader is returned when reading ZLIB data that has an invalid header.
ErrHeader = errors.New("zlib: invalid header")
)
type reader struct {
r flate.Reader
decompressor io.ReadCloser
digest hash.Hash32
err error
scratch [4]byte
}
// Resetter resets a ReadCloser returned by NewReader or NewReaderDict to
// to switch to a new underlying Reader. This permits reusing a ReadCloser
// instead of allocating a new one.
type Resetter interface {
// Reset discards any buffered data and resets the Resetter as if it was
// newly initialized with the given reader.
Reset(r io.Reader, dict []byte) error
}
// NewReader creates a new ReadCloser.
// Reads from the returned ReadCloser read and decompress data from r.
// If r does not implement io.ByteReader, the decompressor may read more
// data than necessary from r.
// It is the caller's responsibility to call Close on the ReadCloser when done.
//
// The ReadCloser returned by NewReader also implements Resetter.
func NewReader(r io.Reader) (io.ReadCloser, error) {
return NewReaderDict(r, nil)
}
// NewReaderDict is like NewReader but uses a preset dictionary.
// NewReaderDict ignores the dictionary if the compressed data does not refer to it.
// If the compressed data refers to a different dictionary, NewReaderDict returns ErrDictionary.
//
// The ReadCloser returned by NewReaderDict also implements Resetter.
func NewReaderDict(r io.Reader, dict []byte) (io.ReadCloser, error) {
z := new(reader)
err := z.Reset(r, dict)
if err != nil {
return nil, err
}
return z, nil
}
func (z *reader) Read(p []byte) (int, error) {
if z.err != nil {
return 0, z.err
}
var n int
n, z.err = z.decompressor.Read(p)
z.digest.Write(p[0:n])
if z.err != io.EOF {
// In the normal case we return here.
return n, z.err
}
// Finished file; check checksum.
if _, err := io.ReadFull(z.r, z.scratch[0:4]); err != nil {
if err == io.EOF {
err = io.ErrUnexpectedEOF
}
z.err = err
return n, z.err
}
// ZLIB (RFC 1950) is big-endian, unlike GZIP (RFC 1952).
checksum := uint32(z.scratch[0])<<24 | uint32(z.scratch[1])<<16 | uint32(z.scratch[2])<<8 | uint32(z.scratch[3])
if checksum != z.digest.Sum32() {
z.err = ErrChecksum
return n, z.err
}
return n, io.EOF
}
// Calling Close does not close the wrapped io.Reader originally passed to NewReader.
// In order for the ZLIB checksum to be verified, the reader must be
// fully consumed until the io.EOF.
func (z *reader) Close() error {
if z.err != nil && z.err != io.EOF {
return z.err
}
z.err = z.decompressor.Close()
return z.err
}
func (z *reader) Reset(r io.Reader, dict []byte) error {
*z = reader{decompressor: z.decompressor, digest: z.digest}
if fr, ok := r.(flate.Reader); ok {
z.r = fr
} else {
z.r = bufio.NewReader(r)
}
// Read the header (RFC 1950 section 2.2.).
_, z.err = io.ReadFull(z.r, z.scratch[0:2])
if z.err != nil {
if z.err == io.EOF {
z.err = io.ErrUnexpectedEOF
}
return z.err
}
h := uint(z.scratch[0])<<8 | uint(z.scratch[1])
if (z.scratch[0]&0x0f != zlibDeflate) || (h%31 != 0) {
z.err = ErrHeader
return z.err
}
haveDict := z.scratch[1]&0x20 != 0
if haveDict {
_, z.err = io.ReadFull(z.r, z.scratch[0:4])
if z.err != nil {
if z.err == io.EOF {
z.err = io.ErrUnexpectedEOF
}
return z.err
}
checksum := uint32(z.scratch[0])<<24 | uint32(z.scratch[1])<<16 | uint32(z.scratch[2])<<8 | uint32(z.scratch[3])
if checksum != adler32.Checksum(dict) {
z.err = ErrDictionary
return z.err
}
}
if z.decompressor == nil {
if haveDict {
z.decompressor = flate.NewReaderDict(z.r, dict)
} else {
z.decompressor = flate.NewReader(z.r)
}
} else {
z.decompressor.(flate.Resetter).Reset(z.r, dict)
}
if z.digest != nil {
z.digest.Reset()
} else {
z.digest = adler32.New()
}
return nil
}

201
vendor/github.com/klauspost/compress/zlib/writer.go generated vendored Normal file
View File

@ -0,0 +1,201 @@
// Copyright 2009 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zlib
import (
"fmt"
"hash"
"hash/adler32"
"io"
"github.com/klauspost/compress/flate"
)
// These constants are copied from the flate package, so that code that imports
// "compress/zlib" does not also have to import "compress/flate".
const (
NoCompression = flate.NoCompression
BestSpeed = flate.BestSpeed
BestCompression = flate.BestCompression
DefaultCompression = flate.DefaultCompression
ConstantCompression = flate.ConstantCompression
HuffmanOnly = flate.HuffmanOnly
)
// A Writer takes data written to it and writes the compressed
// form of that data to an underlying writer (see NewWriter).
type Writer struct {
w io.Writer
level int
dict []byte
compressor *flate.Writer
digest hash.Hash32
err error
scratch [4]byte
wroteHeader bool
}
// NewWriter creates a new Writer.
// Writes to the returned Writer are compressed and written to w.
//
// It is the caller's responsibility to call Close on the WriteCloser when done.
// Writes may be buffered and not flushed until Close.
func NewWriter(w io.Writer) *Writer {
z, _ := NewWriterLevelDict(w, DefaultCompression, nil)
return z
}
// NewWriterLevel is like NewWriter but specifies the compression level instead
// of assuming DefaultCompression.
//
// The compression level can be DefaultCompression, NoCompression, HuffmanOnly
// or any integer value between BestSpeed and BestCompression inclusive.
// The error returned will be nil if the level is valid.
func NewWriterLevel(w io.Writer, level int) (*Writer, error) {
return NewWriterLevelDict(w, level, nil)
}
// NewWriterLevelDict is like NewWriterLevel but specifies a dictionary to
// compress with.
//
// The dictionary may be nil. If not, its contents should not be modified until
// the Writer is closed.
func NewWriterLevelDict(w io.Writer, level int, dict []byte) (*Writer, error) {
if level < HuffmanOnly || level > BestCompression {
return nil, fmt.Errorf("zlib: invalid compression level: %d", level)
}
return &Writer{
w: w,
level: level,
dict: dict,
}, nil
}
// Reset clears the state of the Writer z such that it is equivalent to its
// initial state from NewWriterLevel or NewWriterLevelDict, but instead writing
// to w.
func (z *Writer) Reset(w io.Writer) {
z.w = w
// z.level and z.dict left unchanged.
if z.compressor != nil {
z.compressor.Reset(w)
}
if z.digest != nil {
z.digest.Reset()
}
z.err = nil
z.scratch = [4]byte{}
z.wroteHeader = false
}
// writeHeader writes the ZLIB header.
func (z *Writer) writeHeader() (err error) {
z.wroteHeader = true
// ZLIB has a two-byte header (as documented in RFC 1950).
// The first four bits is the CINFO (compression info), which is 7 for the default deflate window size.
// The next four bits is the CM (compression method), which is 8 for deflate.
z.scratch[0] = 0x78
// The next two bits is the FLEVEL (compression level). The four values are:
// 0=fastest, 1=fast, 2=default, 3=best.
// The next bit, FDICT, is set if a dictionary is given.
// The final five FCHECK bits form a mod-31 checksum.
switch z.level {
case -2, 0, 1:
z.scratch[1] = 0 << 6
case 2, 3, 4, 5:
z.scratch[1] = 1 << 6
case 6, -1:
z.scratch[1] = 2 << 6
case 7, 8, 9:
z.scratch[1] = 3 << 6
default:
panic("unreachable")
}
if z.dict != nil {
z.scratch[1] |= 1 << 5
}
z.scratch[1] += uint8(31 - (uint16(z.scratch[0])<<8+uint16(z.scratch[1]))%31)
if _, err = z.w.Write(z.scratch[0:2]); err != nil {
return err
}
if z.dict != nil {
// The next four bytes are the Adler-32 checksum of the dictionary.
checksum := adler32.Checksum(z.dict)
z.scratch[0] = uint8(checksum >> 24)
z.scratch[1] = uint8(checksum >> 16)
z.scratch[2] = uint8(checksum >> 8)
z.scratch[3] = uint8(checksum >> 0)
if _, err = z.w.Write(z.scratch[0:4]); err != nil {
return err
}
}
if z.compressor == nil {
// Initialize deflater unless the Writer is being reused
// after a Reset call.
z.compressor, err = flate.NewWriterDict(z.w, z.level, z.dict)
if err != nil {
return err
}
z.digest = adler32.New()
}
return nil
}
// Write writes a compressed form of p to the underlying io.Writer. The
// compressed bytes are not necessarily flushed until the Writer is closed or
// explicitly flushed.
func (z *Writer) Write(p []byte) (n int, err error) {
if !z.wroteHeader {
z.err = z.writeHeader()
}
if z.err != nil {
return 0, z.err
}
if len(p) == 0 {
return 0, nil
}
n, err = z.compressor.Write(p)
if err != nil {
z.err = err
return
}
z.digest.Write(p)
return
}
// Flush flushes the Writer to its underlying io.Writer.
func (z *Writer) Flush() error {
if !z.wroteHeader {
z.err = z.writeHeader()
}
if z.err != nil {
return z.err
}
z.err = z.compressor.Flush()
return z.err
}
// Close closes the Writer, flushing any unwritten data to the underlying
// io.Writer, but does not close the underlying io.Writer.
func (z *Writer) Close() error {
if !z.wroteHeader {
z.err = z.writeHeader()
}
if z.err != nil {
return z.err
}
z.err = z.compressor.Close()
if z.err != nil {
return z.err
}
checksum := z.digest.Sum32()
// ZLIB (RFC 1950) is big-endian, unlike GZIP (RFC 1952).
z.scratch[0] = uint8(checksum >> 24)
z.scratch[1] = uint8(checksum >> 16)
z.scratch[2] = uint8(checksum >> 8)
z.scratch[3] = uint8(checksum >> 0)
_, z.err = z.w.Write(z.scratch[0:4])
return z.err
}

3
vendor/github.com/valyala/fasthttp/.gitignore generated vendored Normal file
View File

@ -0,0 +1,3 @@
tags
*.pprof
*.fasthttp.gz

55
vendor/github.com/valyala/fasthttp/.travis.yml generated vendored Normal file
View File

@ -0,0 +1,55 @@
language: go
# Docker is required for fuzzit regression tests
services:
- docker
dist: bionic
os:
- linux
- osx
go:
- tip
- 1.13.x
- 1.12.x
- 1.11.x
- 1.10.x
- 1.9.x
matrix:
allow_failures:
- tip
fast_finish: true
env:
global:
secure: "v/F0oI9zE9mcpEp4AVdHzSSHbe5ZFtH6B0i/BiUXKdQRQ10+JMPDOFRJQti7yxjMwltyd/QSFmR50Fl108sQYpo4xdlEXMHp2Y6OAN6crrp6PuHbLYgDWu3df/cH7/BqDyIq1uX8KZEeQssnygYN8hN4tpJCUg+NIb40Lm57Zsodt8DVjjyDWQQFDL7soNyAwGwQIqEyJsn+NUieXWEB1Qnt0xUtPIReuLlrwXR8wC1nLEjG9yz4ftDHHQdhVbO2b+xGWyaJ7QB5ixztaQP8Jnny6kSW9j6zEhJVuzdZ6d3xz23ibCbzSXBHdIUEI9u6ifQj8BYXr8fFS0FB3++IxgAYSs3ybZ+qEwuAxSBBm6YNW+3FrfDknVwTQscjKqnXPisjUqaRC9b31hke0tXzBq1488hE+wxMXeDM4LwWT5IMEO2gz0WGQXxmdVit72DIjCZxJkf1TvZZ0YH7Y//6wJTYYP9xulsy4gqu8CuFdWiF3fiGc3p5DTIS75nJ/Yy76Sa1pRPASKCujfLxtHE6Mt0XKvSolIXklYIzBkjN6vn80N6JIrqtqlimBGPW/Ec6+dwbmRe2AcOKRl4y7pZsGYhJhqdue1mucUYO/e2QeBZJGkqqG+zF5AW0v8x29BHvMwViAonc8o9eelkJ8khYzc/Qeq05pZnR/N/Pqfc+68k="
before_install:
- go get -t -v ./...
jobs:
include:
- stage: test
script:
# build test for supported platforms
- GOOS=linux go build
- GOOS=darwin go build
- GOOS=freebsd go build
- GOOS=windows go build
- GOARCH=386 go build
# run tests on a standard platform
- go test -v ./...
# run tests with the race detector as well
- go test -race -v ./...
- stage: fuzzit.dev
os:
- linux
go:
- 1.13
script:
- if [ "$TRAVIS_PULL_REQUEST" = "false" ]; then ./fuzzit.sh fuzzing; fi
- if [ "$TRAVIS_PULL_REQUEST" != "false" ]; then ./fuzzit.sh local-regression; fi

9
vendor/github.com/valyala/fasthttp/LICENSE generated vendored Normal file
View File

@ -0,0 +1,9 @@
The MIT License (MIT)
Copyright (c) 2015-present Aliaksandr Valialkin, VertaMedia, Kirill Danshin, Erik Dubbelboer, FastHTTP Authors
Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

585
vendor/github.com/valyala/fasthttp/README.md generated vendored Normal file
View File

@ -0,0 +1,585 @@
# fasthttp [![Build Status](https://travis-ci.org/valyala/fasthttp.svg?branch=master)](https://travis-ci.org/valyala/fasthttp?branch=master) [![GoDoc](https://godoc.org/github.com/valyala/fasthttp?status.svg)](http://godoc.org/github.com/valyala/fasthttp) [![fuzzit](https://app.fuzzit.dev/badge?org_id=fasthttp&branch=master)](https://fuzzit.dev) [![Go Report](https://goreportcard.com/badge/github.com/valyala/fasthttp)](https://goreportcard.com/report/github.com/valyala/fasthttp) [![Sourcegraph](https://sourcegraph.com/github.com/valyala/fasthttp/-/badge.svg)](https://sourcegraph.com/github.com/valyala/fasthttp?badge)
![FastHTTP  Fastest and reliable HTTP implementation in Go](https://github.com/fasthttp/docs-assets/raw/master/banner@0.5.png)
Fast HTTP implementation for Go.
Currently fasthttp is successfully used by [VertaMedia](https://vertamedia.com/)
in a production serving up to 200K rps from more than 1.5M concurrent keep-alive
connections per physical server.
[TechEmpower Benchmark round 18 results](https://www.techempower.com/benchmarks/#section=data-r18&hw=ph&test=plaintext)
[Server Benchmarks](#http-server-performance-comparison-with-nethttp)
[Client Benchmarks](#http-client-comparison-with-nethttp)
[Install](#install)
[Documentation](https://godoc.org/github.com/valyala/fasthttp)
[Examples from docs](https://godoc.org/github.com/valyala/fasthttp#pkg-examples)
[Code examples](examples)
[Awesome fasthttp tools](https://github.com/fasthttp)
[Switching from net/http to fasthttp](#switching-from-nethttp-to-fasthttp)
[Fasthttp best practices](#fasthttp-best-practices)
[Tricks with byte buffers](#tricks-with-byte-buffers)
[Related projects](#related-projects)
[FAQ](#faq)
# HTTP server performance comparison with [net/http](https://golang.org/pkg/net/http/)
In short, fasthttp server is up to 10 times faster than net/http.
Below are benchmark results.
*GOMAXPROCS=1*
net/http server:
```
$ GOMAXPROCS=1 go test -bench=NetHTTPServerGet -benchmem -benchtime=10s
BenchmarkNetHTTPServerGet1ReqPerConn 1000000 12052 ns/op 2297 B/op 29 allocs/op
BenchmarkNetHTTPServerGet2ReqPerConn 1000000 12278 ns/op 2327 B/op 24 allocs/op
BenchmarkNetHTTPServerGet10ReqPerConn 2000000 8903 ns/op 2112 B/op 19 allocs/op
BenchmarkNetHTTPServerGet10KReqPerConn 2000000 8451 ns/op 2058 B/op 18 allocs/op
BenchmarkNetHTTPServerGet1ReqPerConn10KClients 500000 26733 ns/op 3229 B/op 29 allocs/op
BenchmarkNetHTTPServerGet2ReqPerConn10KClients 1000000 23351 ns/op 3211 B/op 24 allocs/op
BenchmarkNetHTTPServerGet10ReqPerConn10KClients 1000000 13390 ns/op 2483 B/op 19 allocs/op
BenchmarkNetHTTPServerGet100ReqPerConn10KClients 1000000 13484 ns/op 2171 B/op 18 allocs/op
```
fasthttp server:
```
$ GOMAXPROCS=1 go test -bench=kServerGet -benchmem -benchtime=10s
BenchmarkServerGet1ReqPerConn 10000000 1559 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet2ReqPerConn 10000000 1248 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10ReqPerConn 20000000 797 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10KReqPerConn 20000000 716 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet1ReqPerConn10KClients 10000000 1974 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet2ReqPerConn10KClients 10000000 1352 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10ReqPerConn10KClients 20000000 789 ns/op 2 B/op 0 allocs/op
BenchmarkServerGet100ReqPerConn10KClients 20000000 604 ns/op 0 B/op 0 allocs/op
```
*GOMAXPROCS=4*
net/http server:
```
$ GOMAXPROCS=4 go test -bench=NetHTTPServerGet -benchmem -benchtime=10s
BenchmarkNetHTTPServerGet1ReqPerConn-4 3000000 4529 ns/op 2389 B/op 29 allocs/op
BenchmarkNetHTTPServerGet2ReqPerConn-4 5000000 3896 ns/op 2418 B/op 24 allocs/op
BenchmarkNetHTTPServerGet10ReqPerConn-4 5000000 3145 ns/op 2160 B/op 19 allocs/op
BenchmarkNetHTTPServerGet10KReqPerConn-4 5000000 3054 ns/op 2065 B/op 18 allocs/op
BenchmarkNetHTTPServerGet1ReqPerConn10KClients-4 1000000 10321 ns/op 3710 B/op 30 allocs/op
BenchmarkNetHTTPServerGet2ReqPerConn10KClients-4 2000000 7556 ns/op 3296 B/op 24 allocs/op
BenchmarkNetHTTPServerGet10ReqPerConn10KClients-4 5000000 3905 ns/op 2349 B/op 19 allocs/op
BenchmarkNetHTTPServerGet100ReqPerConn10KClients-4 5000000 3435 ns/op 2130 B/op 18 allocs/op
```
fasthttp server:
```
$ GOMAXPROCS=4 go test -bench=kServerGet -benchmem -benchtime=10s
BenchmarkServerGet1ReqPerConn-4 10000000 1141 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet2ReqPerConn-4 20000000 707 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10ReqPerConn-4 30000000 341 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10KReqPerConn-4 50000000 310 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet1ReqPerConn10KClients-4 10000000 1119 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet2ReqPerConn10KClients-4 20000000 644 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet10ReqPerConn10KClients-4 30000000 346 ns/op 0 B/op 0 allocs/op
BenchmarkServerGet100ReqPerConn10KClients-4 50000000 282 ns/op 0 B/op 0 allocs/op
```
# HTTP client comparison with net/http
In short, fasthttp client is up to 10 times faster than net/http.
Below are benchmark results.
*GOMAXPROCS=1*
net/http client:
```
$ GOMAXPROCS=1 go test -bench='HTTPClient(Do|GetEndToEnd)' -benchmem -benchtime=10s
BenchmarkNetHTTPClientDoFastServer 1000000 12567 ns/op 2616 B/op 35 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1TCP 200000 67030 ns/op 5028 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd10TCP 300000 51098 ns/op 5031 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd100TCP 300000 45096 ns/op 5026 B/op 55 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1Inmemory 500000 24779 ns/op 5035 B/op 57 allocs/op
BenchmarkNetHTTPClientGetEndToEnd10Inmemory 1000000 26425 ns/op 5035 B/op 57 allocs/op
BenchmarkNetHTTPClientGetEndToEnd100Inmemory 500000 28515 ns/op 5045 B/op 57 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1000Inmemory 500000 39511 ns/op 5096 B/op 56 allocs/op
```
fasthttp client:
```
$ GOMAXPROCS=1 go test -bench='kClient(Do|GetEndToEnd)' -benchmem -benchtime=10s
BenchmarkClientDoFastServer 20000000 865 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1TCP 1000000 18711 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd10TCP 1000000 14664 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd100TCP 1000000 14043 ns/op 1 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1Inmemory 5000000 3965 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd10Inmemory 3000000 4060 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd100Inmemory 5000000 3396 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1000Inmemory 5000000 3306 ns/op 2 B/op 0 allocs/op
```
*GOMAXPROCS=4*
net/http client:
```
$ GOMAXPROCS=4 go test -bench='HTTPClient(Do|GetEndToEnd)' -benchmem -benchtime=10s
BenchmarkNetHTTPClientDoFastServer-4 2000000 8774 ns/op 2619 B/op 35 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1TCP-4 500000 22951 ns/op 5047 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd10TCP-4 1000000 19182 ns/op 5037 B/op 55 allocs/op
BenchmarkNetHTTPClientGetEndToEnd100TCP-4 1000000 16535 ns/op 5031 B/op 55 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1Inmemory-4 1000000 14495 ns/op 5038 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd10Inmemory-4 1000000 10237 ns/op 5034 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd100Inmemory-4 1000000 10125 ns/op 5045 B/op 56 allocs/op
BenchmarkNetHTTPClientGetEndToEnd1000Inmemory-4 1000000 11132 ns/op 5136 B/op 56 allocs/op
```
fasthttp client:
```
$ GOMAXPROCS=4 go test -bench='kClient(Do|GetEndToEnd)' -benchmem -benchtime=10s
BenchmarkClientDoFastServer-4 50000000 397 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1TCP-4 2000000 7388 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd10TCP-4 2000000 6689 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd100TCP-4 3000000 4927 ns/op 1 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1Inmemory-4 10000000 1604 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd10Inmemory-4 10000000 1458 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd100Inmemory-4 10000000 1329 ns/op 0 B/op 0 allocs/op
BenchmarkClientGetEndToEnd1000Inmemory-4 10000000 1316 ns/op 5 B/op 0 allocs/op
```
# Install
```
go get -u github.com/valyala/fasthttp
```
# Switching from net/http to fasthttp
Unfortunately, fasthttp doesn't provide API identical to net/http.
See the [FAQ](#faq) for details.
There is [net/http -> fasthttp handler converter](https://godoc.org/github.com/valyala/fasthttp/fasthttpadaptor),
but it is better to write fasthttp request handlers by hand in order to use
all of the fasthttp advantages (especially high performance :) ).
Important points:
* Fasthttp works with [RequestHandler functions](https://godoc.org/github.com/valyala/fasthttp#RequestHandler)
instead of objects implementing [Handler interface](https://golang.org/pkg/net/http/#Handler).
Fortunately, it is easy to pass bound struct methods to fasthttp:
```go
type MyHandler struct {
foobar string
}
// request handler in net/http style, i.e. method bound to MyHandler struct.
func (h *MyHandler) HandleFastHTTP(ctx *fasthttp.RequestCtx) {
// notice that we may access MyHandler properties here - see h.foobar.
fmt.Fprintf(ctx, "Hello, world! Requested path is %q. Foobar is %q",
ctx.Path(), h.foobar)
}
// request handler in fasthttp style, i.e. just plain function.
func fastHTTPHandler(ctx *fasthttp.RequestCtx) {
fmt.Fprintf(ctx, "Hi there! RequestURI is %q", ctx.RequestURI())
}
// pass bound struct method to fasthttp
myHandler := &MyHandler{
foobar: "foobar",
}
fasthttp.ListenAndServe(":8080", myHandler.HandleFastHTTP)
// pass plain function to fasthttp
fasthttp.ListenAndServe(":8081", fastHTTPHandler)
```
* The [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler)
accepts only one argument - [RequestCtx](https://godoc.org/github.com/valyala/fasthttp#RequestCtx).
It contains all the functionality required for http request processing
and response writing. Below is an example of a simple request handler conversion
from net/http to fasthttp.
```go
// net/http request handler
requestHandler := func(w http.ResponseWriter, r *http.Request) {
switch r.URL.Path {
case "/foo":
fooHandler(w, r)
case "/bar":
barHandler(w, r)
default:
http.Error(w, "Unsupported path", http.StatusNotFound)
}
}
```
```go
// the corresponding fasthttp request handler
requestHandler := func(ctx *fasthttp.RequestCtx) {
switch string(ctx.Path()) {
case "/foo":
fooHandler(ctx)
case "/bar":
barHandler(ctx)
default:
ctx.Error("Unsupported path", fasthttp.StatusNotFound)
}
}
```
* Fasthttp allows setting response headers and writing response body
in an arbitrary order. There is no 'headers first, then body' restriction
like in net/http. The following code is valid for fasthttp:
```go
requestHandler := func(ctx *fasthttp.RequestCtx) {
// set some headers and status code first
ctx.SetContentType("foo/bar")
ctx.SetStatusCode(fasthttp.StatusOK)
// then write the first part of body
fmt.Fprintf(ctx, "this is the first part of body\n")
// then set more headers
ctx.Response.Header.Set("Foo-Bar", "baz")
// then write more body
fmt.Fprintf(ctx, "this is the second part of body\n")
// then override already written body
ctx.SetBody([]byte("this is completely new body contents"))
// then update status code
ctx.SetStatusCode(fasthttp.StatusNotFound)
// basically, anything may be updated many times before
// returning from RequestHandler.
//
// Unlike net/http fasthttp doesn't put response to the wire until
// returning from RequestHandler.
}
```
* Fasthttp doesn't provide [ServeMux](https://golang.org/pkg/net/http/#ServeMux),
but there are more powerful third-party routers and web frameworks
with fasthttp support:
* [fasthttp-routing](https://github.com/qiangxue/fasthttp-routing)
* [fasthttprouter](https://github.com/buaazp/fasthttprouter)
* [lu](https://github.com/vincentLiuxiang/lu)
* [atreugo](https://github.com/savsgio/atreugo)
Net/http code with simple ServeMux is trivially converted to fasthttp code:
```go
// net/http code
m := &http.ServeMux{}
m.HandleFunc("/foo", fooHandlerFunc)
m.HandleFunc("/bar", barHandlerFunc)
m.Handle("/baz", bazHandler)
http.ListenAndServe(":80", m)
```
```go
// the corresponding fasthttp code
m := func(ctx *fasthttp.RequestCtx) {
switch string(ctx.Path()) {
case "/foo":
fooHandlerFunc(ctx)
case "/bar":
barHandlerFunc(ctx)
case "/baz":
bazHandler.HandlerFunc(ctx)
default:
ctx.Error("not found", fasthttp.StatusNotFound)
}
}
fasthttp.ListenAndServe(":80", m)
```
* net/http -> fasthttp conversion table:
* All the pseudocode below assumes w, r and ctx have these types:
```go
var (
w http.ResponseWriter
r *http.Request
ctx *fasthttp.RequestCtx
)
```
* r.Body -> [ctx.PostBody()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.PostBody)
* r.URL.Path -> [ctx.Path()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Path)
* r.URL -> [ctx.URI()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.URI)
* r.Method -> [ctx.Method()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Method)
* r.Header -> [ctx.Request.Header](https://godoc.org/github.com/valyala/fasthttp#RequestHeader)
* r.Header.Get() -> [ctx.Request.Header.Peek()](https://godoc.org/github.com/valyala/fasthttp#RequestHeader.Peek)
* r.Host -> [ctx.Host()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Host)
* r.Form -> [ctx.QueryArgs()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.QueryArgs) +
[ctx.PostArgs()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.PostArgs)
* r.PostForm -> [ctx.PostArgs()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.PostArgs)
* r.FormValue() -> [ctx.FormValue()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.FormValue)
* r.FormFile() -> [ctx.FormFile()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.FormFile)
* r.MultipartForm -> [ctx.MultipartForm()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.MultipartForm)
* r.RemoteAddr -> [ctx.RemoteAddr()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.RemoteAddr)
* r.RequestURI -> [ctx.RequestURI()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.RequestURI)
* r.TLS -> [ctx.IsTLS()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.IsTLS)
* r.Cookie() -> [ctx.Request.Header.Cookie()](https://godoc.org/github.com/valyala/fasthttp#RequestHeader.Cookie)
* r.Referer() -> [ctx.Referer()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Referer)
* r.UserAgent() -> [ctx.UserAgent()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.UserAgent)
* w.Header() -> [ctx.Response.Header](https://godoc.org/github.com/valyala/fasthttp#ResponseHeader)
* w.Header().Set() -> [ctx.Response.Header.Set()](https://godoc.org/github.com/valyala/fasthttp#ResponseHeader.Set)
* w.Header().Set("Content-Type") -> [ctx.SetContentType()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.SetContentType)
* w.Header().Set("Set-Cookie") -> [ctx.Response.Header.SetCookie()](https://godoc.org/github.com/valyala/fasthttp#ResponseHeader.SetCookie)
* w.Write() -> [ctx.Write()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Write),
[ctx.SetBody()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.SetBody),
[ctx.SetBodyStream()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.SetBodyStream),
[ctx.SetBodyStreamWriter()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.SetBodyStreamWriter)
* w.WriteHeader() -> [ctx.SetStatusCode()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.SetStatusCode)
* w.(http.Hijacker).Hijack() -> [ctx.Hijack()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Hijack)
* http.Error() -> [ctx.Error()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Error)
* http.FileServer() -> [fasthttp.FSHandler()](https://godoc.org/github.com/valyala/fasthttp#FSHandler),
[fasthttp.FS](https://godoc.org/github.com/valyala/fasthttp#FS)
* http.ServeFile() -> [fasthttp.ServeFile()](https://godoc.org/github.com/valyala/fasthttp#ServeFile)
* http.Redirect() -> [ctx.Redirect()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Redirect)
* http.NotFound() -> [ctx.NotFound()](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.NotFound)
* http.StripPrefix() -> [fasthttp.PathRewriteFunc](https://godoc.org/github.com/valyala/fasthttp#PathRewriteFunc)
* *VERY IMPORTANT!* Fasthttp disallows holding references
to [RequestCtx](https://godoc.org/github.com/valyala/fasthttp#RequestCtx) or to its'
members after returning from [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler).
Otherwise [data races](http://blog.golang.org/race-detector) are inevitable.
Carefully inspect all the net/http request handlers converted to fasthttp whether
they retain references to RequestCtx or to its' members after returning.
RequestCtx provides the following _band aids_ for this case:
* Wrap RequestHandler into [TimeoutHandler](https://godoc.org/github.com/valyala/fasthttp#TimeoutHandler).
* Call [TimeoutError](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.TimeoutError)
before returning from RequestHandler if there are references to RequestCtx or to its' members.
See [the example](https://godoc.org/github.com/valyala/fasthttp#example-RequestCtx-TimeoutError)
for more details.
Use this brilliant tool - [race detector](http://blog.golang.org/race-detector) -
for detecting and eliminating data races in your program. If you detected
data race related to fasthttp in your program, then there is high probability
you forgot calling [TimeoutError](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.TimeoutError)
before returning from [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler).
* Blind switching from net/http to fasthttp won't give you performance boost.
While fasthttp is optimized for speed, its' performance may be easily saturated
by slow [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler).
So [profile](http://blog.golang.org/profiling-go-programs) and optimize your
code after switching to fasthttp. For instance, use [quicktemplate](https://github.com/valyala/quicktemplate)
instead of [html/template](https://golang.org/pkg/html/template/).
* See also [fasthttputil](https://godoc.org/github.com/valyala/fasthttp/fasthttputil),
[fasthttpadaptor](https://godoc.org/github.com/valyala/fasthttp/fasthttpadaptor) and
[expvarhandler](https://godoc.org/github.com/valyala/fasthttp/expvarhandler).
# Performance optimization tips for multi-core systems
* Use [reuseport](https://godoc.org/github.com/valyala/fasthttp/reuseport) listener.
* Run a separate server instance per CPU core with GOMAXPROCS=1.
* Pin each server instance to a separate CPU core using [taskset](http://linux.die.net/man/1/taskset).
* Ensure the interrupts of multiqueue network card are evenly distributed between CPU cores.
See [this article](https://blog.cloudflare.com/how-to-achieve-low-latency/) for details.
* Use Go 1.13 as it provides some considerable performance improvements.
# Fasthttp best practices
* Do not allocate objects and `[]byte` buffers - just reuse them as much
as possible. Fasthttp API design encourages this.
* [sync.Pool](https://golang.org/pkg/sync/#Pool) is your best friend.
* [Profile your program](http://blog.golang.org/profiling-go-programs)
in production.
`go tool pprof --alloc_objects your-program mem.pprof` usually gives better
insights for optimization opportunities than `go tool pprof your-program cpu.pprof`.
* Write [tests and benchmarks](https://golang.org/pkg/testing/) for hot paths.
* Avoid conversion between `[]byte` and `string`, since this may result in memory
allocation+copy. Fasthttp API provides functions for both `[]byte` and `string` -
use these functions instead of converting manually between `[]byte` and `string`.
There are some exceptions - see [this wiki page](https://github.com/golang/go/wiki/CompilerOptimizations#string-and-byte)
for more details.
* Verify your tests and production code under
[race detector](https://golang.org/doc/articles/race_detector.html) on a regular basis.
* Prefer [quicktemplate](https://github.com/valyala/quicktemplate) instead of
[html/template](https://golang.org/pkg/html/template/) in your webserver.
# Tricks with `[]byte` buffers
The following tricks are used by fasthttp. Use them in your code too.
* Standard Go functions accept nil buffers
```go
var (
// both buffers are uninitialized
dst []byte
src []byte
)
dst = append(dst, src...) // is legal if dst is nil and/or src is nil
copy(dst, src) // is legal if dst is nil and/or src is nil
(string(src) == "") // is true if src is nil
(len(src) == 0) // is true if src is nil
src = src[:0] // works like a charm with nil src
// this for loop doesn't panic if src is nil
for i, ch := range src {
doSomething(i, ch)
}
```
So throw away nil checks for `[]byte` buffers from you code. For example,
```go
srcLen := 0
if src != nil {
srcLen = len(src)
}
```
becomes
```go
srcLen := len(src)
```
* String may be appended to `[]byte` buffer with `append`
```go
dst = append(dst, "foobar"...)
```
* `[]byte` buffer may be extended to its' capacity.
```go
buf := make([]byte, 100)
a := buf[:10] // len(a) == 10, cap(a) == 100.
b := a[:100] // is valid, since cap(a) == 100.
```
* All fasthttp functions accept nil `[]byte` buffer
```go
statusCode, body, err := fasthttp.Get(nil, "http://google.com/")
uintBuf := fasthttp.AppendUint(nil, 1234)
```
# Related projects
* [fasthttp](https://github.com/fasthttp) - various useful
helpers for projects based on fasthttp.
* [fasthttp-routing](https://github.com/qiangxue/fasthttp-routing) - fast and
powerful routing package for fasthttp servers.
* [fasthttprouter](https://github.com/buaazp/fasthttprouter) - a high
performance fasthttp request router that scales well.
* [gramework](https://github.com/gramework/gramework) - a web framework made by one of fasthttp maintainers
* [lu](https://github.com/vincentLiuxiang/lu) - a high performance
go middleware web framework which is based on fasthttp.
* [websocket](https://github.com/fasthttp/websocket) - Gorilla-based
websocket implementation for fasthttp.
* [fasthttpsession](https://github.com/phachon/fasthttpsession) - a fast and powerful session package for fasthttp servers.
* [atreugo](https://github.com/savsgio/atreugo) - High performance and extensible micro web framework with zero memory allocations in hot paths.
* [kratgo](https://github.com/savsgio/kratgo) - Simple, lightweight and ultra-fast HTTP Cache to speed up your websites.
* [kit-plugins](https://github.com/wencan/kit-plugins/tree/master/transport/fasthttp) - go-kit transport implementation for fasthttp.
# FAQ
* *Why creating yet another http package instead of optimizing net/http?*
Because net/http API limits many optimization opportunities.
For example:
* net/http Request object lifetime isn't limited by request handler execution
time. So the server must create a new request object per each request instead
of reusing existing objects like fasthttp does.
* net/http headers are stored in a `map[string][]string`. So the server
must parse all the headers, convert them from `[]byte` to `string` and put
them into the map before calling user-provided request handler.
This all requires unnecessary memory allocations avoided by fasthttp.
* net/http client API requires creating a new response object per each request.
* *Why fasthttp API is incompatible with net/http?*
Because net/http API limits many optimization opportunities. See the answer
above for more details. Also certain net/http API parts are suboptimal
for use:
* Compare [net/http connection hijacking](https://golang.org/pkg/net/http/#Hijacker)
to [fasthttp connection hijacking](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Hijack).
* Compare [net/http Request.Body reading](https://golang.org/pkg/net/http/#Request)
to [fasthttp request body reading](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.PostBody).
* *Why fasthttp doesn't support HTTP/2.0 and WebSockets?*
[HTTP/2.0 support](https://github.com/fasthttp/http2) is in progress. [WebSockets](https://github.com/fasthttp/websockets) has been done already.
Third parties also may use [RequestCtx.Hijack](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.Hijack)
for implementing these goodies.
* *Are there known net/http advantages comparing to fasthttp?*
Yes:
* net/http supports [HTTP/2.0 starting from go1.6](https://http2.golang.org/).
* net/http API is stable, while fasthttp API constantly evolves.
* net/http handles more HTTP corner cases.
* net/http should contain less bugs, since it is used and tested by much
wider audience.
* net/http works on Go older than 1.5.
* *Why fasthttp API prefers returning `[]byte` instead of `string`?*
Because `[]byte` to `string` conversion isn't free - it requires memory
allocation and copy. Feel free wrapping returned `[]byte` result into
`string()` if you prefer working with strings instead of byte slices.
But be aware that this has non-zero overhead.
* *Which GO versions are supported by fasthttp?*
Go1.5+. Older versions won't be supported, since their standard package
[miss useful functions](https://github.com/valyala/fasthttp/issues/5).
**NOTE**: Go 1.9.7 is the oldest tested version. We recommend you to update as soon as you can. As of 1.11.3 we will drop 1.9.x support.
* *Please provide real benchmark data and server information*
See [this issue](https://github.com/valyala/fasthttp/issues/4).
* *Are there plans to add request routing to fasthttp?*
There are no plans to add request routing into fasthttp.
Use third-party routers and web frameworks with fasthttp support:
* [fasthttp-routing](https://github.com/qiangxue/fasthttp-routing)
* [fasthttprouter](https://github.com/buaazp/fasthttprouter)
* [gramework](https://github.com/gramework/gramework)
* [lu](https://github.com/vincentLiuxiang/lu)
* [atreugo](https://github.com/savsgio/atreugo)
See also [this issue](https://github.com/valyala/fasthttp/issues/9) for more info.
* *I detected data race in fasthttp!*
Cool! [File a bug](https://github.com/valyala/fasthttp/issues/new). But before
doing this check the following in your code:
* Make sure there are no references to [RequestCtx](https://godoc.org/github.com/valyala/fasthttp#RequestCtx)
or to its' members after returning from [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler).
* Make sure you call [TimeoutError](https://godoc.org/github.com/valyala/fasthttp#RequestCtx.TimeoutError)
before returning from [RequestHandler](https://godoc.org/github.com/valyala/fasthttp#RequestHandler)
if there are references to [RequestCtx](https://godoc.org/github.com/valyala/fasthttp#RequestCtx)
or to its' members, which may be accessed by other goroutines.
* *I didn't find an answer for my question here*
Try exploring [these questions](https://github.com/valyala/fasthttp/issues?q=label%3Aquestion).

115
vendor/github.com/valyala/fasthttp/SECURITY.md generated vendored Normal file
View File

@ -0,0 +1,115 @@
### TL;DR
We use a simplified version of [Golang Security Policy](https://golang.org/security).
For example, for now we skip CVE assignment.
### Reporting a Security Bug
Please report to us any issues you find. This document explains how to do that and what to expect in return.
All security bugs in our releases should be reported by email to oss-security@highload.solutions.
This mail is delivered to a small security team.
Your email will be acknowledged within 24 hours, and you'll receive a more detailed response
to your email within 72 hours indicating the next steps in handling your report.
For critical problems, you can encrypt your report using our PGP key (listed below).
Please use a descriptive subject line for your report email.
After the initial reply to your report, the security team will
endeavor to keep you informed of the progress being made towards a fix and full announcement.
These updates will be sent at least every five days.
In reality, this is more likely to be every 24-48 hours.
If you have not received a reply to your email within 48 hours or you have not heard from the security
team for the past five days please contact us by email to developers@highload.solutions or by Telegram message
to [our support](https://t.me/highload_support).
Please note that developers@highload.solutions list includes all developers, who may be outside our opensource security team.
When escalating on this list, please do not disclose the details of the issue.
Simply state that you're trying to reach a member of the security team.
### Flagging Existing Issues as Security-related
If you believe that an existing issue is security-related, we ask that you send an email to oss-security@highload.solutions.
The email should include the issue ID and a short description of why it should be handled according to this security policy.
### Disclosure Process
Our project uses the following disclosure process:
- Once the security report is received it is assigned a primary handler. This person coordinates the fix and release process.
- The issue is confirmed and a list of affected software is determined.
- Code is audited to find any potential similar problems.
- Fixes are prepared for the two most recent major releases and the head/master revision. These fixes are not yet committed to the public repository.
- To notify users, a new issue without security details is submitted to our GitHub repository.
- Three working days following this notification, the fixes are applied to the public repository and a new release is issued.
- On the date that the fixes are applied, announcement is published in the issue.
This process can take some time, especially when coordination is required with maintainers of other projects.
Every effort will be made to handle the bug in as timely a manner as possible, however it's important that we follow
the process described above to ensure that disclosures are handled consistently.
### Receiving Security Updates
The best way to receive security announcements is to subscribe ("Watch") to our repository.
Any GitHub issues pertaining to a security issue will be prefixed with [security].
### Comments on This Policy
If you have any suggestions to improve this policy, please send an email to oss-security@highload.solutions for discussion.
### PGP Key for oss-security@highload.solutions
We accept PGP-encrypted email, but the majority of the security team are not regular PGP users
so it's somewhat inconvenient. Please only use PGP for critical security reports.
```
-----BEGIN PGP PUBLIC KEY BLOCK-----
mQINBFzdjYUBEACa3YN+QVSlnXofUjxr+YrmIaF+da0IUq+TRM4aqUXALsemEdGh
yIl7Z6qOOy1d2kPe6t//H9l/92lJ1X7i6aEBK4n/pnPZkwbpy9gGpebgvTZFvcbe
mFhF6k1FM35D8TxneJSjizPyGhJPqcr5qccqf8R64TlQx5Ud1JqT2l8P1C5N7gNS
lEYXq1h4zBCvTWk1wdeLRRPx7Bn6xrgmyu/k61dLoJDvpvWNATVFDA67oTrPgzTW
xtLbbk/xm0mK4a8zMzIpNyz1WkaJW9+4HFXaL+yKlsx7iHe2O7VlGoqS0kdeQup4
1HIw/P7yc0jBlNMLUzpuA6ElYUwESWsnCI71YY1x4rKgI+GqH1mWwgn7tteuXQtb
Zj0vEdjK3IKIOSbzbzAvSbDt8F1+o7EMtdy1eUysjKSQgFkDlT6JRmYvEup5/IoG
iknh/InQq9RmGFKii6pXWWoltC0ebfCwYOXvymyDdr/hYDqJeHS9Tenpy86Doaaf
HGf5nIFAMB2G5ctNpBwzNXR2MAWkeHQgdr5a1xmog0hS125usjnUTet3QeCyo4kd
gVouoOroMcqFFUXdYaMH4c3KWz0afhTmIaAsFFOv/eMdadVA4QyExTJf3TAoQ+kH
lKDlbOAIxEZWRPDFxMRixaVPQC+VxhBcaQ+yNoaUkM0V2m8u8sDBpzi1OQARAQAB
tDxPU1MgU2VjdXJpdHksIEhpZ2hsb2FkIExURCA8b3NzLXNlY3VyaXR5QGhpZ2hs
b2FkLnNvbHV0aW9ucz6JAlQEEwEIAD4WIQRljYp380uKq2g8TeqsQcvu+Qp2TAUC
XN2NhQIbAwUJB4YfgAULCQgHAgYVCgkICwIEFgIDAQIeAQIXgAAKCRCsQcvu+Qp2
TKmED/96YoQoOjD28blFFrigvAsiNcNNZoX9I0dX1lNpD83fBJf+/9i+x4jqUnI5
5XK/DFTDbhpw8kQBpxS9eEuIYnuo0RdLLp1ctNWTlpwfyHn92mGddl/uBdYHUuUk
cjhIQcFaCcWRY+EpamDlv1wmZ83IwBr8Hu5FS+/Msyw1TBvtTRVKW1KoGYMYoXLk
BzIglRPwn821B6s4BvK/RJnZkrmHMBZBfYMf+iSMSYd2yPmfT8wbcAjgjLfQa28U
gbt4u9xslgKjuM83IqwFfEXBnm7su3OouGWqc+62mQTsbnK65zRFnx6GXRXC1BAi
6m9Tm1PU0IiINz66ainquspkXYeHjd9hTwfR3BdFnzBTRRM01cKMFabWbLj8j0p8
fF4g9cxEdiLrzEF7Yz4WY0mI4Cpw4eJZfsHMc07Jn7QxfJhIoq+rqBOtEmTjnxMh
aWeykoXMHlZN4K0ZrAytozVH1D4bugWA9Zuzi9U3F9hrVVABm11yyhd2iSqI6/FR
GcCFOCBW1kEJbzoEguub+BV8LDi8ldljHalvur5k/VFhoDBxniYNsKmiCLVCmDWs
/nF84hCReAOJt0vDGwqHe3E2BFFPbKwdJLRNkjxBY0c/pvaV+JxbWQmaxDZNeIFV
hFcVGp48HNY3qLWZdsQIfT9m1masJFLVuq8Wx7bYs8Et5eFnH7kCDQRc3Y2FARAA
2DJWAxABydyIdCxgFNdqnYyWS46vh2DmLmRMqgasNlD0ozG4S9bszBsgnUI2Xs06
J76kFRh8MMHcu9I4lUKCQzfrA4uHkiOK5wvNCaWP+C6JUYNHsqPwk/ILO3gtQ/Ws
LLf/PW3rJZVOZB+WY8iaYc20l5vukTaVw4qbEi9dtLkJvVpNHt//+jayXU6s3ew1
2X5xdwyAZxaxlnzFaY/Xo/qR+bZhVFC0T9pAECnHv9TVhFGp0JE9ipPGnro5xTIS
LttdAkzv4AuSVTIgWgTkh8nN8t7STJqfPEv0I12nmmYHMUyTYOurkfskF3jY2x6x
8l02NQ4d5KdC3ReV1j51swrGcZCwsWNp51jnEXKwo+B0NM5OmoRrNJgF2iDgLehs
hP00ljU7cB8/1/7kdHZStYaUHICFOFqHzg415FlYm+jpY0nJp/b9BAO0d0/WYnEe
Xjihw8EVBAqzEt4kay1BQonZAypeYnGBJr7vNvdiP+mnRwly5qZSGiInxGvtZZFt
zL1E3osiF+muQxFcM63BeGdJeYXy+MoczkWa4WNggfcHlGAZkMYiv28zpr4PfrK9
mvj4Nu8s71PE9pPpBoZcNDf9v1sHuu96jDSITsPx5YMvvKZWhzJXFKzk6YgAsNH/
MF0G+/qmKJZpCdvtHKpYM1uHX85H81CwWJFfBPthyD8AEQEAAYkCPAQYAQgAJhYh
BGWNinfzS4qraDxN6qxBy+75CnZMBQJc3Y2FAhsMBQkHhh+AAAoJEKxBy+75CnZM
Rn8P/RyL1bhU4Q4WpvmlkepCAwNA0G3QvnKcSZNHEPE5h7H3IyrA/qy16A9eOsgm
sthsHYlo5A5lRIy4wPHkFCClMrMHdKuoS72//qgw+oOrBcwb7Te+Nas+ewhaJ7N9
vAX06vDH9bLl52CPbtats5+eBpePgP3HDPxd7CWHxq9bzJTbzqsTkN7JvoovR2dP
itPJDij7QYLYVEM1t7QxUVpVwAjDi/kCtC9ts5L+V0snF2n3bHZvu04EXdpvxOQI
pG/7Q+/WoI8NU6Bb/FA3tJGYIhSwI3SY+5XV/TAZttZaYSh2SD8vhc+eo+gW9sAN
xa+VESBQCht9+tKIwEwHs1efoRgFdbwwJ2c+33+XydQ6yjdXoX1mn2uyCr82jorZ
xTzbkY04zr7oZ+0fLpouOFg/mrSL4w2bWEhdHuyoVthLBjnRme0wXCaS3g3mYdLG
nSUkogOGOOvvvBtoq/vfx0Eu79piUtw5D8yQSrxLDuz8GxCrVRZ0tYIHb26aTE9G
cDsW/Lg5PjcY/LgVNEWOxDQDFVurlImnlVJFb3q+NrWvPbgeIEWwJDCay/z25SEH
k3bSOXLp8YGRnlkWUmoeL4g/CCK52iAAlfscZNoKMILhBnbCoD657jpa5GQKJj/U
Q8kjgr7kwV/RSosNV9HCPj30mVyiCQ1xg+ZLzMKXVCuBWd+G
=lnt2
-----END PGP PUBLIC KEY BLOCK-----
```

4
vendor/github.com/valyala/fasthttp/TODO generated vendored Normal file
View File

@ -0,0 +1,4 @@
- SessionClient with referer and cookies support.
- ProxyHandler similar to FSHandler.
- WebSockets. See https://tools.ietf.org/html/rfc6455 .
- HTTP/2.0. See https://tools.ietf.org/html/rfc7540 .

588
vendor/github.com/valyala/fasthttp/args.go generated vendored Normal file
View File

@ -0,0 +1,588 @@
package fasthttp
import (
"bytes"
"errors"
"io"
"sort"
"sync"
"github.com/valyala/bytebufferpool"
)
const (
argsNoValue = true
argsHasValue = false
)
// AcquireArgs returns an empty Args object from the pool.
//
// The returned Args may be returned to the pool with ReleaseArgs
// when no longer needed. This allows reducing GC load.
func AcquireArgs() *Args {
return argsPool.Get().(*Args)
}
// ReleaseArgs returns the object acquired via AcquireArgs to the pool.
//
// Do not access the released Args object, otherwise data races may occur.
func ReleaseArgs(a *Args) {
a.Reset()
argsPool.Put(a)
}
var argsPool = &sync.Pool{
New: func() interface{} {
return &Args{}
},
}
// Args represents query arguments.
//
// It is forbidden copying Args instances. Create new instances instead
// and use CopyTo().
//
// Args instance MUST NOT be used from concurrently running goroutines.
type Args struct {
noCopy noCopy //nolint:unused,structcheck
args []argsKV
buf []byte
}
type argsKV struct {
key []byte
value []byte
noValue bool
}
// Reset clears query args.
func (a *Args) Reset() {
a.args = a.args[:0]
}
// CopyTo copies all args to dst.
func (a *Args) CopyTo(dst *Args) {
dst.Reset()
dst.args = copyArgs(dst.args, a.args)
}
// VisitAll calls f for each existing arg.
//
// f must not retain references to key and value after returning.
// Make key and/or value copies if you need storing them after returning.
func (a *Args) VisitAll(f func(key, value []byte)) {
visitArgs(a.args, f)
}
// Len returns the number of query args.
func (a *Args) Len() int {
return len(a.args)
}
// Parse parses the given string containing query args.
func (a *Args) Parse(s string) {
a.buf = append(a.buf[:0], s...)
a.ParseBytes(a.buf)
}
// ParseBytes parses the given b containing query args.
func (a *Args) ParseBytes(b []byte) {
a.Reset()
var s argsScanner
s.b = b
var kv *argsKV
a.args, kv = allocArg(a.args)
for s.next(kv) {
if len(kv.key) > 0 || len(kv.value) > 0 {
a.args, kv = allocArg(a.args)
}
}
a.args = releaseArg(a.args)
}
// String returns string representation of query args.
func (a *Args) String() string {
return string(a.QueryString())
}
// QueryString returns query string for the args.
//
// The returned value is valid until the next call to Args methods.
func (a *Args) QueryString() []byte {
a.buf = a.AppendBytes(a.buf[:0])
return a.buf
}
// Sort sorts Args by key and then value using 'f' as comparison function.
//
// For example args.Sort(bytes.Compare)
func (a *Args) Sort(f func(x, y []byte) int) {
sort.SliceStable(a.args, func(i, j int) bool {
n := f(a.args[i].key, a.args[j].key)
if n == 0 {
return f(a.args[i].value, a.args[j].value) == -1
}
return n == -1
})
}
// AppendBytes appends query string to dst and returns the extended dst.
func (a *Args) AppendBytes(dst []byte) []byte {
for i, n := 0, len(a.args); i < n; i++ {
kv := &a.args[i]
dst = AppendQuotedArg(dst, kv.key)
if !kv.noValue {
dst = append(dst, '=')
if len(kv.value) > 0 {
dst = AppendQuotedArg(dst, kv.value)
}
}
if i+1 < n {
dst = append(dst, '&')
}
}
return dst
}
// WriteTo writes query string to w.
//
// WriteTo implements io.WriterTo interface.
func (a *Args) WriteTo(w io.Writer) (int64, error) {
n, err := w.Write(a.QueryString())
return int64(n), err
}
// Del deletes argument with the given key from query args.
func (a *Args) Del(key string) {
a.args = delAllArgs(a.args, key)
}
// DelBytes deletes argument with the given key from query args.
func (a *Args) DelBytes(key []byte) {
a.args = delAllArgs(a.args, b2s(key))
}
// Add adds 'key=value' argument.
//
// Multiple values for the same key may be added.
func (a *Args) Add(key, value string) {
a.args = appendArg(a.args, key, value, argsHasValue)
}
// AddBytesK adds 'key=value' argument.
//
// Multiple values for the same key may be added.
func (a *Args) AddBytesK(key []byte, value string) {
a.args = appendArg(a.args, b2s(key), value, argsHasValue)
}
// AddBytesV adds 'key=value' argument.
//
// Multiple values for the same key may be added.
func (a *Args) AddBytesV(key string, value []byte) {
a.args = appendArg(a.args, key, b2s(value), argsHasValue)
}
// AddBytesKV adds 'key=value' argument.
//
// Multiple values for the same key may be added.
func (a *Args) AddBytesKV(key, value []byte) {
a.args = appendArg(a.args, b2s(key), b2s(value), argsHasValue)
}
// AddNoValue adds only 'key' as argument without the '='.
//
// Multiple values for the same key may be added.
func (a *Args) AddNoValue(key string) {
a.args = appendArg(a.args, key, "", argsNoValue)
}
// AddBytesKNoValue adds only 'key' as argument without the '='.
//
// Multiple values for the same key may be added.
func (a *Args) AddBytesKNoValue(key []byte) {
a.args = appendArg(a.args, b2s(key), "", argsNoValue)
}
// Set sets 'key=value' argument.
func (a *Args) Set(key, value string) {
a.args = setArg(a.args, key, value, argsHasValue)
}
// SetBytesK sets 'key=value' argument.
func (a *Args) SetBytesK(key []byte, value string) {
a.args = setArg(a.args, b2s(key), value, argsHasValue)
}
// SetBytesV sets 'key=value' argument.
func (a *Args) SetBytesV(key string, value []byte) {
a.args = setArg(a.args, key, b2s(value), argsHasValue)
}
// SetBytesKV sets 'key=value' argument.
func (a *Args) SetBytesKV(key, value []byte) {
a.args = setArgBytes(a.args, key, value, argsHasValue)
}
// SetNoValue sets only 'key' as argument without the '='.
//
// Only key in argumemt, like key1&key2
func (a *Args) SetNoValue(key string) {
a.args = setArg(a.args, key, "", argsNoValue)
}
// SetBytesKNoValue sets 'key' argument.
func (a *Args) SetBytesKNoValue(key []byte) {
a.args = setArg(a.args, b2s(key), "", argsNoValue)
}
// Peek returns query arg value for the given key.
//
// Returned value is valid until the next Args call.
func (a *Args) Peek(key string) []byte {
return peekArgStr(a.args, key)
}
// PeekBytes returns query arg value for the given key.
//
// Returned value is valid until the next Args call.
func (a *Args) PeekBytes(key []byte) []byte {
return peekArgBytes(a.args, key)
}
// PeekMulti returns all the arg values for the given key.
func (a *Args) PeekMulti(key string) [][]byte {
var values [][]byte
a.VisitAll(func(k, v []byte) {
if string(k) == key {
values = append(values, v)
}
})
return values
}
// PeekMultiBytes returns all the arg values for the given key.
func (a *Args) PeekMultiBytes(key []byte) [][]byte {
return a.PeekMulti(b2s(key))
}
// Has returns true if the given key exists in Args.
func (a *Args) Has(key string) bool {
return hasArg(a.args, key)
}
// HasBytes returns true if the given key exists in Args.
func (a *Args) HasBytes(key []byte) bool {
return hasArg(a.args, b2s(key))
}
// ErrNoArgValue is returned when Args value with the given key is missing.
var ErrNoArgValue = errors.New("no Args value for the given key")
// GetUint returns uint value for the given key.
func (a *Args) GetUint(key string) (int, error) {
value := a.Peek(key)
if len(value) == 0 {
return -1, ErrNoArgValue
}
return ParseUint(value)
}
// SetUint sets uint value for the given key.
func (a *Args) SetUint(key string, value int) {
bb := bytebufferpool.Get()
bb.B = AppendUint(bb.B[:0], value)
a.SetBytesV(key, bb.B)
bytebufferpool.Put(bb)
}
// SetUintBytes sets uint value for the given key.
func (a *Args) SetUintBytes(key []byte, value int) {
a.SetUint(b2s(key), value)
}
// GetUintOrZero returns uint value for the given key.
//
// Zero (0) is returned on error.
func (a *Args) GetUintOrZero(key string) int {
n, err := a.GetUint(key)
if err != nil {
n = 0
}
return n
}
// GetUfloat returns ufloat value for the given key.
func (a *Args) GetUfloat(key string) (float64, error) {
value := a.Peek(key)
if len(value) == 0 {
return -1, ErrNoArgValue
}
return ParseUfloat(value)
}
// GetUfloatOrZero returns ufloat value for the given key.
//
// Zero (0) is returned on error.
func (a *Args) GetUfloatOrZero(key string) float64 {
f, err := a.GetUfloat(key)
if err != nil {
f = 0
}
return f
}
// GetBool returns boolean value for the given key.
//
// true is returned for "1", "t", "T", "true", "TRUE", "True", "y", "yes", "Y", "YES", "Yes",
// otherwise false is returned.
func (a *Args) GetBool(key string) bool {
switch b2s(a.Peek(key)) {
// Support the same true cases as strconv.ParseBool
// See: https://github.com/golang/go/blob/4e1b11e2c9bdb0ddea1141eed487be1a626ff5be/src/strconv/atob.go#L12
// and Y and Yes versions.
case "1", "t", "T", "true", "TRUE", "True", "y", "yes", "Y", "YES", "Yes":
return true
default:
return false
}
}
func visitArgs(args []argsKV, f func(k, v []byte)) {
for i, n := 0, len(args); i < n; i++ {
kv := &args[i]
f(kv.key, kv.value)
}
}
func copyArgs(dst, src []argsKV) []argsKV {
if cap(dst) < len(src) {
tmp := make([]argsKV, len(src))
copy(tmp, dst)
dst = tmp
}
n := len(src)
dst = dst[:n]
for i := 0; i < n; i++ {
dstKV := &dst[i]
srcKV := &src[i]
dstKV.key = append(dstKV.key[:0], srcKV.key...)
if srcKV.noValue {
dstKV.value = dstKV.value[:0]
} else {
dstKV.value = append(dstKV.value[:0], srcKV.value...)
}
dstKV.noValue = srcKV.noValue
}
return dst
}
func delAllArgsBytes(args []argsKV, key []byte) []argsKV {
return delAllArgs(args, b2s(key))
}
func delAllArgs(args []argsKV, key string) []argsKV {
for i, n := 0, len(args); i < n; i++ {
kv := &args[i]
if key == string(kv.key) {
tmp := *kv
copy(args[i:], args[i+1:])
n--
args[n] = tmp
args = args[:n]
}
}
return args
}
func setArgBytes(h []argsKV, key, value []byte, noValue bool) []argsKV {
return setArg(h, b2s(key), b2s(value), noValue)
}
func setArg(h []argsKV, key, value string, noValue bool) []argsKV {
n := len(h)
for i := 0; i < n; i++ {
kv := &h[i]
if key == string(kv.key) {
if noValue {
kv.value = kv.value[:0]
} else {
kv.value = append(kv.value[:0], value...)
}
kv.noValue = noValue
return h
}
}
return appendArg(h, key, value, noValue)
}
func appendArgBytes(h []argsKV, key, value []byte, noValue bool) []argsKV {
return appendArg(h, b2s(key), b2s(value), noValue)
}
func appendArg(args []argsKV, key, value string, noValue bool) []argsKV {
var kv *argsKV
args, kv = allocArg(args)
kv.key = append(kv.key[:0], key...)
if noValue {
kv.value = kv.value[:0]
} else {
kv.value = append(kv.value[:0], value...)
}
kv.noValue = noValue
return args
}
func allocArg(h []argsKV) ([]argsKV, *argsKV) {
n := len(h)
if cap(h) > n {
h = h[:n+1]
} else {
h = append(h, argsKV{})
}
return h, &h[n]
}
func releaseArg(h []argsKV) []argsKV {
return h[:len(h)-1]
}
func hasArg(h []argsKV, key string) bool {
for i, n := 0, len(h); i < n; i++ {
kv := &h[i]
if key == string(kv.key) {
return true
}
}
return false
}
func peekArgBytes(h []argsKV, k []byte) []byte {
for i, n := 0, len(h); i < n; i++ {
kv := &h[i]
if bytes.Equal(kv.key, k) {
return kv.value
}
}
return nil
}
func peekArgStr(h []argsKV, k string) []byte {
for i, n := 0, len(h); i < n; i++ {
kv := &h[i]
if string(kv.key) == k {
return kv.value
}
}
return nil
}
type argsScanner struct {
b []byte
}
func (s *argsScanner) next(kv *argsKV) bool {
if len(s.b) == 0 {
return false
}
kv.noValue = argsHasValue
isKey := true
k := 0
for i, c := range s.b {
switch c {
case '=':
if isKey {
isKey = false
kv.key = decodeArgAppend(kv.key[:0], s.b[:i])
k = i + 1
}
case '&':
if isKey {
kv.key = decodeArgAppend(kv.key[:0], s.b[:i])
kv.value = kv.value[:0]
kv.noValue = argsNoValue
} else {
kv.value = decodeArgAppend(kv.value[:0], s.b[k:i])
}
s.b = s.b[i+1:]
return true
}
}
if isKey {
kv.key = decodeArgAppend(kv.key[:0], s.b)
kv.value = kv.value[:0]
kv.noValue = argsNoValue
} else {
kv.value = decodeArgAppend(kv.value[:0], s.b[k:])
}
s.b = s.b[len(s.b):]
return true
}
func decodeArgAppend(dst, src []byte) []byte {
if bytes.IndexByte(src, '%') < 0 && bytes.IndexByte(src, '+') < 0 {
// fast path: src doesn't contain encoded chars
return append(dst, src...)
}
// slow path
for i := 0; i < len(src); i++ {
c := src[i]
if c == '%' {
if i+2 >= len(src) {
return append(dst, src[i:]...)
}
x2 := hex2intTable[src[i+2]]
x1 := hex2intTable[src[i+1]]
if x1 == 16 || x2 == 16 {
dst = append(dst, '%')
} else {
dst = append(dst, x1<<4|x2)
i += 2
}
} else if c == '+' {
dst = append(dst, ' ')
} else {
dst = append(dst, c)
}
}
return dst
}
// decodeArgAppendNoPlus is almost identical to decodeArgAppend, but it doesn't
// substitute '+' with ' '.
//
// The function is copy-pasted from decodeArgAppend due to the performance
// reasons only.
func decodeArgAppendNoPlus(dst, src []byte) []byte {
if bytes.IndexByte(src, '%') < 0 {
// fast path: src doesn't contain encoded chars
return append(dst, src...)
}
// slow path
for i := 0; i < len(src); i++ {
c := src[i]
if c == '%' {
if i+2 >= len(src) {
return append(dst, src[i:]...)
}
x2 := hex2intTable[src[i+2]]
x1 := hex2intTable[src[i+1]]
if x1 == 16 || x2 == 16 {
dst = append(dst, '%')
} else {
dst = append(dst, x1<<4|x2)
i += 2
}
} else {
dst = append(dst, c)
}
}
return dst
}

385
vendor/github.com/valyala/fasthttp/bytesconv.go generated vendored Normal file
View File

@ -0,0 +1,385 @@
//go:generate go run bytesconv_table_gen.go
package fasthttp
import (
"bufio"
"bytes"
"errors"
"fmt"
"io"
"math"
"net"
"reflect"
"strings"
"sync"
"time"
"unsafe"
)
// AppendHTMLEscape appends html-escaped s to dst and returns the extended dst.
func AppendHTMLEscape(dst []byte, s string) []byte {
if strings.IndexByte(s, '<') < 0 &&
strings.IndexByte(s, '>') < 0 &&
strings.IndexByte(s, '"') < 0 &&
strings.IndexByte(s, '\'') < 0 {
// fast path - nothing to escape
return append(dst, s...)
}
// slow path
var prev int
var sub string
for i, n := 0, len(s); i < n; i++ {
sub = ""
switch s[i] {
case '<':
sub = "&lt;"
case '>':
sub = "&gt;"
case '"':
sub = "&quot;"
case '\'':
sub = "&#39;"
}
if len(sub) > 0 {
dst = append(dst, s[prev:i]...)
dst = append(dst, sub...)
prev = i + 1
}
}
return append(dst, s[prev:]...)
}
// AppendHTMLEscapeBytes appends html-escaped s to dst and returns
// the extended dst.
func AppendHTMLEscapeBytes(dst, s []byte) []byte {
return AppendHTMLEscape(dst, b2s(s))
}
// AppendIPv4 appends string representation of the given ip v4 to dst
// and returns the extended dst.
func AppendIPv4(dst []byte, ip net.IP) []byte {
ip = ip.To4()
if ip == nil {
return append(dst, "non-v4 ip passed to AppendIPv4"...)
}
dst = AppendUint(dst, int(ip[0]))
for i := 1; i < 4; i++ {
dst = append(dst, '.')
dst = AppendUint(dst, int(ip[i]))
}
return dst
}
var errEmptyIPStr = errors.New("empty ip address string")
// ParseIPv4 parses ip address from ipStr into dst and returns the extended dst.
func ParseIPv4(dst net.IP, ipStr []byte) (net.IP, error) {
if len(ipStr) == 0 {
return dst, errEmptyIPStr
}
if len(dst) < net.IPv4len {
dst = make([]byte, net.IPv4len)
}
copy(dst, net.IPv4zero)
dst = dst.To4()
if dst == nil {
panic("BUG: dst must not be nil")
}
b := ipStr
for i := 0; i < 3; i++ {
n := bytes.IndexByte(b, '.')
if n < 0 {
return dst, fmt.Errorf("cannot find dot in ipStr %q", ipStr)
}
v, err := ParseUint(b[:n])
if err != nil {
return dst, fmt.Errorf("cannot parse ipStr %q: %s", ipStr, err)
}
if v > 255 {
return dst, fmt.Errorf("cannot parse ipStr %q: ip part cannot exceed 255: parsed %d", ipStr, v)
}
dst[i] = byte(v)
b = b[n+1:]
}
v, err := ParseUint(b)
if err != nil {
return dst, fmt.Errorf("cannot parse ipStr %q: %s", ipStr, err)
}
if v > 255 {
return dst, fmt.Errorf("cannot parse ipStr %q: ip part cannot exceed 255: parsed %d", ipStr, v)
}
dst[3] = byte(v)
return dst, nil
}
// AppendHTTPDate appends HTTP-compliant (RFC1123) representation of date
// to dst and returns the extended dst.
func AppendHTTPDate(dst []byte, date time.Time) []byte {
dst = date.In(time.UTC).AppendFormat(dst, time.RFC1123)
copy(dst[len(dst)-3:], strGMT)
return dst
}
// ParseHTTPDate parses HTTP-compliant (RFC1123) date.
func ParseHTTPDate(date []byte) (time.Time, error) {
return time.Parse(time.RFC1123, b2s(date))
}
// AppendUint appends n to dst and returns the extended dst.
func AppendUint(dst []byte, n int) []byte {
if n < 0 {
panic("BUG: int must be positive")
}
var b [20]byte
buf := b[:]
i := len(buf)
var q int
for n >= 10 {
i--
q = n / 10
buf[i] = '0' + byte(n-q*10)
n = q
}
i--
buf[i] = '0' + byte(n)
dst = append(dst, buf[i:]...)
return dst
}
// ParseUint parses uint from buf.
func ParseUint(buf []byte) (int, error) {
v, n, err := parseUintBuf(buf)
if n != len(buf) {
return -1, errUnexpectedTrailingChar
}
return v, err
}
var (
errEmptyInt = errors.New("empty integer")
errUnexpectedFirstChar = errors.New("unexpected first char found. Expecting 0-9")
errUnexpectedTrailingChar = errors.New("unexpected trailing char found. Expecting 0-9")
errTooLongInt = errors.New("too long int")
)
func parseUintBuf(b []byte) (int, int, error) {
n := len(b)
if n == 0 {
return -1, 0, errEmptyInt
}
v := 0
for i := 0; i < n; i++ {
c := b[i]
k := c - '0'
if k > 9 {
if i == 0 {
return -1, i, errUnexpectedFirstChar
}
return v, i, nil
}
// Test for overflow.
if v*10 < v {
return -1, i, errTooLongInt
}
v = 10*v + int(k)
}
return v, n, nil
}
var (
errEmptyFloat = errors.New("empty float number")
errDuplicateFloatPoint = errors.New("duplicate point found in float number")
errUnexpectedFloatEnd = errors.New("unexpected end of float number")
errInvalidFloatExponent = errors.New("invalid float number exponent")
errUnexpectedFloatChar = errors.New("unexpected char found in float number")
)
// ParseUfloat parses unsigned float from buf.
func ParseUfloat(buf []byte) (float64, error) {
if len(buf) == 0 {
return -1, errEmptyFloat
}
b := buf
var v uint64
var offset = 1.0
var pointFound bool
for i, c := range b {
if c < '0' || c > '9' {
if c == '.' {
if pointFound {
return -1, errDuplicateFloatPoint
}
pointFound = true
continue
}
if c == 'e' || c == 'E' {
if i+1 >= len(b) {
return -1, errUnexpectedFloatEnd
}
b = b[i+1:]
minus := -1
switch b[0] {
case '+':
b = b[1:]
minus = 1
case '-':
b = b[1:]
default:
minus = 1
}
vv, err := ParseUint(b)
if err != nil {
return -1, errInvalidFloatExponent
}
return float64(v) * offset * math.Pow10(minus*int(vv)), nil
}
return -1, errUnexpectedFloatChar
}
v = 10*v + uint64(c-'0')
if pointFound {
offset /= 10
}
}
return float64(v) * offset, nil
}
var (
errEmptyHexNum = errors.New("empty hex number")
errTooLargeHexNum = errors.New("too large hex number")
)
func readHexInt(r *bufio.Reader) (int, error) {
n := 0
i := 0
var k int
for {
c, err := r.ReadByte()
if err != nil {
if err == io.EOF && i > 0 {
return n, nil
}
return -1, err
}
k = int(hex2intTable[c])
if k == 16 {
if i == 0 {
return -1, errEmptyHexNum
}
if err := r.UnreadByte(); err != nil {
return -1, err
}
return n, nil
}
if i >= maxHexIntChars {
return -1, errTooLargeHexNum
}
n = (n << 4) | k
i++
}
}
var hexIntBufPool sync.Pool
func writeHexInt(w *bufio.Writer, n int) error {
if n < 0 {
panic("BUG: int must be positive")
}
v := hexIntBufPool.Get()
if v == nil {
v = make([]byte, maxHexIntChars+1)
}
buf := v.([]byte)
i := len(buf) - 1
for {
buf[i] = lowerhex[n&0xf]
n >>= 4
if n == 0 {
break
}
i--
}
_, err := w.Write(buf[i:])
hexIntBufPool.Put(v)
return err
}
const (
upperhex = "0123456789ABCDEF"
lowerhex = "0123456789abcdef"
)
func lowercaseBytes(b []byte) {
for i := 0; i < len(b); i++ {
p := &b[i]
*p = toLowerTable[*p]
}
}
// b2s converts byte slice to a string without memory allocation.
// See https://groups.google.com/forum/#!msg/Golang-Nuts/ENgbUzYvCuU/90yGx7GUAgAJ .
//
// Note it may break if string and/or slice header will change
// in the future go versions.
func b2s(b []byte) string {
return *(*string)(unsafe.Pointer(&b))
}
// s2b converts string to a byte slice without memory allocation.
//
// Note it may break if string and/or slice header will change
// in the future go versions.
func s2b(s string) (b []byte) {
bh := (*reflect.SliceHeader)(unsafe.Pointer(&b))
sh := *(*reflect.StringHeader)(unsafe.Pointer(&s))
bh.Data = sh.Data
bh.Len = sh.Len
bh.Cap = sh.Len
return b
}
// AppendUnquotedArg appends url-decoded src to dst and returns appended dst.
//
// dst may point to src. In this case src will be overwritten.
func AppendUnquotedArg(dst, src []byte) []byte {
return decodeArgAppend(dst, src)
}
// AppendQuotedArg appends url-encoded src to dst and returns appended dst.
func AppendQuotedArg(dst, src []byte) []byte {
for _, c := range src {
switch {
case c == ' ':
dst = append(dst, '+')
case quotedArgShouldEscapeTable[int(c)] != 0:
dst = append(dst, '%', upperhex[c>>4], upperhex[c&0xf])
default:
dst = append(dst, c)
}
}
return dst
}
func appendQuotedPath(dst, src []byte) []byte {
// Fix issue in https://github.com/golang/go/issues/11202
if len(src) == 1 && src[0] == '*' {
return append(dst, '*')
}
for _, c := range src {
if quotedPathShouldEscapeTable[int(c)] != 0 {
dst = append(dst, '%', upperhex[c>>4], upperhex[c&15])
} else {
dst = append(dst, c)
}
}
return dst
}

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