2020-02-23 11:35:47 +00:00
package remotewrite
import (
"flag"
"fmt"
2021-09-28 21:52:07 +00:00
"net/url"
2023-03-28 01:15:28 +00:00
"os"
"path/filepath"
2021-05-20 10:13:40 +00:00
"strconv"
2020-05-30 11:36:40 +00:00
"sync"
2020-02-23 11:35:47 +00:00
"sync/atomic"
2021-05-20 10:13:40 +00:00
"time"
2020-02-23 11:35:47 +00:00
2023-03-28 01:15:28 +00:00
"github.com/cespare/xxhash/v2"
2021-08-05 06:46:19 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
2021-05-20 10:13:40 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bloomfilter"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
2020-12-08 18:49:32 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
2022-11-21 22:38:43 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
2020-02-23 11:35:47 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
2023-03-28 01:15:28 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
2020-02-23 11:35:47 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/persistentqueue"
2020-05-30 11:36:40 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil"
2020-02-23 11:35:47 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
2021-05-19 23:12:36 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
2023-01-04 06:19:18 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/streamaggr"
2021-08-05 06:46:19 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/tenantmetrics"
2020-02-23 11:35:47 +00:00
"github.com/VictoriaMetrics/metrics"
)
var (
2023-02-24 01:36:52 +00:00
remoteWriteURLs = flagutil . NewArrayString ( "remoteWrite.url" , "Remote storage URL to write data to. It must support either VictoriaMetrics remote write protocol " +
"or Prometheus remote_write protocol. Example url: http://<victoriametrics-host>:8428/api/v1/write . " +
2023-02-21 02:38:49 +00:00
"Pass multiple -remoteWrite.url options in order to replicate the collected data to multiple remote storage systems. See also -remoteWrite.multitenantURL" )
2022-10-01 15:26:05 +00:00
remoteWriteMultitenantURLs = flagutil . NewArrayString ( "remoteWrite.multitenantURL" , "Base path for multitenant remote storage URL to write data to. " +
2021-08-05 06:46:19 +00:00
"See https://docs.victoriametrics.com/vmagent.html#multitenancy for details. Example url: http://<vminsert>:8480 . " +
"Pass multiple -remoteWrite.multitenantURL flags in order to replicate data to multiple remote storage systems. See also -remoteWrite.url" )
2021-03-23 14:34:20 +00:00
tmpDataPath = flag . String ( "remoteWrite.tmpDataPath" , "vmagent-remotewrite-data" , "Path to directory where temporary data for remote write component is stored. " +
"See also -remoteWrite.maxDiskUsagePerURL" )
2023-03-28 01:15:28 +00:00
keepDanglingQueues = flag . Bool ( "remoteWrite.keepDanglingQueues" , false , "Keep persistent queues contents at -remoteWrite.tmpDataPath in case there are no matching -remoteWrite.url. " +
"Useful when -remoteWrite.url is changed temporarily and persistent queue files will be needed later on." )
2021-06-17 10:26:35 +00:00
queues = flag . Int ( "remoteWrite.queues" , cgroup . AvailableCPUs ( ) * 2 , "The number of concurrent queues to each -remoteWrite.url. Set more queues if default number of queues " +
2021-08-05 07:00:24 +00:00
"isn't enough for sending high volume of collected data to remote storage. Default value is 2 * numberOfAvailableCPUs" )
2020-02-23 11:35:47 +00:00
showRemoteWriteURL = flag . Bool ( "remoteWrite.showURL" , false , "Whether to show -remoteWrite.url in the exported metrics. " +
2020-07-10 11:07:02 +00:00
"It is hidden by default, since it can contain sensitive info such as auth key" )
2022-10-01 15:39:34 +00:00
maxPendingBytesPerURL = flagutil . NewArrayBytes ( "remoteWrite.maxDiskUsagePerURL" , "The maximum file-based buffer size in bytes at -remoteWrite.tmpDataPath " +
2020-03-03 17:48:46 +00:00
"for each -remoteWrite.url. When buffer size reaches the configured maximum, then old data is dropped when adding new data to the buffer. " +
2022-04-26 12:48:20 +00:00
"Buffered data is stored in ~500MB chunks, so the minimum practical value for this flag is 500MB. " +
2020-03-03 17:48:46 +00:00
"Disk usage is unlimited if the value is set to 0" )
2021-02-01 12:27:05 +00:00
significantFigures = flagutil . NewArrayInt ( "remoteWrite.significantFigures" , "The number of significant figures to leave in metric values before writing them " +
"to remote storage. See https://en.wikipedia.org/wiki/Significant_figures . Zero value saves all the significant figures. " +
"This option may be used for improving data compression for the stored metrics. See also -remoteWrite.roundDigits" )
roundDigits = flagutil . NewArrayInt ( "remoteWrite.roundDigits" , "Round metric values to this number of decimal digits after the point before writing them to remote storage. " +
"Examples: -remoteWrite.roundDigits=2 would round 1.236 to 1.24, while -remoteWrite.roundDigits=-1 would round 126.78 to 130. " +
"By default digits rounding is disabled. Set it to 100 for disabling it for a particular remote storage. " +
"This option may be used for improving data compression for the stored metrics" )
2021-05-19 23:12:36 +00:00
sortLabels = flag . Bool ( "sortLabels" , false , ` Whether to sort labels for incoming samples before writing them to all the configured remote storage systems. ` +
` This may be needed for reducing memory usage at remote storage when the order of labels in incoming samples is random. ` +
` For example, if m { k1="v1",k2="v2"} may be sent as m { k2="v2",k1="v1"} ` +
` Enabled sorting for labels can slow down ingestion performance a bit ` )
2021-05-20 10:13:40 +00:00
maxHourlySeries = flag . Int ( "remoteWrite.maxHourlySeries" , 0 , "The maximum number of unique series vmagent can send to remote storage systems during the last hour. " +
2021-09-01 13:34:32 +00:00
"Excess series are logged and dropped. This can be useful for limiting series cardinality. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter" )
2021-05-20 10:13:40 +00:00
maxDailySeries = flag . Int ( "remoteWrite.maxDailySeries" , 0 , "The maximum number of unique series vmagent can send to remote storage systems during the last 24 hours. " +
2021-09-01 13:34:32 +00:00
"Excess series are logged and dropped. This can be useful for limiting series churn rate. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter" )
2020-02-23 11:35:47 +00:00
)
2021-08-05 06:46:19 +00:00
var (
// rwctxsDefault contains statically populated entries when -remoteWrite.url is specified.
rwctxsDefault [ ] * remoteWriteCtx
// rwctxsMap contains dynamically populated entries when -remoteWrite.multitenantURL is specified.
rwctxsMap = make ( map [ tenantmetrics . TenantID ] [ ] * remoteWriteCtx )
rwctxsMapLock sync . Mutex
2021-08-05 06:44:29 +00:00
2021-08-05 06:46:19 +00:00
// Data without tenant id is written to defaultAuthToken if -remoteWrite.multitenantURL is specified.
defaultAuthToken = & auth . Token { }
)
// MultitenancyEnabled returns true if -remoteWrite.multitenantURL is specified.
func MultitenancyEnabled ( ) bool {
return len ( * remoteWriteMultitenantURLs ) > 0
}
2020-03-03 11:08:17 +00:00
2020-05-30 11:36:40 +00:00
// Contains the current relabelConfigs.
var allRelabelConfigs atomic . Value
2023-03-29 16:05:58 +00:00
// Contains the loader for stream aggregation configs.
var saCfgLoader * saConfigsLoader
2020-08-30 18:23:38 +00:00
// maxQueues limits the maximum value for `-remoteWrite.queues`. There is no sense in setting too high value,
// since it may lead to high memory usage due to big number of buffers.
2021-04-23 19:01:57 +00:00
var maxQueues = cgroup . AvailableCPUs ( ) * 16
2020-08-30 18:23:38 +00:00
2023-03-28 01:33:05 +00:00
const persistentQueueDirname = "persistent-queue"
2023-03-28 01:15:28 +00:00
2020-09-29 16:48:53 +00:00
// InitSecretFlags must be called after flag.Parse and before any logging.
func InitSecretFlags ( ) {
if ! * showRemoteWriteURL {
// remoteWrite.url can contain authentication codes, so hide it at `/metrics` output.
flagutil . RegisterSecretFlag ( "remoteWrite.url" )
}
}
2020-02-23 11:35:47 +00:00
// Init initializes remotewrite.
//
// It must be called after flag.Parse().
//
// Stop must be called for graceful shutdown.
2021-08-05 06:46:19 +00:00
func Init ( ) {
2021-08-05 06:44:29 +00:00
if len ( * remoteWriteURLs ) == 0 && len ( * remoteWriteMultitenantURLs ) == 0 {
logger . Fatalf ( "at least one `-remoteWrite.url` or `-remoteWrite.multitenantURL` command-line flag must be set" )
}
2021-08-05 06:46:19 +00:00
if len ( * remoteWriteURLs ) > 0 && len ( * remoteWriteMultitenantURLs ) > 0 {
logger . Fatalf ( "cannot set both `-remoteWrite.url` and `-remoteWrite.multitenantURL` command-line flags" )
2021-08-05 06:44:29 +00:00
}
2021-05-20 10:13:40 +00:00
if * maxHourlySeries > 0 {
hourlySeriesLimiter = bloomfilter . NewLimiter ( * maxHourlySeries , time . Hour )
2021-05-20 12:27:06 +00:00
_ = metrics . NewGauge ( ` vmagent_hourly_series_limit_max_series ` , func ( ) float64 {
return float64 ( hourlySeriesLimiter . MaxItems ( ) )
} )
_ = metrics . NewGauge ( ` vmagent_hourly_series_limit_current_series ` , func ( ) float64 {
return float64 ( hourlySeriesLimiter . CurrentItems ( ) )
} )
2021-05-20 10:13:40 +00:00
}
if * maxDailySeries > 0 {
dailySeriesLimiter = bloomfilter . NewLimiter ( * maxDailySeries , 24 * time . Hour )
2021-05-20 12:27:06 +00:00
_ = metrics . NewGauge ( ` vmagent_daily_series_limit_max_series ` , func ( ) float64 {
return float64 ( dailySeriesLimiter . MaxItems ( ) )
} )
_ = metrics . NewGauge ( ` vmagent_daily_series_limit_current_series ` , func ( ) float64 {
return float64 ( dailySeriesLimiter . CurrentItems ( ) )
} )
2021-05-20 10:13:40 +00:00
}
2020-08-30 18:23:38 +00:00
if * queues > maxQueues {
* queues = maxQueues
}
if * queues <= 0 {
* queues = 1
2020-02-23 11:35:47 +00:00
}
2020-05-30 11:36:40 +00:00
initLabelsGlobal ( )
2021-05-21 13:34:03 +00:00
// Register SIGHUP handler for config reload before loadRelabelConfigs.
// This guarantees that the config will be re-read if the signal arrives just after loadRelabelConfig.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1240
sighupCh := procutil . NewSighupChan ( )
2020-05-30 11:36:40 +00:00
rcs , err := loadRelabelConfigs ( )
if err != nil {
logger . Fatalf ( "cannot load relabel configs: %s" , err )
}
allRelabelConfigs . Store ( rcs )
2023-01-04 06:19:18 +00:00
2023-03-29 16:05:58 +00:00
relabelConfigSuccess . Set ( 1 )
relabelConfigTimestamp . Set ( fasttime . UnixTimestamp ( ) )
saCfgLoader , err = newSaConfigsLoader ( * streamAggrConfig )
if err != nil {
logger . Fatalf ( "cannot load stream aggregation config: %s" , err )
}
2020-02-23 11:35:47 +00:00
2021-08-05 06:46:19 +00:00
if len ( * remoteWriteURLs ) > 0 {
rwctxsDefault = newRemoteWriteCtxs ( nil , * remoteWriteURLs )
2021-08-05 06:44:29 +00:00
}
2020-05-30 11:36:40 +00:00
// Start config reloader.
configReloaderWG . Add ( 1 )
go func ( ) {
defer configReloaderWG . Done ( )
for {
select {
case <- sighupCh :
case <- stopCh :
return
}
2023-03-29 16:05:58 +00:00
relabelConfigReloads . Inc ( )
2020-05-30 11:36:40 +00:00
logger . Infof ( "SIGHUP received; reloading relabel configs pointed by -remoteWrite.relabelConfig and -remoteWrite.urlRelabelConfig" )
rcs , err := loadRelabelConfigs ( )
if err != nil {
2023-03-29 16:05:58 +00:00
relabelConfigReloadErrors . Inc ( )
relabelConfigSuccess . Set ( 0 )
2020-05-30 11:36:40 +00:00
logger . Errorf ( "cannot reload relabel configs; preserving the previous configs; error: %s" , err )
continue
}
allRelabelConfigs . Store ( rcs )
2023-03-29 16:05:58 +00:00
relabelConfigSuccess . Set ( 1 )
relabelConfigTimestamp . Set ( fasttime . UnixTimestamp ( ) )
2020-05-30 11:36:40 +00:00
logger . Infof ( "Successfully reloaded relabel configs" )
2023-03-29 16:05:58 +00:00
logger . Infof ( "reloading stream agg configs pointed by -remoteWrite.streamAggr.config" )
err = saCfgLoader . reloadConfigs ( )
if err != nil {
logger . Errorf ( "Cannot reload stream aggregation configs: %s" , err )
}
if len ( * remoteWriteMultitenantURLs ) > 0 {
rwctxsMapLock . Lock ( )
for _ , rwctxs := range rwctxsMap {
for _ , rwctx := range rwctxs {
rwctx . reinitStreamAggr ( )
}
}
rwctxsMapLock . Unlock ( )
} else {
for _ , rwctx := range rwctxsDefault {
rwctx . reinitStreamAggr ( )
}
}
logger . Infof ( "Successfully reloaded stream aggregation configs" )
2020-05-30 11:36:40 +00:00
}
} ( )
2020-02-23 11:35:47 +00:00
}
2022-11-21 22:38:43 +00:00
var (
2023-03-29 16:05:58 +00:00
relabelConfigReloads = metrics . NewCounter ( ` vmagent_relabel_config_reloads_total ` )
relabelConfigReloadErrors = metrics . NewCounter ( ` vmagent_relabel_config_reloads_errors_total ` )
relabelConfigSuccess = metrics . NewCounter ( ` vmagent_relabel_config_last_reload_successful ` )
relabelConfigTimestamp = metrics . NewCounter ( ` vmagent_relabel_config_last_reload_success_timestamp_seconds ` )
2022-11-21 22:38:43 +00:00
)
2021-08-05 06:46:19 +00:00
func newRemoteWriteCtxs ( at * auth . Token , urls [ ] string ) [ ] * remoteWriteCtx {
if len ( urls ) == 0 {
logger . Panicf ( "BUG: urls must be non-empty" )
}
2021-11-04 13:39:14 +00:00
maxInmemoryBlocks := memory . Allowed ( ) / len ( urls ) / * maxRowsPerBlock / 100
2021-11-05 13:14:49 +00:00
if maxInmemoryBlocks / * queues > 100 {
2021-08-05 06:46:19 +00:00
// 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.
2021-11-05 13:14:49 +00:00
maxInmemoryBlocks = 100 * * queues
2021-08-05 06:46:19 +00:00
}
if maxInmemoryBlocks < 2 {
maxInmemoryBlocks = 2
}
rwctxs := make ( [ ] * remoteWriteCtx , len ( urls ) )
2021-09-28 21:52:07 +00:00
for i , remoteWriteURLRaw := range urls {
remoteWriteURL , err := url . Parse ( remoteWriteURLRaw )
if err != nil {
logger . Fatalf ( "invalid -remoteWrite.url=%q: %s" , remoteWriteURL , err )
}
2021-08-05 06:46:19 +00:00
sanitizedURL := fmt . Sprintf ( "%d:secret-url" , i + 1 )
if at != nil {
// Construct full remote_write url for the given tenant according to https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format
2021-09-28 21:52:07 +00:00
remoteWriteURL . Path = fmt . Sprintf ( "%s/insert/%d:%d/prometheus/api/v1/write" , remoteWriteURL . Path , at . AccountID , at . ProjectID )
2021-08-05 06:46:19 +00:00
sanitizedURL = fmt . Sprintf ( "%s:%d:%d" , sanitizedURL , at . AccountID , at . ProjectID )
}
if * showRemoteWriteURL {
sanitizedURL = fmt . Sprintf ( "%d:%s" , i + 1 , remoteWriteURL )
}
2021-09-28 21:52:07 +00:00
rwctxs [ i ] = newRemoteWriteCtx ( i , at , remoteWriteURL , maxInmemoryBlocks , sanitizedURL )
2021-08-05 06:46:19 +00:00
}
2023-03-28 01:15:28 +00:00
if ! * keepDanglingQueues {
// Remove dangling queues, if any.
// This is required for the case when the number of queues has been changed or URL have been changed.
// See: https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4014
existingQueues := make ( map [ string ] struct { } , len ( rwctxs ) )
for _ , rwctx := range rwctxs {
2023-03-28 01:33:05 +00:00
existingQueues [ rwctx . fq . Dirname ( ) ] = struct { } { }
2023-03-28 01:15:28 +00:00
}
2023-03-28 01:33:05 +00:00
queuesDir := filepath . Join ( * tmpDataPath , persistentQueueDirname )
2023-03-28 01:15:28 +00:00
files , err := os . ReadDir ( queuesDir )
if err != nil {
logger . Fatalf ( "cannot read queues dir %q: %s" , queuesDir , err )
}
removed := 0
for _ , f := range files {
2023-03-28 01:33:05 +00:00
dirname := f . Name ( )
if _ , ok := existingQueues [ dirname ] ; ! ok {
logger . Infof ( "removing dangling queue %q" , dirname )
fullPath := filepath . Join ( queuesDir , dirname )
2023-03-28 01:15:28 +00:00
fs . MustRemoveAll ( fullPath )
removed ++
}
}
2023-03-28 01:33:05 +00:00
if removed > 0 {
logger . Infof ( "removed %d dangling queues from %q, active queues: %d" , removed , * tmpDataPath , len ( rwctxs ) )
}
2023-03-28 01:15:28 +00:00
}
2021-08-05 06:46:19 +00:00
return rwctxs
}
2020-05-30 11:36:40 +00:00
var stopCh = make ( chan struct { } )
var configReloaderWG sync . WaitGroup
2020-02-23 11:35:47 +00:00
// Stop stops remotewrite.
//
// It is expected that nobody calls Push during and after the call to this func.
func Stop ( ) {
2020-05-30 11:36:40 +00:00
close ( stopCh )
configReloaderWG . Wait ( )
2021-08-05 06:46:19 +00:00
for _ , rwctx := range rwctxsDefault {
rwctx . MustStop ( )
}
rwctxsDefault = nil
// There is no need in locking rwctxsMapLock here, since nobody should call Push during the Stop call.
2021-08-05 06:44:29 +00:00
for _ , rwctxs := range rwctxsMap {
for _ , rwctx := range rwctxs {
rwctx . MustStop ( )
}
2020-02-23 11:35:47 +00:00
}
2021-08-05 06:44:29 +00:00
rwctxsMap = nil
2021-09-01 11:14:37 +00:00
if sl := hourlySeriesLimiter ; sl != nil {
sl . MustStop ( )
}
if sl := dailySeriesLimiter ; sl != nil {
sl . MustStop ( )
}
2020-02-23 11:35:47 +00:00
}
// Push sends wr to remote storage systems set via `-remoteWrite.url`.
//
2022-08-08 11:46:24 +00:00
// If at is nil, then the data is pushed to the configured `-remoteWrite.url`.
2022-12-04 05:53:01 +00:00
// If at isn't nil, the data is pushed to the configured `-remoteWrite.multitenantURL`.
2021-08-05 06:46:19 +00:00
//
// Note that wr may be modified by Push due to relabeling and rounding.
2022-08-08 11:46:24 +00:00
func Push ( at * auth . Token , wr * prompbmarshal . WriteRequest ) {
2021-08-05 06:46:19 +00:00
if at == nil && len ( * remoteWriteMultitenantURLs ) > 0 {
// Write data to default tenant if at isn't set while -remoteWrite.multitenantURL is set.
at = defaultAuthToken
2021-08-05 06:44:29 +00:00
}
2021-08-05 06:46:19 +00:00
var rwctxs [ ] * remoteWriteCtx
if at == nil {
rwctxs = rwctxsDefault
} else {
if len ( * remoteWriteMultitenantURLs ) == 0 {
2022-08-08 11:46:24 +00:00
logger . Panicf ( "BUG: -remoteWrite.multitenantURL command-line flag must be set when __tenant_id__=%q label is set" , at )
2021-08-05 06:46:19 +00:00
}
rwctxsMapLock . Lock ( )
tenantID := tenantmetrics . TenantID {
AccountID : at . AccountID ,
ProjectID : at . ProjectID ,
}
rwctxs = rwctxsMap [ tenantID ]
if rwctxs == nil {
rwctxs = newRemoteWriteCtxs ( at , * remoteWriteMultitenantURLs )
rwctxsMap [ tenantID ] = rwctxs
}
rwctxsMapLock . Unlock ( )
2021-08-05 06:44:29 +00:00
}
2020-03-03 11:08:17 +00:00
var rctx * relabelCtx
2020-05-30 11:36:40 +00:00
rcs := allRelabelConfigs . Load ( ) . ( * relabelConfigs )
2021-02-22 14:33:55 +00:00
pcsGlobal := rcs . global
if pcsGlobal . Len ( ) > 0 || len ( labelsGlobal ) > 0 {
2020-03-03 11:08:17 +00:00
rctx = getRelabelCtx ( )
}
2020-02-28 16:57:45 +00:00
tss := wr . Timeseries
2022-05-06 12:28:59 +00:00
rowsCount := getRowsCount ( tss )
globalRowsPushedBeforeRelabel . Add ( rowsCount )
2021-11-04 13:39:14 +00:00
maxSamplesPerBlock := * maxRowsPerBlock
// Allow up to 10x of labels per each block on average.
maxLabelsPerBlock := 10 * maxSamplesPerBlock
2020-02-28 16:57:45 +00:00
for len ( tss ) > 0 {
2020-07-10 12:13:26 +00:00
// Process big tss in smaller blocks in order to reduce the maximum memory usage
2020-09-26 01:07:45 +00:00
samplesCount := 0
2021-03-30 21:44:31 +00:00
labelsCount := 0
2020-09-26 01:07:45 +00:00
i := 0
for i < len ( tss ) {
samplesCount += len ( tss [ i ] . Samples )
2021-03-30 21:44:31 +00:00
labelsCount += len ( tss [ i ] . Labels )
2020-09-26 01:07:45 +00:00
i ++
2021-11-04 13:39:14 +00:00
if samplesCount >= maxSamplesPerBlock || labelsCount >= maxLabelsPerBlock {
2020-09-26 01:07:45 +00:00
break
}
}
2020-02-28 16:57:45 +00:00
tssBlock := tss
2020-09-26 01:07:45 +00:00
if i < len ( tss ) {
tssBlock = tss [ : i ]
tss = tss [ i : ]
2020-02-28 18:03:38 +00:00
} else {
tss = nil
2020-02-28 16:57:45 +00:00
}
2020-03-03 11:08:17 +00:00
if rctx != nil {
2022-05-06 12:28:59 +00:00
rowsCountBeforeRelabel := getRowsCount ( tssBlock )
2021-02-22 14:33:55 +00:00
tssBlock = rctx . applyRelabeling ( tssBlock , labelsGlobal , pcsGlobal )
2022-05-06 12:28:59 +00:00
rowsCountAfterRelabel := getRowsCount ( tssBlock )
rowsDroppedByGlobalRelabel . Add ( rowsCountBeforeRelabel - rowsCountAfterRelabel )
2020-03-03 11:08:17 +00:00
}
2021-05-19 23:12:36 +00:00
sortLabelsIfNeeded ( tssBlock )
2021-05-20 10:13:40 +00:00
tssBlock = limitSeriesCardinality ( tssBlock )
2021-11-04 13:00:51 +00:00
pushBlockToRemoteStorages ( rwctxs , tssBlock )
2020-03-03 13:00:52 +00:00
if rctx != nil {
rctx . reset ( )
}
2020-02-28 16:57:45 +00:00
}
2020-03-03 11:08:17 +00:00
if rctx != nil {
putRelabelCtx ( rctx )
}
2020-02-23 11:35:47 +00:00
}
2021-11-04 13:00:51 +00:00
func pushBlockToRemoteStorages ( rwctxs [ ] * remoteWriteCtx , tssBlock [ ] prompbmarshal . TimeSeries ) {
if len ( tssBlock ) == 0 {
// Nothing to push
return
}
// Push block to remote storages in parallel in order to reduce the time needed for sending the data to multiple remote storage systems.
var wg sync . WaitGroup
for _ , rwctx := range rwctxs {
wg . Add ( 1 )
2021-11-04 14:58:28 +00:00
go func ( rwctx * remoteWriteCtx ) {
2021-11-04 13:00:51 +00:00
defer wg . Done ( )
rwctx . Push ( tssBlock )
2021-11-04 14:58:28 +00:00
} ( rwctx )
2021-11-04 13:00:51 +00:00
}
2021-11-04 14:58:28 +00:00
wg . Wait ( )
2021-11-04 13:00:51 +00:00
}
2021-05-19 23:12:36 +00:00
// sortLabelsIfNeeded sorts labels if -sortLabels command-line flag is set.
func sortLabelsIfNeeded ( tss [ ] prompbmarshal . TimeSeries ) {
if ! * sortLabels {
return
}
for i := range tss {
promrelabel . SortLabels ( tss [ i ] . Labels )
}
}
2021-05-20 10:13:40 +00:00
func limitSeriesCardinality ( tss [ ] prompbmarshal . TimeSeries ) [ ] prompbmarshal . TimeSeries {
if hourlySeriesLimiter == nil && dailySeriesLimiter == nil {
return tss
}
dst := make ( [ ] prompbmarshal . TimeSeries , 0 , len ( tss ) )
for i := range tss {
labels := tss [ i ] . Labels
h := getLabelsHash ( labels )
if hourlySeriesLimiter != nil && ! hourlySeriesLimiter . Add ( h ) {
2021-05-20 11:15:19 +00:00
hourlySeriesLimitRowsDropped . Add ( len ( tss [ i ] . Samples ) )
logSkippedSeries ( labels , "-remoteWrite.maxHourlySeries" , hourlySeriesLimiter . MaxItems ( ) )
2021-05-20 10:13:40 +00:00
continue
}
if dailySeriesLimiter != nil && ! dailySeriesLimiter . Add ( h ) {
2021-05-20 11:15:19 +00:00
dailySeriesLimitRowsDropped . Add ( len ( tss [ i ] . Samples ) )
logSkippedSeries ( labels , "-remoteWrite.maxDailySeries" , dailySeriesLimiter . MaxItems ( ) )
2021-05-20 10:13:40 +00:00
continue
}
dst = append ( dst , tss [ i ] )
}
return dst
}
var (
hourlySeriesLimiter * bloomfilter . Limiter
dailySeriesLimiter * bloomfilter . Limiter
2021-05-20 11:15:19 +00:00
hourlySeriesLimitRowsDropped = metrics . NewCounter ( ` vmagent_hourly_series_limit_rows_dropped_total ` )
dailySeriesLimitRowsDropped = metrics . NewCounter ( ` vmagent_daily_series_limit_rows_dropped_total ` )
2021-05-20 10:13:40 +00:00
)
func getLabelsHash ( labels [ ] prompbmarshal . Label ) uint64 {
bb := labelsHashBufPool . Get ( )
b := bb . B [ : 0 ]
for _ , label := range labels {
b = append ( b , label . Name ... )
b = append ( b , label . Value ... )
}
h := xxhash . Sum64 ( b )
bb . B = b
labelsHashBufPool . Put ( bb )
return h
}
var labelsHashBufPool bytesutil . ByteBufferPool
func logSkippedSeries ( labels [ ] prompbmarshal . Label , flagName string , flagValue int ) {
select {
case <- logSkippedSeriesTicker . C :
2021-12-21 15:03:25 +00:00
// Do not use logger.WithThrottler() here, since this will increase CPU usage
// because every call to logSkippedSeries will result to a call to labelsToString.
2021-05-20 10:13:40 +00:00
logger . Warnf ( "skip series %s because %s=%d reached" , labelsToString ( labels ) , flagName , flagValue )
default :
}
}
var logSkippedSeriesTicker = time . NewTicker ( 5 * time . Second )
func labelsToString ( labels [ ] prompbmarshal . Label ) string {
var b [ ] byte
b = append ( b , '{' )
for i , label := range labels {
b = append ( b , label . Name ... )
b = append ( b , '=' )
b = strconv . AppendQuote ( b , label . Value )
if i + 1 < len ( labels ) {
b = append ( b , ',' )
}
}
b = append ( b , '}' )
return string ( b )
}
2022-05-06 12:28:59 +00:00
var (
2022-05-06 12:50:50 +00:00
globalRowsPushedBeforeRelabel = metrics . NewCounter ( "vmagent_remotewrite_global_rows_pushed_before_relabel_total" )
2022-05-06 12:28:59 +00:00
rowsDroppedByGlobalRelabel = metrics . NewCounter ( "vmagent_remotewrite_global_relabel_metrics_dropped_total" )
)
2020-03-03 11:08:17 +00:00
type remoteWriteCtx struct {
2023-01-04 06:19:18 +00:00
idx int
fq * persistentqueue . FastQueue
c * client
sas * streamaggr . Aggregators
2023-03-29 16:05:58 +00:00
saHash uint64
2023-01-04 06:19:18 +00:00
streamAggrKeepInput bool
2020-03-03 11:08:17 +00:00
pss [ ] * pendingSeries
pssNextIdx uint64
2022-05-06 12:28:59 +00:00
rowsPushedAfterRelabel * metrics . Counter
rowsDroppedByRelabel * metrics . Counter
2020-03-03 11:08:17 +00:00
}
2021-09-28 21:52:07 +00:00
func newRemoteWriteCtx ( argIdx int , at * auth . Token , remoteWriteURL * url . URL , maxInmemoryBlocks int , sanitizedURL string ) * remoteWriteCtx {
// strip query params, otherwise changing params resets pq
pqURL := * remoteWriteURL
pqURL . RawQuery = ""
pqURL . Fragment = ""
h := xxhash . Sum64 ( [ ] byte ( pqURL . String ( ) ) )
2023-03-28 01:33:05 +00:00
queuePath := filepath . Join ( * tmpDataPath , persistentQueueDirname , fmt . Sprintf ( "%d_%016X" , argIdx + 1 , h ) )
2022-10-01 15:39:34 +00:00
maxPendingBytes := maxPendingBytesPerURL . GetOptionalArgOrDefault ( argIdx , 0 )
fq := persistentqueue . MustOpenFastQueue ( queuePath , sanitizedURL , maxInmemoryBlocks , maxPendingBytes )
2021-09-28 21:52:07 +00:00
_ = metrics . GetOrCreateGauge ( fmt . Sprintf ( ` vmagent_remotewrite_pending_data_bytes { path=%q, url=%q} ` , queuePath , sanitizedURL ) , func ( ) float64 {
2020-03-03 11:08:17 +00:00
return float64 ( fq . GetPendingBytes ( ) )
} )
2021-09-28 21:52:07 +00:00
_ = metrics . GetOrCreateGauge ( fmt . Sprintf ( ` vmagent_remotewrite_pending_inmemory_blocks { path=%q, url=%q} ` , queuePath , sanitizedURL ) , func ( ) float64 {
2020-03-03 11:08:17 +00:00
return float64 ( fq . GetInmemoryQueueLen ( ) )
} )
2023-02-24 01:36:52 +00:00
2021-09-28 21:52:07 +00:00
var c * client
switch remoteWriteURL . Scheme {
case "http" , "https" :
2023-02-26 20:07:30 +00:00
c = newHTTPClient ( argIdx , remoteWriteURL . String ( ) , sanitizedURL , fq , * queues )
2021-09-28 21:52:07 +00:00
default :
logger . Fatalf ( "unsupported scheme: %s for remoteWriteURL: %s, want `http`, `https`" , remoteWriteURL . Scheme , sanitizedURL )
}
c . init ( argIdx , * queues , sanitizedURL )
2023-01-04 06:19:18 +00:00
// Initialize pss
2021-02-01 12:27:05 +00:00
sf := significantFigures . GetOptionalArgOrDefault ( argIdx , 0 )
rd := roundDigits . GetOptionalArgOrDefault ( argIdx , 100 )
2021-03-31 13:16:26 +00:00
pssLen := * queues
if n := cgroup . AvailableCPUs ( ) ; pssLen > n {
// There is no sense in running more than availableCPUs concurrent pendingSeries,
// since every pendingSeries can saturate up to a single CPU.
pssLen = n
}
pss := make ( [ ] * pendingSeries , pssLen )
2020-03-03 11:08:17 +00:00
for i := range pss {
2023-02-26 20:07:30 +00:00
pss [ i ] = newPendingSeries ( fq . MustWriteBlock , c . useVMProto , sf , rd )
2020-03-03 11:08:17 +00:00
}
2023-01-04 06:19:18 +00:00
rwctx := & remoteWriteCtx {
2020-05-30 11:36:40 +00:00
idx : argIdx ,
fq : fq ,
c : c ,
pss : pss ,
2020-03-03 11:08:17 +00:00
2022-05-06 12:28:59 +00:00
rowsPushedAfterRelabel : metrics . GetOrCreateCounter ( fmt . Sprintf ( ` vmagent_remotewrite_rows_pushed_after_relabel_total { path=%q, url=%q} ` , queuePath , sanitizedURL ) ) ,
rowsDroppedByRelabel : metrics . GetOrCreateCounter ( fmt . Sprintf ( ` vmagent_remotewrite_relabel_metrics_dropped_total { path=%q, url=%q} ` , queuePath , sanitizedURL ) ) ,
2020-02-23 11:35:47 +00:00
}
2023-01-04 06:19:18 +00:00
// Initialize sas
2023-03-29 16:05:58 +00:00
saCfg , saHash := saCfgLoader . getCurrentConfig ( argIdx )
if len ( saCfg ) > 0 {
sasFile := streamAggrConfig . GetOptionalArg ( argIdx )
2023-01-25 17:14:49 +00:00
dedupInterval := streamAggrDedupInterval . GetOptionalArgOrDefault ( argIdx , 0 )
2023-03-29 16:05:58 +00:00
sas , err := streamaggr . NewAggregators ( saCfg , rwctx . pushInternal , dedupInterval )
2023-01-04 06:19:18 +00:00
if err != nil {
logger . Fatalf ( "cannot initialize stream aggregators from -remoteWrite.streamAggrFile=%q: %s" , sasFile , err )
}
rwctx . sas = sas
2023-03-29 16:05:58 +00:00
rwctx . saHash = saHash
2023-01-04 06:19:18 +00:00
rwctx . streamAggrKeepInput = streamAggrKeepInput . GetOptionalArg ( argIdx )
}
return rwctx
2020-02-23 11:35:47 +00:00
}
2020-03-03 11:08:17 +00:00
func ( rwctx * remoteWriteCtx ) MustStop ( ) {
for _ , ps := range rwctx . pss {
ps . MustStop ( )
}
2020-05-30 11:36:40 +00:00
rwctx . idx = 0
2020-03-03 11:08:17 +00:00
rwctx . pss = nil
2021-02-18 22:31:07 +00:00
rwctx . fq . UnblockAllReaders ( )
2020-03-03 11:08:17 +00:00
rwctx . c . MustStop ( )
rwctx . c = nil
2023-01-04 06:19:18 +00:00
rwctx . sas . MustStop ( )
rwctx . sas = nil
2021-02-17 19:23:38 +00:00
rwctx . fq . MustClose ( )
rwctx . fq = nil
2020-03-03 11:08:17 +00:00
2022-05-06 12:28:59 +00:00
rwctx . rowsPushedAfterRelabel = nil
rwctx . rowsDroppedByRelabel = nil
2020-03-03 11:08:17 +00:00
}
2020-02-23 11:35:47 +00:00
2020-03-03 11:08:17 +00:00
func ( rwctx * remoteWriteCtx ) Push ( tss [ ] prompbmarshal . TimeSeries ) {
2023-01-04 06:19:18 +00:00
// Apply relabeling
2020-03-03 11:08:17 +00:00
var rctx * relabelCtx
2020-07-10 12:13:26 +00:00
var v * [ ] prompbmarshal . TimeSeries
2020-05-30 11:36:40 +00:00
rcs := allRelabelConfigs . Load ( ) . ( * relabelConfigs )
2021-02-22 14:33:55 +00:00
pcs := rcs . perURL [ rwctx . idx ]
if pcs . Len ( ) > 0 {
2020-07-10 12:13:26 +00:00
rctx = getRelabelCtx ( )
2020-05-12 19:01:47 +00:00
// Make a copy of tss before applying relabeling in order to prevent
// from affecting time series for other remoteWrite.url configs.
2020-07-10 12:13:26 +00:00
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/467
// and https://github.com/VictoriaMetrics/VictoriaMetrics/issues/599
v = tssRelabelPool . Get ( ) . ( * [ ] prompbmarshal . TimeSeries )
tss = append ( * v , tss ... )
2022-05-06 12:28:59 +00:00
rowsCountBeforeRelabel := getRowsCount ( tss )
2021-02-22 14:33:55 +00:00
tss = rctx . applyRelabeling ( tss , nil , pcs )
2022-05-06 12:28:59 +00:00
rowsCountAfterRelabel := getRowsCount ( tss )
rwctx . rowsDroppedByRelabel . Add ( rowsCountBeforeRelabel - rowsCountAfterRelabel )
2020-03-03 11:08:17 +00:00
}
2022-05-06 12:28:59 +00:00
rowsCount := getRowsCount ( tss )
rwctx . rowsPushedAfterRelabel . Add ( rowsCount )
2023-01-04 06:19:18 +00:00
// Apply stream aggregation if any
rwctx . sas . Push ( tss )
if rwctx . sas == nil || rwctx . streamAggrKeepInput {
// Push samples to the remote storage
rwctx . pushInternal ( tss )
}
// Return back relabeling contexts to the pool
2020-03-03 11:08:17 +00:00
if rctx != nil {
2020-07-10 12:13:26 +00:00
* v = prompbmarshal . ResetTimeSeries ( tss )
tssRelabelPool . Put ( v )
2020-03-03 11:08:17 +00:00
putRelabelCtx ( rctx )
}
}
2020-07-10 12:13:26 +00:00
2023-01-04 06:19:18 +00:00
func ( rwctx * remoteWriteCtx ) pushInternal ( tss [ ] prompbmarshal . TimeSeries ) {
pss := rwctx . pss
idx := atomic . AddUint64 ( & rwctx . pssNextIdx , 1 ) % uint64 ( len ( pss ) )
pss [ idx ] . Push ( tss )
}
2023-03-29 16:05:58 +00:00
func ( rwctx * remoteWriteCtx ) reinitStreamAggr ( ) {
if rwctx . sas == nil {
return
}
saCfg , saHash := saCfgLoader . getCurrentConfig ( rwctx . idx )
if rwctx . saHash == saHash {
return
}
if err := rwctx . sas . ReInitConfigs ( saCfg ) ; err != nil {
logger . Errorf ( "Cannot apply stream aggregation configs %d: %s" , rwctx . idx , err )
}
rwctx . saHash = saHash
}
2020-07-10 12:13:26 +00:00
var tssRelabelPool = & sync . Pool {
New : func ( ) interface { } {
2020-07-14 11:27:50 +00:00
a := [ ] prompbmarshal . TimeSeries { }
return & a
2020-07-10 12:13:26 +00:00
} ,
}
2022-05-06 12:28:59 +00:00
func getRowsCount ( tss [ ] prompbmarshal . TimeSeries ) int {
rowsCount := 0
for _ , ts := range tss {
rowsCount += len ( ts . Samples )
}
return rowsCount
}