Compare commits

..

16 Commits

Author SHA1 Message Date
Max Kotliar
24ae4758f2 count global inserted ts 2026-06-17 15:44:48 +03:00
Max Kotliar
b94dd42126 fix potential race in cardinality metrics writer 2026-06-17 15:44:48 +03:00
Max Kotliar
54768be72d fix potential name\value slice reuse in protoparser 2026-06-17 15:44:47 +03:00
Max Kotliar
fe1f2b7c2a rename metrics cestimator_ -> vmestimator 2026-06-17 15:24:31 +03:00
Max Kotliar
4f27d60563 fix golangci issues; fix groupValuesKey slice reuse bug; fix group reject bug 2026-06-17 15:16:48 +03:00
Max Kotliar
3d4e8b59fd app/vmestimator: Introduce a stand alone cardinality estimator service
Fixes https://github.com/VictoriaMetrics/VictoriaMetrics/issues/10848
2026-06-17 14:20:43 +03:00
Max Kotliar
b58c73ac90 app/vmauth: refactor oidc discovery pool creation (#11123)
Initializing `oidcDP` outside or before `parseJWTUsers` would simplify its reuse with SSO implementation https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11122

PR https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11123
2026-06-17 13:52:39 +03:00
Nikolay
77efbb2e36 lib/fs: retry file deletion on NFS error
This commit adds additional file removal retries.
This is follow-up for
https://github.com/VictoriaMetrics/VictoriaMetrics/issues/9842.

According to the stack trace, NFS also could return error for
`deleteFilePath`.
This retry is safe, since part already removed from `parts.json` and
performs a skips empty directories on restart.

Fixes https://github.com/VictoriaMetrics/VictoriaMetrics/issues/11060
2026-06-17 09:16:42 +02:00
Alexander Frolov
e388e41430 app/vmagent: properly copy metadata unit value
Previously, Unit value at metrics metadata was incorrectly referenced instead of memory copy:

ed795a8443/app/vmagent/remotewrite/pendingseries.go (L209-L212)

 This commit properly copies Unit into dedicated buffer and prevents memory corruption.

Related PR https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11120
2026-06-17 09:16:04 +02:00
Max Kotliar
ed795a8443 lib/httpserver: add comments and links to tlsErrorSkipLogger
Follow-up on
64e43e59a7

See discussion in
https://github.com/VictoriaMetrics/VictoriaMetrics/issues/10538#issuecomment-4708196302
2026-06-16 18:39:21 +03:00
f41gh7
94e5955b1f make vendor-update 2026-06-16 17:01:18 +02:00
Phuong Le
5b31a047a5 lib/mergeset: make the flushCallback interval configurable
Add a configurable `flushCallbackInterval` to `mergeset.MustOpenTable`
instead of the hardcoded 10s flushCallback ticker. This lets
VictoriaLogs lower it to 1s so filter-cache invalidation keeps up with
live tailing,

See https://github.com/VictoriaMetrics/VictoriaLogs/issues/1477.
2026-06-16 15:04:39 +02:00
f41gh7
17c3fb5656 app/{vmalert,vmauth}: follow-up for 8b27a36fb5
Add additional graceful shutdown check to prevent possible race between
 configCheckInterval ticker and globalStopChannel close actions.
2026-06-16 14:56:44 +02:00
f41gh7
30133ec182 docs/changelog: update changelog entry 2026-06-16 14:37:15 +02:00
Nicholas Feinberg
8b27a36fb5 lib/promscrape: fix shutdown race
In vminsert and vmagent, on shutdown, the `globalStopCh` channel is
closed. This is intended to cause the `runScraper` loop to exit.

However, nothing gave the `globalStopCh` priority over the input
channels, `sighupCh` or `tickerCh`. This causes unpredictable behavior.
For example, if `configCheckInterval` is set to 5s but `loadConfig`
takes 6s, then there will be a new message on `tickerCh `for every
iteration of the `runScraper` loop. Go will choose one channel to pull
from at random, meaning that shutdown can take arbitrarily long.

To address this, check for `globalStopCh` *first* at the start of each
loop, ensuring that it will be reached at most one loop iteration after
the shutdown starts.

Related PR https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11107/
2026-06-16 14:35:00 +02:00
Alexander Frolov
f33cd8a937 app/vmselect: properly unmarshal metadata response
Previusly `readMetadataRows` incorrectly re-used dataBuf for `metricsmetadata.Row.Unmarshal` call. Which is not safe, since unmarshalled row takes ownership of dataBuf and it cannot be mutated.

 This commit removes dataBuf and allocates a new buffer each time for row.Unmarshal.

Related PR https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11115
2026-06-16 14:06:43 +02:00
742 changed files with 53933 additions and 19610 deletions

View File

@@ -209,13 +209,12 @@ func (wr *writeRequest) tryPushMetadata(mms []prompb.MetricMetadata) bool {
func (wr *writeRequest) copyMetadata(dst, src *prompb.MetricMetadata) {
// Direct copy for non-string fields, which are safe by value.
dst.Type = src.Type
dst.Unit = src.Unit
dst.AccountID = src.AccountID
dst.ProjectID = src.ProjectID
// Pre-allocate memory for all string fields.
neededBufLen := len(src.MetricFamilyName) + len(src.Help)
neededBufLen := len(src.MetricFamilyName) + len(src.Help) + len(src.Unit)
bufLen := len(wr.metadatabuf)
wr.metadatabuf = slicesutil.SetLength(wr.metadatabuf, bufLen+neededBufLen)
buf := wr.metadatabuf[:bufLen]
@@ -230,6 +229,11 @@ func (wr *writeRequest) copyMetadata(dst, src *prompb.MetricMetadata) {
buf = append(buf, src.Help...)
dst.Help = bytesutil.ToUnsafeString(buf[bufLen:])
// Copy Unit
bufLen = len(buf)
buf = append(buf, src.Unit...)
dst.Unit = bytesutil.ToUnsafeString(buf[bufLen:])
wr.metadatabuf = buf
}

View File

@@ -315,6 +315,11 @@ func configReload(ctx context.Context, m *manager, groupsCfg []config.Group, sig
parseFn := config.Parse
for {
select {
case <-ctx.Done():
return
default:
}
select {
case <-ctx.Done():
return

View File

@@ -840,6 +840,11 @@ func authConfigReloader(sighupCh <-chan os.Signal) {
}
for {
select {
case <-stopCh:
return
default:
}
select {
case <-stopCh:
return
@@ -906,7 +911,8 @@ func reloadAuthConfigData(data []byte) (bool, error) {
return false, fmt.Errorf("failed to parse auth config: %w", err)
}
jui, oidcDP, err := parseJWTUsers(ac)
oidcDP := &oidcDiscovererPool{}
jui, err := parseJWTUsers(ac, oidcDP)
if err != nil {
return false, fmt.Errorf("failed to parse JWT users from auth config: %w", err)
}

View File

@@ -72,9 +72,8 @@ type JWTConfig struct {
verifierPool atomic.Pointer[jwt.VerifierPool]
}
func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
func parseJWTUsers(ac *AuthConfig, oidcDP *oidcDiscovererPool) ([]*UserInfo, error) {
jui := make([]*UserInfo, 0, len(ac.Users))
oidcDP := &oidcDiscovererPool{}
uniqClaims := make(map[string]*UserInfo)
var sortedClaims []string
@@ -85,10 +84,10 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
}
if ui.AuthToken != "" || ui.BearerToken != "" || ui.Username != "" || ui.Password != "" {
return nil, nil, fmt.Errorf("auth_token, bearer_token, username and password cannot be specified if jwt is set")
return nil, fmt.Errorf("auth_token, bearer_token, username and password cannot be specified if jwt is set")
}
if len(jwtToken.PublicKeys) == 0 && len(jwtToken.PublicKeyFiles) == 0 && !jwtToken.SkipVerify && jwtToken.OIDC == nil {
return nil, nil, fmt.Errorf("jwt must contain at least a single public key, public_key_files, oidc or have skip_verify=true")
return nil, fmt.Errorf("jwt must contain at least a single public key, public_key_files, oidc or have skip_verify=true")
}
var claimsString string
sortedClaims = sortedClaims[:0]
@@ -97,7 +96,7 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
sortedClaims = append(sortedClaims, fmt.Sprintf("%s=%s", ck, cv))
pc, err := jwt.NewClaim(ck, cv)
if err != nil {
return nil, nil, fmt.Errorf("incorrect match claim, key=%q, value regex=%q: %w", ck, cv, err)
return nil, fmt.Errorf("incorrect match claim, key=%q, value regex=%q: %w", ck, cv, err)
}
parsedClaims = append(parsedClaims, pc)
}
@@ -106,7 +105,7 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
claimsString = strings.Join(sortedClaims, ",")
if oldUI, ok := uniqClaims[claimsString]; ok {
return nil, nil, fmt.Errorf("duplicate match claims=%q found for name=%q at idx=%d; the previous one is set for name=%q", claimsString, ui.Name, idx, oldUI.Name)
return nil, fmt.Errorf("duplicate match claims=%q found for name=%q at idx=%d; the previous one is set for name=%q", claimsString, ui.Name, idx, oldUI.Name)
}
uniqClaims[claimsString] = &ui
if len(jwtToken.PublicKeys) > 0 || len(jwtToken.PublicKeyFiles) > 0 {
@@ -115,7 +114,7 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
for i := range jwtToken.PublicKeys {
k, err := jwt.ParseKey([]byte(jwtToken.PublicKeys[i]))
if err != nil {
return nil, nil, err
return nil, err
}
keys = append(keys, k)
}
@@ -123,52 +122,52 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
for _, filePath := range jwtToken.PublicKeyFiles {
keyData, err := os.ReadFile(filePath)
if err != nil {
return nil, nil, fmt.Errorf("cannot read public key from file %q: %w", filePath, err)
return nil, fmt.Errorf("cannot read public key from file %q: %w", filePath, err)
}
k, err := jwt.ParseKey(keyData)
if err != nil {
return nil, nil, fmt.Errorf("cannot parse public key from file %q: %w", filePath, err)
return nil, fmt.Errorf("cannot parse public key from file %q: %w", filePath, err)
}
keys = append(keys, k)
}
vp, err := jwt.NewVerifierPool(keys)
if err != nil {
return nil, nil, err
return nil, err
}
jwtToken.verifierPool.Store(vp)
}
if jwtToken.OIDC != nil {
if len(jwtToken.PublicKeys) > 0 || len(jwtToken.PublicKeyFiles) > 0 || jwtToken.SkipVerify {
return nil, nil, fmt.Errorf("jwt with oidc cannot contain public keys or have skip_verify=true")
return nil, fmt.Errorf("jwt with oidc cannot contain public keys or have skip_verify=true")
}
if jwtToken.OIDC.Issuer == "" {
return nil, nil, fmt.Errorf("oidc issuer cannot be empty")
return nil, fmt.Errorf("oidc issuer cannot be empty")
}
isserURL, err := url.Parse(jwtToken.OIDC.Issuer)
if err != nil {
return nil, nil, fmt.Errorf("oidc issuer %q must be a valid URL", jwtToken.OIDC.Issuer)
return nil, fmt.Errorf("oidc issuer %q must be a valid URL", jwtToken.OIDC.Issuer)
}
if isserURL.Scheme != "https" && isserURL.Scheme != "http" {
return nil, nil, fmt.Errorf("oidc issuer %q must have http or https scheme", jwtToken.OIDC.Issuer)
return nil, fmt.Errorf("oidc issuer %q must have http or https scheme", jwtToken.OIDC.Issuer)
}
oidcDP.createOrAdd(ui.JWT.OIDC.Issuer, &ui.JWT.verifierPool)
}
if err := parseJWTPlaceholdersForUserInfo(&ui, true); err != nil {
return nil, nil, err
return nil, err
}
if err := ui.initURLs(); err != nil {
return nil, nil, err
return nil, err
}
metricLabels, err := ui.getMetricLabels()
if err != nil {
return nil, nil, fmt.Errorf("cannot parse metric_labels: %w", err)
return nil, fmt.Errorf("cannot parse metric_labels: %w", err)
}
ui.requests = ac.ms.GetOrCreateCounter(`vmauth_user_requests_total` + metricLabels)
ui.requestErrors = ac.ms.GetOrCreateCounter(`vmauth_user_request_errors_total` + metricLabels)
@@ -187,7 +186,7 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
rt, err := newRoundTripper(ui.TLSCAFile, ui.TLSCertFile, ui.TLSKeyFile, ui.TLSServerName, ui.TLSInsecureSkipVerify)
if err != nil {
return nil, nil, fmt.Errorf("cannot initialize HTTP RoundTripper: %w", err)
return nil, fmt.Errorf("cannot initialize HTTP RoundTripper: %w", err)
}
ui.rt = rt
@@ -200,7 +199,7 @@ func parseJWTUsers(ac *AuthConfig) ([]*UserInfo, *oidcDiscovererPool, error) {
return len(jui[i].JWT.MatchClaims) > len(jui[j].JWT.MatchClaims)
})
return jui, oidcDP, nil
return jui, nil
}
var tokenPool sync.Pool

View File

@@ -39,16 +39,14 @@ XOtclIk1uhc03oL9nOQ=
}
return
}
users, oidcDP, err := parseJWTUsers(ac)
oidcDP := &oidcDiscovererPool{}
users, err := parseJWTUsers(ac, oidcDP)
if err == nil {
t.Fatalf("expecting non-nil error; got %v", users)
}
if expErr != err.Error() {
t.Fatalf("unexpected error; got\n%q\nwant \n%q", err.Error(), expErr)
}
if oidcDP != nil {
t.Fatalf("expecting nil oidcDP; got %v", oidcDP)
}
}
// unauthorized_user cannot be used with jwt
@@ -326,7 +324,8 @@ XOtclIk1uhc03oL9nOQ=
t.Fatalf("unexpected error: %s", err)
}
jui, oidcDP, err := parseJWTUsers(ac)
oidcDP := &oidcDiscovererPool{}
jui, err := parseJWTUsers(ac, oidcDP)
if err != nil {
t.Fatalf("unexpected error: %s", err)
}

View File

@@ -0,0 +1,49 @@
package main
import (
"bytes"
"flag"
"io"
"sync"
"time"
"github.com/VictoriaMetrics/metrics"
)
var (
cardinalityMetricsWrites = metrics.NewCounter(`vmestimator_write_cardinality_metrics_total`)
cardinalityMetricsWriteDuration = metrics.NewFloatCounter(`vmestimator_write_cardinality_metrics_duration_seconds_total`)
cardinalityMetricsWriteBytes = metrics.NewCounter(`vmestimator_write_cardinality_metrics_size_bytes_total`)
cardinalityCacheMu sync.Mutex
cardinalityMetricsCacheAt time.Time
cardinalityMetricsCache []byte
cardinalityMetricsCacheTTL = flag.Duration("cardinalityMetrics.cacheTTL", time.Second*30, "Duration for caching cardinality metrics response")
cardinalityMetricsExposeAt = flag.String(`cardinalityMetrics.exposeAt`, `/metrics`, "HTTP path for exposing cardinality metrics. "+
"If set to the default /metrics, cardinality metrics are merged with regular metrics and exposed together. "+
"If set to a different path, only cardinality metrics are exposed at that endpoint. "+
"If set to an empty value, cardinality metrics are not exposed via HTTP at all.")
)
func writeCardinalityMetrics(w io.Writer, es []*estimator) {
startTime := time.Now()
cardinalityCacheMu.Lock()
if time.Since(cardinalityMetricsCacheAt) >= *cardinalityMetricsCacheTTL || *cardinalityMetricsCacheTTL == 0 {
plain := bytes.NewBuffer(cardinalityMetricsCache[:0])
for _, e := range es {
e.writeMetrics(plain)
}
cardinalityMetricsCache = plain.Bytes()
cardinalityMetricsCacheAt = time.Now()
}
cm := make([]byte, len(cardinalityMetricsCache))
copy(cm, cardinalityMetricsCache)
cardinalityCacheMu.Unlock()
_, _ = w.Write(cm)
cardinalityMetricsWrites.Inc()
cardinalityMetricsWriteDuration.Add(time.Since(startTime).Seconds())
cardinalityMetricsWriteBytes.Add(len(cm))
}

43
app/vmestimator/config.go Normal file
View File

@@ -0,0 +1,43 @@
package main
import (
"fmt"
"os"
"sort"
"time"
"gopkg.in/yaml.v2"
)
type Config struct {
Streams []EstimatorConfig `yaml:"streams"`
}
type EstimatorConfig struct {
GroupBy []string `yaml:"group_by"`
GroupLimit int `yaml:"group_limit"`
Labels map[string]string `yaml:"labels"`
Interval time.Duration `yaml:"interval"`
Buckets int `yaml:"buckets"`
HLLPrecision uint8 `yaml:"hll_precision"`
HLLSparse *bool `yaml:"hll_sparse"`
}
func loadConfig(path string) (*Config, error) {
data, err := os.ReadFile(path)
if err != nil {
return nil, fmt.Errorf("cannot read config file %q: %w", path, err)
}
var cfg Config
if err := yaml.UnmarshalStrict(data, &cfg); err != nil {
return nil, fmt.Errorf("cannot parse config file %q: %w", path, err)
}
for _, stream := range cfg.Streams {
sort.Strings(stream.GroupBy)
if stream.HLLPrecision != 0 && (stream.HLLPrecision < 4 || stream.HLLPrecision > 18) {
return nil, fmt.Errorf("invalid precision %d: must be in range [4, 18]", stream.HLLPrecision)
}
}
return &cfg, nil
}

View File

@@ -0,0 +1,508 @@
package main
import (
"fmt"
"io"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/metrics"
"github.com/axiomhq/hyperloglog"
"github.com/dgryski/go-metro"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmestimator/protoparser"
)
type estimator struct {
groupBy []string
groupByKeysLabel string
groupLimit int64
groupSize atomic.Int64
groupRejectedMu sync.Mutex
groupRejectedSketch *hyperloglog.Sketch
groupRejectedSketchPrev *hyperloglog.Sketch
buckets []*estimatorBucket
metricsSet *metrics.Set
insertTotal *metrics.Counter
stopCh chan struct{}
}
func newEstimator(cfg EstimatorConfig) (*estimator, error) {
if cfg.Interval == 0 {
cfg.Interval = time.Minute * 5
}
if cfg.GroupLimit <= 0 {
cfg.GroupLimit = 10000
}
if cfg.Buckets <= 0 {
cfg.Buckets = min(64, 2*cgroup.AvailableCPUs())
}
if cfg.HLLPrecision == 0 {
cfg.HLLPrecision = 14
}
if cfg.HLLSparse == nil {
cfg.HLLSparse = new(true)
}
metricPrefix := fmt.Sprintf("cardinality_estimate{interval=%q", cfg.Interval)
if len(cfg.Labels) > 0 {
keys := make([]string, 0, len(cfg.Labels))
for k := range cfg.Labels {
keys = append(keys, k)
}
sort.Strings(keys)
for _, k := range keys {
metricPrefix += fmt.Sprintf(",%s=%q", k, cfg.Labels[k])
}
}
groupByKeysLabel := "__global__"
if len(cfg.GroupBy) > 0 {
groupByKeysLabel = strings.Join(cfg.GroupBy, `,`)
}
e := &estimator{
groupBy: cfg.GroupBy,
groupByKeysLabel: groupByKeysLabel,
groupLimit: int64(cfg.GroupLimit),
groupRejectedSketch: mustNewGroupRejectSketch(),
groupRejectedSketchPrev: mustNewGroupRejectSketch(),
buckets: make([]*estimatorBucket, cfg.Buckets),
metricsSet: metrics.NewSet(),
stopCh: make(chan struct{}),
}
e.insertTotal = e.metricsSet.NewCounter(
fmt.Sprintf(`vmestimator_estimator_insert_total{group_by_keys=%q}`, e.groupByKeysLabel),
)
e.metricsSet.NewGauge(fmt.Sprintf(`vmestimator_estimator_group_rejected_size{group_by_keys=%q}`, e.groupByKeysLabel), func() float64 {
e.groupRejectedMu.Lock()
defer e.groupRejectedMu.Unlock()
return float64(e.groupRejectedSketch.Estimate())
})
for i := 0; i < len(e.buckets); i++ {
eb := &estimatorBucket{
groupBy: cfg.GroupBy,
extraLabels: cfg.Labels,
interval: cfg.Interval,
metricPrefix: metricPrefix,
groupByKeysLabel: groupByKeysLabel,
groupLimit: int64(cfg.GroupLimit),
groupSize: &e.groupSize,
groupRejectedMu: &e.groupRejectedMu,
groupRejectedSketch: e.groupRejectedSketch,
precision: cfg.HLLPrecision,
sparse: *cfg.HLLSparse,
}
if len(cfg.GroupBy) == 0 {
eb.sketch = eb.newSketch()
} else {
eb.groups = make(map[string]groupSketch)
eb.prevGroups = make(map[string]groupSketch)
e.metricsSet.NewGauge(fmt.Sprintf(`vmestimator_estimator_group_size{group_by_keys=%q,bucket="%d"}`, eb.groupByKeysLabel, i), func() float64 {
return float64(eb.groupSize.Load())
})
e.metricsSet.NewGauge(fmt.Sprintf(`vmestimator_estimator_group_limit{group_by_keys=%q,bucket="%d"}`, eb.groupByKeysLabel, i), func() float64 {
return float64(eb.groupLimit)
})
}
e.buckets[i] = eb
}
go e.runRotation(cfg.Interval)
metrics.RegisterSet(e.metricsSet)
return e, nil
}
func (e *estimator) stop() {
close(e.stopCh)
e.metricsSet.UnregisterAllMetrics()
}
var groupValuesPool = sync.Pool{}
func getGroupValuesKeySlice() *[]byte {
v0 := groupValuesPool.Get()
if v0 == nil {
v := make([]byte, 128)
return &v
}
return v0.(*[]byte)
}
func putGroupValuesSlice(key *[]byte) {
if key == nil {
return
}
*key = (*key)[:0]
groupValuesPool.Put(key)
}
func (e *estimator) insertMany(tss []protoparser.TimeSerie) {
bucketsNum := uint64(len(e.buckets))
groupValuesKeyP := getGroupValuesKeySlice()
groupValuesKey := *groupValuesKeyP
defer func() {
*groupValuesKeyP = groupValuesKey
putGroupValuesSlice(groupValuesKeyP)
}()
groupValues := make([]string, len(e.groupBy))
var cnt int
for _, ts := range tss {
if len(e.groupBy) == 0 {
i := int(ts.Fingerprint % bucketsNum)
e.buckets[i].insert(ts, "", nil)
cnt++
continue
}
groupValuesKey = groupValuesKey[:0]
clear(groupValues)
var hasNames bool
for i, labelName := range e.groupBy {
if i > 0 {
groupValuesKey = append(groupValuesKey, ',')
}
for _, l := range ts.GroupLabels {
if l.Name == labelName {
hasNames = true
groupValuesKey = append(groupValuesKey, l.Value...)
groupValues[i] = l.Value
break
}
}
}
// time series does not contribute to this groupBy
if !hasNames {
continue
}
i := int(hash(groupValuesKey) % bucketsNum)
e.buckets[i].insert(ts, bytesutil.ToUnsafeString(groupValuesKey), groupValues)
cnt++
}
e.insertTotal.Add(cnt)
}
func (e *estimator) reset() {
e.groupSize.Store(0)
for _, b := range e.buckets {
b.reset()
}
e.groupRejectedMu.Lock()
e.groupRejectedSketch.Reset()
e.groupRejectedMu.Unlock()
}
func (e *estimator) writeMetrics(w io.Writer) {
eb0 := e.buckets[0]
if len(e.groupBy) == 0 {
formatBuf := make([]byte, 0, 1024)
resSK := eb0.newSketch()
for _, eb := range e.buckets {
eb.writeNoGroupMetric(resSK)
}
formatBuf = append(formatBuf, eb0.metricPrefix...)
formatBuf = append(formatBuf, `,group_by_keys="__global__"} `...)
formatBuf = strconv.AppendUint(formatBuf, resSK.Estimate(), 10)
formatBuf = append(formatBuf, "\n"...)
if _, err := w.Write(formatBuf); err != nil {
logger.Errorf("writing metrics failed: %s; written cardinality metrics might be incomplete or invalid", err)
}
return
}
formatBuf := make([]byte, 0, 16384)
formatBuf = append(formatBuf, eb0.metricPrefix...)
formatBuf = append(formatBuf, `,group_by_keys="`...)
formatBuf = append(formatBuf, eb0.groupByKeysLabel...)
formatBuf = append(formatBuf, `",group_by_values=`...)
prefixLen := len(formatBuf)
resSK := eb0.newSketch()
for _, eb := range e.buckets {
formatBuf = eb.writeGroupMetrics(w, resSK, formatBuf[:prefixLen])
}
groupSize := e.groupSize.Load()
if groupSize >= int64(float64(e.groupLimit)*0.8) {
e.groupRejectedMu.Lock()
res := mustNewGroupRejectSketch()
if err := res.Merge(e.groupRejectedSketch); err != nil {
logger.Fatalf("BUG: groupRejectedSketch merge failed: %s", err)
}
if err := res.Merge(e.groupRejectedSketchPrev); err != nil {
logger.Fatalf("BUG: groupRejectedSketchPrev merge failed: %s", err)
}
e.groupRejectedMu.Unlock()
groupSize += int64(res.Estimate())
}
formatBuf = formatBuf[:0]
formatBuf = append(formatBuf, eb0.metricPrefix...)
formatBuf = append(formatBuf, `,group_by_keys="__group__",group_by_values="`...)
formatBuf = append(formatBuf, eb0.groupByKeysLabel...)
formatBuf = append(formatBuf, `"} `...)
formatBuf = strconv.AppendInt(formatBuf, groupSize, 10)
formatBuf = append(formatBuf, "\n"...)
if _, err := w.Write(formatBuf); err != nil {
logger.Errorf("writing metrics failed: %s; written cardinality metrics might be incomplete or invalid", err)
}
}
func (e *estimator) runRotation(interval time.Duration) {
t := time.NewTicker(interval / 2)
defer t.Stop()
for {
select {
case <-t.C:
e.rotate()
case <-e.stopCh:
return
}
}
}
func (e *estimator) rotate() {
e.groupSize.Store(0)
var wg sync.WaitGroup
for i := range e.buckets {
wg.Go(e.buckets[i].rotate)
}
wg.Wait()
e.groupRejectedMu.Lock()
prevSK := e.groupRejectedSketchPrev
prevSK.Reset()
e.groupRejectedSketchPrev = e.groupRejectedSketch
e.groupRejectedSketch = prevSK
e.groupRejectedMu.Unlock()
}
type estimatorBucket struct {
mu sync.Mutex
groupBy []string
groupLimit int64
extraLabels map[string]string
interval time.Duration
metricPrefix string
groupByKeysLabel string
precision uint8
sparse bool
sketch *hyperloglog.Sketch
prevSketch *hyperloglog.Sketch
groupSize *atomic.Int64
groups map[string]groupSketch
prevGroups map[string]groupSketch
groupRejectedMu *sync.Mutex
groupRejectedSketch *hyperloglog.Sketch
}
func (eb *estimatorBucket) String() string {
return fmt.Sprintf(
"interval: %s; group_by: %v; extra_labels: %v", eb.interval, eb.groupBy, eb.extraLabels)
}
func (eb *estimatorBucket) reset() {
eb.mu.Lock()
defer eb.mu.Unlock()
if len(eb.groupBy) == 0 {
eb.prevSketch.Reset()
eb.sketch.Reset()
return
}
eb.groups = make(map[string]groupSketch)
eb.prevGroups = make(map[string]groupSketch)
}
func (eb *estimatorBucket) rotate() {
if len(eb.groupBy) == 0 {
eb.mu.Lock()
eb.prevSketch = eb.sketch
eb.sketch = eb.newSketch()
eb.mu.Unlock()
return
}
eb.mu.Lock()
eb.prevGroups = eb.groups
eb.groups = make(map[string]groupSketch, len(eb.groups))
eb.mu.Unlock()
eb.groupSize.Add(int64(len(eb.prevGroups)))
}
func (eb *estimatorBucket) insert(ts protoparser.TimeSerie, groupValuesKey string, groupValues []string) {
eb.mu.Lock()
defer eb.mu.Unlock()
if len(eb.groupBy) == 0 {
eb.sketch.InsertHash(ts.Fingerprint)
return
}
gsk, ok := eb.groups[groupValuesKey]
if !ok {
if _, ok := eb.prevGroups[groupValuesKey]; !ok {
groupSize := eb.groupSize.Load()
if groupSize+1 > eb.groupLimit {
eb.groupRejectedMu.Lock()
eb.groupRejectedSketch.InsertHash(hash([]byte(groupValuesKey)))
eb.groupRejectedMu.Unlock()
return
}
eb.groupSize.Add(1)
}
formatBuf := make([]byte, 0, 1024)
formatBuf = strconv.AppendQuote(formatBuf, groupValuesKey)
for i := range groupValues {
formatBuf = append(formatBuf, ',')
if eb.groupBy[i] == `__name__` {
formatBuf = append(formatBuf, `by__name__`...)
} else {
formatBuf = append(formatBuf, `by_`...)
formatBuf = append(formatBuf, eb.groupBy[i]...)
}
formatBuf = append(formatBuf, '=')
formatBuf = strconv.AppendQuote(formatBuf, groupValues[i])
}
formatBuf = append(formatBuf, `} `...)
gsk = groupSketch{
groupValueLabels: bytesutil.ToUnsafeString(formatBuf),
Sketch: eb.newSketch(),
}
eb.groups[strings.Clone(groupValuesKey)] = gsk
}
gsk.InsertHash(ts.Fingerprint)
}
func (eb *estimatorBucket) writeNoGroupMetric(res *hyperloglog.Sketch) {
eb.mu.Lock()
defer eb.mu.Unlock()
eb.mergeSketches(eb.sketch, eb.prevSketch, res)
}
func (eb *estimatorBucket) writeGroupMetrics(w io.Writer, res *hyperloglog.Sketch, formatBuf []byte) []byte {
eb.mu.Lock()
defer eb.mu.Unlock()
prefixLen := len(formatBuf)
for valuesKey, gsk := range eb.groups {
res.Reset()
formatBuf = formatBuf[:prefixLen]
formatBuf = append(formatBuf, gsk.groupValueLabels...)
eb.mergeSketches(gsk.Sketch, eb.prevGroups[valuesKey].Sketch, res)
formatBuf = strconv.AppendUint(formatBuf, res.Estimate(), 10)
formatBuf = append(formatBuf, "\n"...)
if _, err := w.Write(formatBuf); err != nil {
logger.Errorf("writing metrics failed: %s; written cardinality metrics might be incomplete or invalid", err)
}
}
for valuesKey := range eb.prevGroups {
if _, ok := eb.groups[valuesKey]; ok {
continue
}
res.Reset()
formatBuf = formatBuf[:prefixLen]
gsk := eb.prevGroups[valuesKey]
formatBuf = append(formatBuf, gsk.groupValueLabels...)
eb.mergeSketches(nil, eb.prevGroups[valuesKey].Sketch, res)
formatBuf = strconv.AppendUint(formatBuf, res.Estimate(), 10)
formatBuf = append(formatBuf, "\n"...)
if _, err := w.Write(formatBuf); err != nil {
logger.Errorf("writing metrics failed: %s; written cardinality metrics might be incomplete or invalid", err)
}
}
return formatBuf[:prefixLen]
}
func (eb *estimatorBucket) mergeSketches(cur, prev, res *hyperloglog.Sketch) {
if err := res.Merge(cur); err != nil {
panic(err)
}
if prev != nil {
if err := res.Merge(prev); err != nil {
panic(err)
}
}
}
func (eb *estimatorBucket) newSketch() *hyperloglog.Sketch {
return mustNewSketch(eb.precision, eb.sparse)
}
type groupSketch struct {
groupValueLabels string
*hyperloglog.Sketch
}
func mustNewGroupRejectSketch() *hyperloglog.Sketch {
return mustNewSketch(10, true)
}
func mustNewSketch(precision uint8, sparse bool) *hyperloglog.Sketch {
sk, err := hyperloglog.NewSketch(precision, sparse)
if err != nil {
panic(fmt.Sprintf("cannot create HLL sketch with precision=%d and sparse=%v: %s", precision, sparse, err))
}
return sk
}
func hash(v []byte) uint64 {
return metro.Hash64(v, 1337)
}

View File

@@ -0,0 +1,274 @@
package main
import (
"fmt"
"io"
"testing"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmestimator/protoparser"
)
func BenchmarkEstimator_WriteMetrics(b *testing.B) {
b.Run("NoGroup/NoPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{Interval: time.Hour})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 5_000, 0)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
b.Run("NoGroup/WithPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{Interval: time.Hour})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 5_000, 0)
for _, eb := range e.buckets {
eb.rotate()
}
insertSeriesIntoEstimator(e, 5_000, 0)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
b.Run("Group100/NoPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 5_000, 100)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
b.Run("Group100/WithPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 5_000, 100)
for _, eb := range e.buckets {
eb.rotate()
}
insertSeriesIntoEstimator(e, 5_000, 100)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
b.Run("Group10k/NoPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 50_000, 10_000)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
b.Run("Group10k/WithPrev", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
insertSeriesIntoEstimator(e, 50_000, 10_000)
for _, eb := range e.buckets {
eb.rotate()
}
insertSeriesIntoEstimator(e, 50_000, 10_000)
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
e.writeMetrics(io.Discard)
}
})
}
func BenchmarkEstimator_InsertManyParallel(b *testing.B) {
b.Run("NoGroup", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{Interval: time.Hour})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
b.RunParallel(func(pb *testing.PB) {
var i uint64
for pb.Next() {
e.insertMany([]protoparser.TimeSerie{{Fingerprint: i}})
i++
}
})
})
b.Run("Group100", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
b.RunParallel(func(pb *testing.PB) {
var i uint64
for pb.Next() {
e.insertMany([]protoparser.TimeSerie{{
GroupLabels: []protoparser.Label{{Name: "groupLabel", Value: fmt.Sprintf("%d", i%100)}},
Fingerprint: i,
}})
i++
}
})
})
b.Run("Group10k", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
b.RunParallel(func(pb *testing.PB) {
var i uint64
for pb.Next() {
e.insertMany([]protoparser.TimeSerie{{
GroupLabels: []protoparser.Label{{Name: "groupLabel", Value: fmt.Sprintf("%d", i%10_000)}},
Fingerprint: i,
}})
i++
}
})
})
b.Run("Group100k", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{
GroupBy: []string{"groupLabel"},
Interval: time.Hour,
})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
b.RunParallel(func(pb *testing.PB) {
var i uint64
for pb.Next() {
e.insertMany([]protoparser.TimeSerie{{
GroupLabels: []protoparser.Label{{Name: "groupLabel", Value: fmt.Sprintf("%d", i%100_000)}},
Fingerprint: i,
}})
i++
}
})
})
}
// BenchmarkEstimator_InsertRotateCycle benchmarks the insert→rotate→insert cycle
// for the global (no-group) estimator in two HLL regimes:
// - Sparse: 1 000 series per interval (sketch stays in sparse mode)
// - Normal: 30 000 series per interval (sketch converts to dense mode)
func BenchmarkEstimator_InsertRotateCycle(b *testing.B) {
b.Run("SparseHLL", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{Interval: time.Hour})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
insertSeriesIntoEstimator(e, 1_000, 0)
e.rotate()
}
})
b.Run("NormalHLL", func(b *testing.B) {
e, err := newEstimator(EstimatorConfig{Interval: time.Hour})
if err != nil {
b.Fatalf("newEstimator: %v", err)
}
defer e.stop()
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
insertSeriesIntoEstimator(e, 30_000, 0)
e.rotate()
}
})
}
// insertSeriesIntoEstimator inserts numSeries time series into e.
// When groupsNum > 0 each series gets a "groupLabel" cycling through groupsNum values.
func insertSeriesIntoEstimator(e *estimator, numSeries, groupsNum int) {
for i := 0; i < numSeries; i++ {
var labels []protoparser.Label
if groupsNum > 0 {
labels = append(labels, protoparser.Label{
Name: "groupLabel",
Value: fmt.Sprintf("%d", i%groupsNum),
})
}
e.insertMany([]protoparser.TimeSerie{
{
GroupLabels: labels,
Fingerprint: hash([]byte(fmt.Sprintf("foobarbaz%d", i))),
},
})
}
}

View File

@@ -0,0 +1,595 @@
package main
import (
"bytes"
"encoding/binary"
"fmt"
"sort"
"strings"
"testing"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmestimator/protoparser"
)
func TestGlobalEstimate(t *testing.T) {
genCard := func(cardinality int, seed string) func(e *estimator) {
return func(e *estimator) {
var tss []protoparser.TimeSerie
fpBuf := make([]byte, 8, 8+len(seed))
for i := 0; i < cardinality; i++ {
binary.LittleEndian.PutUint64(fpBuf[:8], uint64(i))
fpBuf = append(fpBuf, seed...)
tss = append(tss, protoparser.TimeSerie{
Fingerprint: hash(fpBuf[:]),
})
if i%10 == 0 {
e.insertMany(tss)
tss = tss[:0]
}
}
if len(tss) > 0 {
e.insertMany(tss)
}
}
}
f := func(gen func(e *estimator), expMetric string) {
t.Helper()
cfg := EstimatorConfig{
Interval: time.Minute * 10,
Buckets: 5,
}
e, err := newEstimator(cfg)
if err != nil {
t.Fatalf("failed to create new estimator: %v", err)
}
defer e.stop()
gen(e)
if len(e.buckets) != cfg.Buckets {
t.Fatalf("expected buckets length to be %d but got %d", cfg.Buckets, len(e.buckets))
}
for i, eb := range e.buckets {
if len(eb.groupBy) > 0 {
t.Fatalf("expected bucket %d groupBy length to be 0 but got %d", i, len(eb.groupBy))
}
if eb.groups != nil {
t.Fatalf("expected bucket %d groups length to be 0 but got %d", i, len(eb.groups))
}
if eb.groupSize.Load() != 0 {
t.Fatalf("expected bucket %d groupSize to be 0 but got %d", i, eb.groupSize.Load())
}
}
buf := bytes.NewBuffer(nil)
e.writeMetrics(buf)
if strings.TrimSpace(buf.String()) != expMetric {
t.Fatalf("\nexpected:\n%s\n\ngot:\n%s", expMetric, buf.String())
}
}
// no previous
f(genCard(0, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genCard(1, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 1`)
f(genCard(10, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 10`)
f(genCard(100, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 100`)
f(genCard(1000, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 1000`)
f(genCard(5000, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 4998`)
f(genCard(10000, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 9920`)
f(genCard(100000, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 99658`)
f(genCard(500000, ""), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 496552`)
// rotate once
genRotateOnce := func(cardinality int) func(e *estimator) {
return func(e *estimator) {
genCard(cardinality, "")(e)
e.rotate()
}
}
f(genRotateOnce(0), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateOnce(1), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 1`)
f(genRotateOnce(10), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 10`)
f(genRotateOnce(100), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 100`)
f(genRotateOnce(1000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 1000`)
f(genRotateOnce(5000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 4998`)
f(genRotateOnce(10000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 9920`)
f(genRotateOnce(100000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 99658`)
f(genRotateOnce(500000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 496552`)
// insert, rotate insert the same
genInsertRotateInsertSameOnce := func(cardinality int) func(e *estimator) {
return func(e *estimator) {
genCard(cardinality/2, "")(e)
e.rotate()
genCard(cardinality/2, "")(e)
}
}
f(genInsertRotateInsertSameOnce(0), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genInsertRotateInsertSameOnce(1), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genInsertRotateInsertSameOnce(10), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 5`)
f(genInsertRotateInsertSameOnce(100), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 50`)
f(genInsertRotateInsertSameOnce(1000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 500`)
f(genInsertRotateInsertSameOnce(5000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 2499`)
f(genInsertRotateInsertSameOnce(10000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 4998`)
f(genInsertRotateInsertSameOnce(100000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 49529`)
f(genInsertRotateInsertSameOnce(200000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 99658`)
// insert, rotate insert
genInsertRotateInsertOnce := func(cardinality int) func(e *estimator) {
return func(e *estimator) {
genCard(cardinality/2, "one")(e)
e.rotate()
genCard(cardinality/2, "two")(e)
}
}
f(genInsertRotateInsertOnce(0), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genInsertRotateInsertOnce(1), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genInsertRotateInsertOnce(10), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 10`)
f(genInsertRotateInsertOnce(100), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 100`)
f(genInsertRotateInsertOnce(1000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 1000`)
f(genInsertRotateInsertOnce(5000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 5000`)
f(genInsertRotateInsertOnce(10000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 10058`)
f(genInsertRotateInsertOnce(100000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 99543`)
f(genInsertRotateInsertOnce(200000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 198814`)
// insert, rotate insert
genRotateTwoTimes := func(cardinality int) func(e *estimator) {
return func(e *estimator) {
genCard(cardinality, "")(e)
e.rotate()
e.rotate()
}
}
f(genRotateTwoTimes(0), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(1), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(10), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(100), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(1000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(5000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(10000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(100000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
f(genRotateTwoTimes(500000), `cardinality_estimate{interval="10m0s",group_by_keys="__global__"} 0`)
}
func TestGroupEstimate(t *testing.T) {
genCard := func(fooCard, barCard, bazCard int, seed string) func(e *estimator) {
return func(e *estimator) {
var tss []protoparser.TimeSerie
for fooI := 0; fooI < max(1, fooCard); fooI++ {
for barI := 0; barI < max(1, barCard); barI++ {
for bazI := 0; bazI < max(1, bazCard); bazI++ {
ts := protoparser.TimeSerie{}
ts.GroupLabels = append(ts.GroupLabels, protoparser.Label{Name: "__name__", Value: "the_metric_name"})
if fooCard > 0 {
ts.GroupLabels = append(ts.GroupLabels, protoparser.Label{Name: "foo", Value: fmt.Sprintf("%s%d", seed, fooI)})
}
if barCard > 0 {
ts.GroupLabels = append(ts.GroupLabels, protoparser.Label{Name: "bar", Value: fmt.Sprintf("%s%d", seed, barI)})
}
if bazCard > 0 {
ts.GroupLabels = append(ts.GroupLabels, protoparser.Label{Name: "baz", Value: fmt.Sprintf("%s%d", seed, bazI)})
}
var fpBuf []byte
for _, l := range ts.GroupLabels {
fpBuf = append(fpBuf, l.Name...)
fpBuf = append(fpBuf, '=')
fpBuf = append(fpBuf, l.Value...)
fpBuf = append(fpBuf, ',')
}
fpBuf = append(fpBuf, seed...)
ts.Fingerprint = hash(fpBuf)
tss = append(tss, ts)
}
}
}
e.insertMany(tss)
}
}
f := func(groupBy []string, gen func(e *estimator), expMetrics string) {
t.Helper()
cfg := EstimatorConfig{
Interval: time.Minute * 10,
GroupBy: groupBy,
Buckets: 5,
}
e, err := newEstimator(cfg)
if err != nil {
t.Fatalf("failed to create new estimator: %v", err)
}
defer e.stop()
gen(e)
if len(e.buckets) != cfg.Buckets {
t.Fatalf("expected buckets length to be %d but got %d", cfg.Buckets, len(e.buckets))
}
for i, eb := range e.buckets {
if eb.sketch != nil {
t.Fatalf("expected bucket %d sketch to be nil", i)
}
if eb.prevSketch != nil {
t.Fatalf("expected bucket %d prevSketch to be nil", i)
}
}
buf := bytes.NewBuffer(nil)
e.writeMetrics(buf)
lines := strings.Split(strings.TrimSpace(buf.String()), "\n")
sort.Strings(lines)
actMetrics := "\n" + strings.Join(lines, "\n")
if expMetrics != actMetrics {
t.Fatalf("\nexpected:\n%s\n\ngot:\n%s", expMetrics, actMetrics)
}
}
// group by metric name
f([]string{"__name__"}, genCard(10, 10, 10, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="__name__"} 1
cardinality_estimate{interval="10m0s",group_by_keys="__name__",group_by_values="the_metric_name",by__name__="the_metric_name"} 1000`,
)
// time series does not contribute to a group
f([]string{"foo"}, genCard(0, 10, 10, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 0`,
)
f([]string{"foo", "bar"}, genCard(0, 0, 10, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 0`,
)
// group by one label
f([]string{"foo"}, genCard(1, 1, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1`,
)
f([]string{"foo"}, genCard(1, 2, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 2`,
)
f([]string{"foo"}, genCard(1, 10, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 10`,
)
f([]string{"foo"}, genCard(1, 100, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 100`,
)
f([]string{"foo"}, genCard(1, 1000, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1000`,
)
f([]string{"foo"}, genCard(1, 10000, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 9957`,
)
f([]string{"foo"}, genCard(1, 50000, 0, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 50387`,
)
f([]string{"foo"}, genCard(1, 1, 1, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1`,
)
f([]string{"foo"}, genCard(1, 2, 2, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 4`,
)
f([]string{"foo"}, genCard(1, 10, 10, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 100`,
)
f([]string{"foo"}, genCard(1, 100, 100, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 9954`,
)
f([]string{"foo"}, genCard(1, 1000, 1000, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1013124`,
)
// group by one label, rotate
genCardRotate := func(fooCard, barCard, bazCard int, seed string) func(e *estimator) {
return func(e *estimator) {
genCard(fooCard, barCard, bazCard, seed)(e)
e.rotate()
}
}
f([]string{"foo"}, genCardRotate(1, 10, 10, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 100`,
)
f([]string{"foo"}, genCardRotate(1, 1000, 1000, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1013124`,
)
// group by one label, rotate, insert same
genCardRotateInsertSame := func(barCard, bazCard int) func(e *estimator) {
return func(e *estimator) {
genCard(1, barCard, bazCard, "")(e)
e.rotate()
genCard(1, barCard, bazCard, "")(e)
}
}
f([]string{"foo"}, genCardRotateInsertSame(10, 10), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 100`,
)
f([]string{"foo"}, genCardRotateInsertSame(1000, 1000), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="0",by_foo="0"} 1013124`,
)
// group by one label, rotate, insert diff
genCardRotateInsertDiff := func(barCard, bazCard int) func(e *estimator) {
return func(e *estimator) {
genCard(1, barCard, bazCard, "one")(e)
e.rotate()
genCard(1, barCard, bazCard, "two")(e)
}
}
f([]string{"foo"}, genCardRotateInsertDiff(10, 10), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 2
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="one0",by_foo="one0"} 100
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="two0",by_foo="two0"} 100`,
)
f([]string{"foo"}, genCardRotateInsertDiff(1000, 1000), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 2
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="one0",by_foo="one0"} 995153
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="two0",by_foo="two0"} 992158`,
)
// group by one label, rotate, insert diff
genCardRotateTwice := func(barCard, bazCard int) func(e *estimator) {
return func(e *estimator) {
genCard(1, barCard, bazCard, "one")(e)
e.rotate()
e.rotate()
}
}
f([]string{"foo"}, genCardRotateTwice(10, 10), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 0`,
)
f([]string{"foo"}, genCardRotateTwice(1000, 1000), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 0`,
)
// group by two labels
f([]string{"foo", "bar"}, genCard(1, 1, 1000, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,0",by_foo="0",by_bar="0"} 1000`,
)
f([]string{"foo", "bar"}, genCard(2, 1, 1000, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 2
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,0",by_foo="0",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,0",by_foo="1",by_bar="0"} 1000`,
)
f([]string{"foo", "bar"}, genCard(2, 2, 1000, ""), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 4
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,0",by_foo="0",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,1",by_foo="0",by_bar="1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,0",by_foo="1",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,1",by_foo="1",by_bar="1"} 1000`,
)
// group by two labels, rotate
genCardTwoLabelsRotate := func() func(e *estimator) {
return func(e *estimator) {
genCard(2, 2, 1000, "")(e)
e.rotate()
}
}
f([]string{"foo", "bar"}, genCardTwoLabelsRotate(), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 4
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,0",by_foo="0",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,1",by_foo="0",by_bar="1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,0",by_foo="1",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,1",by_foo="1",by_bar="1"} 1000`,
)
// group by two labels, rotate, insert same
genCardTwoLabelsRotateInsertSame := func() func(e *estimator) {
return func(e *estimator) {
genCard(2, 2, 1000, "")(e)
e.rotate()
genCard(2, 2, 1000, "")(e)
}
}
f([]string{"foo", "bar"}, genCardTwoLabelsRotateInsertSame(), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 4
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,0",by_foo="0",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="0,1",by_foo="0",by_bar="1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,0",by_foo="1",by_bar="0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="1,1",by_foo="1",by_bar="1"} 1000`,
)
// group by two labels, rotate, insert diff
genCardTwoLabelsRotateInsertDiff := func() func(e *estimator) {
return func(e *estimator) {
genCard(2, 2, 1000, "one")(e)
e.rotate()
genCard(2, 2, 1000, "two")(e)
}
}
f([]string{"foo", "bar"}, genCardTwoLabelsRotateInsertDiff(), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 8
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="one0,one0",by_foo="one0",by_bar="one0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="one0,one1",by_foo="one0",by_bar="one1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="one1,one0",by_foo="one1",by_bar="one0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="one1,one1",by_foo="one1",by_bar="one1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="two0,two0",by_foo="two0",by_bar="two0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="two0,two1",by_foo="two0",by_bar="two1"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="two1,two0",by_foo="two1",by_bar="two0"} 1000
cardinality_estimate{interval="10m0s",group_by_keys="foo,bar",group_by_values="two1,two1",by_foo="two1",by_bar="two1"} 1000`,
)
// group by two labels, rotate, insert diff
genCardTwoLabelsRotateTwice := func() func(e *estimator) {
return func(e *estimator) {
genCard(2, 2, 1000, "one")(e)
e.rotate()
e.rotate()
}
}
f([]string{"foo", "bar"}, genCardTwoLabelsRotateTwice(), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo,bar"} 0`,
)
// quote values: label values with special characters must be properly escaped
genSpecialCard := func(fooVal string) func(e *estimator) {
return func(e *estimator) {
e.insertMany([]protoparser.TimeSerie{
{
GroupLabels: []protoparser.Label{{Name: "foo", Value: fooVal}},
Fingerprint: hash([]byte("foo=" + fooVal + ",")),
},
})
}
}
// double quote in value
f([]string{"foo"}, genSpecialCard(`a"b`), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a\"b",by_foo="a\"b"} 1`,
)
f([]string{"foo"}, genSpecialCard(`a\b`), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a\\b",by_foo="a\\b"} 1`,
)
f([]string{"foo"}, genSpecialCard("a\nb"), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a\nb",by_foo="a\nb"} 1`,
)
f([]string{"foo"}, genSpecialCard("a\tb"), `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a\tb",by_foo="a\tb"} 1`,
)
}
func TestGroupEstimateGroupLimit(t *testing.T) {
makeTS := func(fooVal string) protoparser.TimeSerie {
return protoparser.TimeSerie{
GroupLabels: []protoparser.Label{{Name: "foo", Value: fooVal}},
Fingerprint: hash([]byte("foo=" + fooVal + ",")),
}
}
f := func(groupLimit int, gen func(e *estimator), expRejected int, expMetrics string) {
t.Helper()
cfg := EstimatorConfig{
Interval: time.Minute * 10,
GroupBy: []string{"foo"},
GroupLimit: groupLimit,
Buckets: 3,
}
e, err := newEstimator(cfg)
if err != nil {
t.Fatalf("failed to create new estimator: %v", err)
}
defer e.stop()
gen(e)
buf := bytes.NewBuffer(nil)
e.writeMetrics(buf)
lines := strings.Split(strings.TrimSpace(buf.String()), "\n")
sort.Strings(lines)
actMetrics := "\n" + strings.Join(lines, "\n")
if expMetrics != actMetrics {
t.Fatalf("\nexpected:\n%s\n\ngot:\n%s", expMetrics, actMetrics)
}
var actRejected int
if e.buckets[0].groupRejectedSketch != nil {
actRejected = int(e.buckets[0].groupRejectedSketch.Estimate())
}
if expRejected != actRejected {
t.Fatalf("rejected expected: %d; got: %d", expRejected, actRejected)
}
}
// all groups accepted
f(3, func(e *estimator) {
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("b"), makeTS("c")})
}, 0, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 3
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a",by_foo="a"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="b",by_foo="b"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="c",by_foo="c"} 1`,
)
// 2 groups only accepted
f(2, func(e *estimator) {
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("b"), makeTS("c")})
}, 1, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 3
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a",by_foo="a"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="b",by_foo="b"} 1`,
)
// one group only accepted
f(1, func(e *estimator) {
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("b"), makeTS("c")})
}, 2, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 3
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a",by_foo="a"} 1`,
)
// after rotate: groups in prevGroups bypass the limit; new groups are still checked
f(2, func(e *estimator) {
// fills limit
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("b")})
e.rotate()
// "a" bypasses, "c" rejected
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("c")})
}, 1, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 3
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a",by_foo="a"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="b",by_foo="b"} 1`,
)
// after rotate: new group accepted when remaining capacity allows
f(3, func(e *estimator) {
// 2 groups, limit=3
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("b")})
e.rotate()
// "a" bypasses, "c" accepted (2+1=3 <= 3)
e.insertMany([]protoparser.TimeSerie{makeTS("a"), makeTS("c")})
}, 0, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 3
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a",by_foo="a"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="b",by_foo="b"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="c",by_foo="c"} 1`,
)
// reject 100
f(3, func(e *estimator) {
var tss []protoparser.TimeSerie
for i := 0; i < 103; i++ {
tss = append(tss, makeTS(fmt.Sprintf("a%d", i)))
}
e.insertMany(tss)
}, 100, `
cardinality_estimate{interval="10m0s",group_by_keys="__group__",group_by_values="foo"} 103
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a0",by_foo="a0"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a1",by_foo="a1"} 1
cardinality_estimate{interval="10m0s",group_by_keys="foo",group_by_values="a2",by_foo="a2"} 1`,
)
}

123
app/vmestimator/main.go Normal file
View File

@@ -0,0 +1,123 @@
package main
import (
"flag"
"io"
"net/http"
"os"
"strings"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmestimator/protoparser"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/buildinfo"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/envflag"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/httpserver"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/pushmetrics"
"github.com/VictoriaMetrics/metrics"
)
var (
httpListenAddrs = flagutil.NewArrayString("httpListenAddr", "TCP address to listen for incoming HTTP requests")
configPath = flag.String("config", "config.yaml", "Path to YAML configuration file")
prometheusWriteRequests = metrics.NewCounter(`vmestimator_http_requests_total{path="/api/v1/write", protocol="promremotewrite"}`)
)
func main() {
flag.CommandLine.SetOutput(os.Stdout)
envflag.Parse()
buildinfo.Init()
logger.Init()
cfg, err := loadConfig(*configPath)
if err != nil {
logger.Fatalf("cannot load config: %v", err)
}
estimators := make([]*estimator, 0, len(cfg.Streams))
for _, ec := range cfg.Streams {
e, err := newEstimator(ec)
if err != nil {
logger.Fatalf("cannot create estimator: %v", err)
}
estimators = append(estimators, e)
}
if *cardinalityMetricsExposeAt == `/metrics` {
metrics.RegisterMetricsWriter(func(w io.Writer) {
writeCardinalityMetrics(w, estimators)
})
}
groupLabelsMap := make(map[string]struct{})
for _, e := range estimators {
for _, l := range e.groupBy {
groupLabelsMap[l] = struct{}{}
}
}
groupLabels := make([]string, 0, len(groupLabelsMap))
for k := range groupLabelsMap {
groupLabels = append(groupLabels, k)
}
listenAddrs := *httpListenAddrs
if len(listenAddrs) == 0 {
listenAddrs = []string{":8490"}
}
logger.Infof("starting vmestimator at %q", listenAddrs)
startTime := time.Now()
go httpserver.Serve(listenAddrs, func(w http.ResponseWriter, r *http.Request) bool {
cmPath := *cardinalityMetricsExposeAt
if cmPath != "/metrics" && cmPath != "" && r.URL.Path == cmPath {
w.WriteHeader(http.StatusOK)
writeCardinalityMetrics(w, estimators)
return true
}
path, _ := strings.CutPrefix(r.URL.Path, `/cardinality`)
switch path {
case "/api/v1/write":
prometheusWriteRequests.Inc()
err := protoparser.Parse(r.Body, groupLabels, func(tss []protoparser.TimeSerie) {
for _, e := range estimators {
e.insertMany(tss)
}
})
if err != nil {
httpserver.Errorf(w, r, "error parsing remote write request: %s", err)
return true
}
w.WriteHeader(http.StatusNoContent)
return true
case "/reset":
for _, e := range estimators {
e.reset()
}
w.WriteHeader(http.StatusOK)
return true
}
return false
}, httpserver.ServeOptions{})
logger.Infof("started vmestimator in %.3f seconds", time.Since(startTime).Seconds())
pushmetrics.Init()
sig := procutil.WaitForSigterm()
logger.Infof("received signal %s", sig)
pushmetrics.Stop()
logger.Infof("gracefully shutting down webservice at %q", listenAddrs)
if err := httpserver.Stop(listenAddrs); err != nil {
logger.Errorf("cannot stop http server: %s", err)
}
for _, e := range estimators {
e.stop()
}
logger.Infof("shutting down vmestimator")
}

View File

@@ -0,0 +1,78 @@
package protoparser
import (
"fmt"
"io"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding/snappy"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/protoparserutil"
"github.com/VictoriaMetrics/metrics"
)
var maxInsertRequestSize = flagutil.NewBytes("maxInsertRequestSize", 32*1024*1024, "The maximum size in bytes of a single Prometheus remote_write API request")
// Parse parses Prometheus remote_write message from reader and calls callback for the parsed timeseries.
//
// callback shouldn't hold tss after returning.
func Parse(r io.Reader, groupLabels []string, callback func(tss []TimeSerie)) error {
startTime := fasttime.UnixTimestamp()
readCalls.Inc()
err := protoparserutil.ReadUncompressedData(r, "", maxInsertRequestSize, func(data []byte) error {
return parseRequestBody(data, groupLabels, callback)
})
if err != nil {
readErrors.Inc()
return fmt.Errorf("cannot read prometheus remote_write data from client in %d seconds: %w", fasttime.UnixTimestamp()-startTime, err)
}
return nil
}
func parseRequestBody(data []byte, groupLabels []string, callback func(tss []TimeSerie)) error {
// Synchronously process the request in order to properly return errors to Parse caller,
// so it could properly return HTTP 503 status code in response.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/896
bb := bodyBufferPool.Get()
defer bodyBufferPool.Put(bb)
if encoding.IsZstd(data) {
var err error
bb.B, err = encoding.DecompressZSTDLimited(bb.B[:0], data, maxInsertRequestSize.IntN())
if err != nil {
return fmt.Errorf("cannot decompress zstd-encoded request with length %d: %w", len(data), err)
}
} else {
var err error
bb.B, err = snappy.Decode(bb.B, data, maxInsertRequestSize.IntN())
if err != nil {
return fmt.Errorf("cannot decompress snappy-encoded request with length %d: %w", len(data), err)
}
}
if int64(len(bb.B)) > maxInsertRequestSize.N {
return fmt.Errorf("too big unpacked request; mustn't exceed `-maxInsertRequestSize=%d` bytes; got %d bytes", maxInsertRequestSize.N, len(bb.B))
}
wru := getWriteRequestUnmarshaler()
defer putWriteRequestUnmarshaler(wru)
if err := wru.UnmarshalProtobuf(bb.B, groupLabels, func(tss []TimeSerie) {
rowsRead.Add(len(tss))
callback(tss)
}); err != nil {
unmarshalErrors.Inc()
return fmt.Errorf("cannot unmarshal prompb.WriteRequest with size %d bytes: %w", len(bb.B), err)
}
return nil
}
var bodyBufferPool bytesutil.ByteBufferPool
var (
readCalls = metrics.NewCounter(`vm_protoparser_read_calls_total{type="promremotewrite"}`)
readErrors = metrics.NewCounter(`vm_protoparser_read_errors_total{type="promremotewrite"}`)
rowsRead = metrics.NewCounter(`vm_protoparser_rows_read_total{type="promremotewrite"}`)
unmarshalErrors = metrics.NewCounter(`vm_protoparser_unmarshal_errors_total{type="promremotewrite"}`)
)

View File

@@ -0,0 +1,67 @@
package protoparser
import (
"bytes"
"fmt"
"strings"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
"github.com/golang/snappy"
)
func BenchmarkParse(b *testing.B) {
data := buildSnappyEncodedWriteRequest(5000, 20, 20, 3)
groupLabels := []string{
"foo",
"bar",
"baz",
"__name__",
"job",
"groupLabel",
}
var cnt int
b.ResetTimer()
b.ReportAllocs()
b.SetBytes(int64(len(data)))
for b.Loop() {
err := Parse(bytes.NewReader(data), groupLabels, func(tss []TimeSerie) {
cnt += len(tss)
})
if err != nil {
b.Fatalf("stream.Parse: %v", err)
}
}
}
// buildSnappyEncodedWriteRequest builds a snappy-encoded protobuf WriteRequest
// with numSeries time series, each having numLabels labels of labelSize bytes each.
func buildSnappyEncodedWriteRequest(numSeries, numLabels, labelSize, groupsNum int) []byte {
labelValue := strings.Repeat("x", labelSize)
tss := make([]prompb.TimeSeries, numSeries)
for i := range tss {
labels := make([]prompb.Label, numLabels)
for j := range labels {
labels[j] = prompb.Label{
Name: fmt.Sprintf("label%02d", j),
Value: fmt.Sprintf("val%05d_%s", i, labelValue),
}
}
labels = append(labels, prompb.Label{
Name: "groupLabel",
Value: fmt.Sprintf("%d", i%groupsNum),
})
tss[i] = prompb.TimeSeries{
Labels: labels,
Samples: []prompb.Sample{{Value: 1, Timestamp: 1000}},
}
}
wr := &prompb.WriteRequest{Timeseries: tss}
pbData := wr.MarshalProtobuf(nil)
return snappy.Encode(nil, pbData)
}

View File

@@ -0,0 +1,170 @@
package protoparser
import (
"fmt"
"slices"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/easyproto"
"github.com/cespare/xxhash/v2"
)
type TimeSerie struct {
GroupLabels []Label
Fingerprint uint64
}
type Label struct {
Name string
Value string
}
func getWriteRequestUnmarshaler() *writeRequestUnmarshaler {
v := wruPool.Get()
if v == nil {
return &writeRequestUnmarshaler{
tss: make([]TimeSerie, 0, 1024),
labelsPool: make([]Label, 0, 4096),
d: xxhash.New(),
}
}
return v.(*writeRequestUnmarshaler)
}
func putWriteRequestUnmarshaler(wru *writeRequestUnmarshaler) {
wru.Reset()
wruPool.Put(wru)
}
var wruPool sync.Pool
// WriteRequestUnmarshaler is reusable unmarshaler for WriteRequest protobuf messages.
//
// It maintains internal pools for labels and samples to reduce memory allocations.
// See UnmarshalProtobuf for details on how to use it.
type writeRequestUnmarshaler struct {
tss []TimeSerie
labelsPool []Label
d *xxhash.Digest
}
// Reset resets wru, so it could be re-used.
func (wru *writeRequestUnmarshaler) Reset() {
wru.tss = wru.tss[:0]
wru.labelsPool = wru.labelsPool[:0]
wru.d.Reset()
}
func (wru *writeRequestUnmarshaler) UnmarshalProtobuf(src []byte, groupLabels []string, callback func(tss []TimeSerie)) error {
wru.Reset()
var err error
tss := wru.tss
// message WriteRequest {
// repeated TimeSeries timeseries = 1;
// reserved 2;
// repeated Metadata metadata = 3;
// }
labelsPool := wru.labelsPool
var fc easyproto.FieldContext
for len(src) > 0 {
if len(tss) >= cap(tss) {
callback(tss)
tss = tss[:0]
labelsPool = labelsPool[:0]
}
src, err = fc.NextField(src)
if err != nil {
return fmt.Errorf("cannot read the next field: %w", err)
}
switch fc.FieldNum {
case 1:
data, ok := fc.MessageData()
if !ok {
return fmt.Errorf("cannot read timeseries data")
}
tss = tss[:len(tss)+1]
ts := &tss[len(tss)-1]
d := wru.d
d.Reset()
labelsPool, err = ts.unmarshalProtobuf(data, groupLabels, labelsPool, d)
if err != nil {
return fmt.Errorf("cannot unmarshal timeseries: %w", err)
}
}
}
if len(tss) > 0 {
callback(tss)
tss = tss[:0]
labelsPool = labelsPool[:0]
}
wru.tss = tss[:0]
wru.labelsPool = labelsPool
wru.d.Reset()
return nil
}
func (ts *TimeSerie) unmarshalProtobuf(src []byte, groupLabels []string, labelsPool []Label, d *xxhash.Digest) ([]Label, error) {
// message TimeSeries {
// repeated Label labels = 1;
// repeated Sample samples = 2;
// }
labelsPoolLen := len(labelsPool)
var fc easyproto.FieldContext
var lfc easyproto.FieldContext
for len(src) > 0 {
var err error
src, err = fc.NextField(src)
if err != nil {
return labelsPool, fmt.Errorf("cannot read the next field: %w", err)
}
switch fc.FieldNum {
case 1:
data, ok := fc.MessageData()
if !ok {
return labelsPool, fmt.Errorf("cannot read label data")
}
var nameBytes, valueBytes []byte
ldata := data
for len(ldata) > 0 {
ldata, err = lfc.NextField(ldata)
if err != nil {
return labelsPool, fmt.Errorf("cannot read label field: %w", err)
}
switch lfc.FieldNum {
case 1:
nameBytes, ok = lfc.Bytes()
if !ok {
return labelsPool, fmt.Errorf("cannot read label name")
}
case 2:
valueBytes, ok = lfc.Bytes()
if !ok {
return labelsPool, fmt.Errorf("cannot read label value")
}
}
}
_, _ = d.Write(data)
name := bytesutil.ToUnsafeString(nameBytes)
if slices.Contains(groupLabels, name) {
labelsPool = append(labelsPool, Label{
Name: name,
Value: bytesutil.ToUnsafeString(valueBytes),
})
}
}
}
ts.GroupLabels = labelsPool[labelsPoolLen:]
ts.Fingerprint = d.Sum64()
return labelsPool, nil
}

View File

@@ -0,0 +1,86 @@
package protoparser
import (
"fmt"
"strings"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompb"
)
func BenchmarkWriteRequest_UnmarshalProtobuf(b *testing.B) {
var data = make([]byte, 0, 21_000_000)
f := func(rows, labels, labelSize, groupBy int) {
bName := fmt.Sprintf("Rows=%d/Labels=%d/LabelSize=%d/GroupBy=%d", rows, labels, labelSize, groupBy)
b.Run(bName, func(b *testing.B) {
data := buildEncodedWriteRequest(data, rows, labels, labelSize, groupBy)
groupLabels := []string{
"foo",
"bar",
"baz",
"__name__",
"job",
"groupLabel",
}
wru := getWriteRequestUnmarshaler()
cnt := 0
b.ResetTimer()
b.ReportAllocs()
b.SetBytes(int64(len(data)))
for b.Loop() {
wru.Reset()
if err := wru.UnmarshalProtobuf(data, groupLabels, func(tss []TimeSerie) {
cnt += len(tss)
}); err != nil {
b.Fatalf("unexpected error: %s", err)
}
}
})
}
f(5_000, 0, 0, 3)
f(5_000, 1, 20, 3)
f(1_000, 20, 20, 3)
f(5_000, 20, 20, 3)
f(10_000, 20, 20, 3)
f(20_000, 20, 20, 3)
// long label values
f(1_000, 20, 2000, 3)
// many labels
f(1_000, 2000, 100, 3)
}
// buildEncodedWriteRequest builds a snappy-encoded protobuf WriteRequest
// with numSeries time series, each having numLabels labels of labelSize bytes each.
func buildEncodedWriteRequest(dst []byte, numSeries, numLabels, labelSize, groupsNum int) []byte {
labelValue := strings.Repeat("x", labelSize)
tss := make([]prompb.TimeSeries, numSeries)
for i := range tss {
labels := make([]prompb.Label, numLabels)
for j := range labels {
labels[j] = prompb.Label{
Name: fmt.Sprintf("label%02d", j),
Value: fmt.Sprintf("val%05d_%s", i, labelValue),
}
}
labels = append(labels, prompb.Label{
Name: "groupLabel",
Value: fmt.Sprintf("%d", i%groupsNum),
})
tss[i] = prompb.TimeSeries{
Labels: labels,
Samples: []prompb.Sample{{Value: 1, Timestamp: 1000}},
}
}
wr := &prompb.WriteRequest{Timeseries: tss}
return wr.MarshalProtobuf(dst[:0])
}

View File

@@ -30,10 +30,13 @@ See also [LTS releases](https://docs.victoriametrics.com/victoriametrics/lts-rel
* FEATURE: [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/) and `vmselect` in [VictoriaMetrics cluster](https://docs.victoriametrics.com/victoriametrics/cluster-victoriametrics/): log calls to [/api/v1/admin/tsdb/delete_series](https://docs.victoriametrics.com/victoriametrics/url-examples/#apiv1admintsdbdelete_series) API handler. This should help to identify events of metrics deletion from the database. See [#11104](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11104).
* FEATURE: [vmctl](https://docs.victoriametrics.com/victoriametrics/vmctl/): add `-vm-headers` and `-vm-bearer-token` flags for authenticating requests to the VictoriaMetrics import destination. The flags are available in `opentsdb`, `influx`, `remote-read`, `prometheus`, `mimir`, and `thanos` vmctl sub-commands. See [#8897](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/8897).
* FEATURE: [vmui](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/#vmui): add the `last` value to graph legend statistics. See [#10759](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/10759).
* FEATURE: [vmagent](https://docs.victoriametrics.com/victoriametrics/vmagent/): add `-promscrape.cluster.shardByLabels` command-line flag for selecting target labels used for sharding scrape targets among `vmagent` instances in cluster mode. See [#11044](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/11044).
* BUGFIX: [stream aggregation](https://docs.victoriametrics.com/victoriametrics/stream-aggregation/): fix issue with producing aggregated samples with identical timestamps between flushes. See [#10808](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/10808).
* BUGFIX: [vmalert](https://docs.victoriametrics.com/victoriametrics/vmalert/),[vmauth](https://docs.victoriametrics.com/victoriametrics/vmauth/),[vmagent](https://docs.victoriametrics.com/victoriametrics/vmagent/) and [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/): fix rare unbounded shutdown delay when config reload takes longer than `-configCheckInterval`. See [#11107](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11107). Thanks to @PleasingFungus for contribution.
* BUGFIX: `vmselect` in [VictoriaMetrics cluster](https://docs.victoriametrics.com/victoriametrics/cluster-victoriametrics/): fix corrupted metrics metadata when a response contains multiple rows. See [#11115](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11115). Thanks for @fxrlv for the contribution.
* BUGFIX: [vmagent](https://docs.victoriametrics.com/victoriametrics/vmagent/): fix potential corruption of remote-write metadata `Unit` values. See [#11120](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/11120). Thanks for @fxrlv for the contribution.
* BUGFIX: [vmbackup](https://docs.victoriametrics.com/vmbackup/), [vmbackupmanager](https://docs.victoriametrics.com/victoriametrics/vmbackupmanager/): do not fail backup list if directory is absent while using `fs://` destination to align with other protocols. See [6c3c548](https://github.com/VictoriaMetrics/VictoriaMetrics/commit/6c3c548ddb0385b749e731f52276f130e2a4e4a8)
* BUGFIX: [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/) and `vmstorage` in [VictoriaMetrics cluster](https://docs.victoriametrics.com/victoriametrics/cluster-victoriametrics/): prevent more cases of panic during directory deletion on `NFS`-based mounts. See [#11060](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/11060).
## [v1.145.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.145.0)

View File

@@ -797,13 +797,6 @@ For example, the following commands spread scrape targets among a cluster of two
The `-promscrape.cluster.memberNum` can be set to a StatefulSet pod name when `vmagent` runs in Kubernetes.
The pod name must end with a number in the range `0 ... promscrape.cluster.membersCount-1`. For example, `-promscrape.cluster.memberNum=vmagent-0`.
By default, targets are sharded among `vmagent` instances by all target labels after relabeling.
Use `-promscrape.cluster.shardByLabels` to shard targets by specified labels instead.
For example, `-promscrape.cluster.shardByLabels=service,pod` keeps targets with the same `service` and `pod` label value on the same `vmagent` instance.
If some of the specified labels are present on a target, then only the present labels are used for sharding.
If none of the specified labels are present, then all target labels are used for sharding.
By default, each scrape target is scraped only by a single `vmagent` instance in the cluster. If there is a need for replicating scrape targets among multiple `vmagent` instances,
then `-promscrape.cluster.replicationFactor` command-line flag must be set to the desired number of replicas. For example, the following commands
start a cluster of three `vmagent` instances, where two `vmagent` instances scrape each target:

View File

@@ -0,0 +1,103 @@
---
weight: 3
menu:
docs:
parent: victoriametrics
weight: 13
title: vmestimator
tags:
- metrics
- cardinality
aliases:
- /vmestimator.html
- /vmestimator/index.html
- /vmestimator/
---
`cestimator` is a cardinality estimator that receives Prometheus remote write streams and exposes approximate time series cardinality as metrics.
It is useful for tracking how many unique time series are flowing through across all metrics, metric name, or broken down by specific labels.
## How it works
Running:
```
go run ./app/cestimator/... -config=streams.yaml -httpListenAddr=:8490
```
Configuration:
```yaml
streams:
# Track total cardinality with no grouping.
- interval: '1h'
# Track cardinality grouped by metric name.
- interval: '1h'
group_by: ["__name__"]
# Track cardinality grouped by job label.
- interval: '1m'
group_by: ["job"]
# Track cardinality grouped by tenant info
- group_by: ["vm_account_id", "vm_project_id"]
# Track cardinality of jobs, with extra labels on the output metrics.
- group_by: ["job"]
labels:
region: 'eu-central-1'
env: 'production'
```
Fields:
- `group_by` (optional): list of label names to split cardinality by; each distinct combination gets its own estimate
- `group_limit` (optional): maximum number of distinct groups to track; excess groups are counted in a rejected sketch but not individually; defaults to `10000`
- `buckets` (optional): number of internal shards for parallel ingestion; defaults to `min(20, availableCPUs)`
- `labels` (optional): extra labels attached to all output metrics for this estimator
- `interval` (optional): how often to rotate (reset) counters; defaults to `5m`
- `hll_precision` (optional): HyperLogLog precision, must be in range `[4, 18]`; higher values yield more accurate estimates at the cost of more memory; defaults to `14`
- `hll_sparse` (optional): whether to use sparse HyperLogLog representation, which reduces memory for low-cardinality groups; defaults to `true`
Cardinality generator:
```
go run ./app/cegen/main.go -cardI=100 -cardY=20 -template="foo{instance=\"127.0.0.[cardI]\",job=\"ametric[cardY]\"}"
```
## Metrics
By default, cardinality estimates are merged with regular metrics and exposed at `/metrics`.
This behavior is controlled by the `-cardinalityMetrics.exposeAt` flag:
- `-cardinalityMetrics.exposeAt=/metrics` (default): cardinality metrics merged with regular metrics at `/metrics`
- `-cardinalityMetrics.exposeAt=/cardinality/metrics`: only cardinality metrics exposed at that path
- `-cardinalityMetrics.exposeAt=`: cardinality metrics not exposed via HTTP
All metrics include `interval`, `group_by_keys`, and `group_by_values` labels. Extra labels from the `labels` config field are inserted between `interval` and `group_by_keys` (sorted alphabetically).
**Without grouping** (`group_by_keys` is `__global__` and `group_by_values` is not set):
```
cardinality_estimate{interval="1h0m0s",group_by_keys="__global__"} 142300
```
**With grouping** — one summary line (total distinct group count) plus one line per distinct label value combination. Each per-group line also includes individual `by_{key}="{val}"` labels for each group key:
```
cardinality_estimate{interval="5m0s",group_by_keys="__group__",group_by_values="instance,job"} 2
cardinality_estimate{interval="5m0s",group_by_keys="instance,job",group_by_values="host1:9090,prometheus",by_instance="host1:9090",by_job="prometheus"} 312
cardinality_estimate{interval="5m0s",group_by_keys="instance,job",group_by_values="host2:9100,node",by_instance="host2:9100",by_job="node"} 87
```
**With extra labels:**
```
cardinality_estimate{interval="5m0s",env="production",region="eu-central-1",group_by_keys="job",group_by_values="prometheus",by_job="prometheus"} 312
```
## Operational metrics
When grouping is enabled, cestimator exposes per-bucket operational metrics at `/metrics`:
- `cestimator_group_estimator_size{groupBy, bucket}` — number of active groups in this bucket after the last rotation
- `cestimator_group_estimator_rejected_size{groupBy, bucket}` — estimated number of distinct group values rejected since the last rotation because `group_limit` was reached
- `cestimator_group_limit{groupBy, bucket}` — configured `group_limit` for this bucket

177
go.mod
View File

@@ -2,31 +2,35 @@ module github.com/VictoriaMetrics/VictoriaMetrics
go 1.26.4
replace github.com/axiomhq/hyperloglog => github.com/makasim/hyperloglog v0.0.10-reuse-memory
require (
cloud.google.com/go/storage v1.62.1
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.21.0
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.13.1
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.6.4
github.com/VictoriaMetrics/VictoriaLogs v1.50.1-0.20260415124154-6b7a6357aec0
cloud.google.com/go/storage v1.62.3
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.22.0
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.14.0
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.8.0
github.com/VictoriaMetrics/VictoriaLogs v1.121.1-0.20260616132739-c901a1e31cb3
github.com/VictoriaMetrics/easyproto v1.2.0
github.com/VictoriaMetrics/fastcache v1.13.3
github.com/VictoriaMetrics/metrics v1.43.2
github.com/VictoriaMetrics/metricsql v0.87.0
github.com/aws/aws-sdk-go-v2 v1.41.5
github.com/aws/aws-sdk-go-v2/config v1.32.14
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.13
github.com/aws/aws-sdk-go-v2/service/s3 v1.99.0
github.com/VictoriaMetrics/metricsql v0.87.1
github.com/aws/aws-sdk-go-v2 v1.42.0
github.com/aws/aws-sdk-go-v2/config v1.32.25
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.27
github.com/aws/aws-sdk-go-v2/service/s3 v1.103.3
github.com/axiomhq/hyperloglog v0.0.0-00010101000000-000000000000
github.com/bmatcuk/doublestar/v4 v4.10.0
github.com/cespare/xxhash/v2 v2.3.0
github.com/cheggaaa/pb/v3 v3.1.7
github.com/dgryski/go-metro v0.0.0-20250106013310-edb8663e5e33
github.com/gogo/protobuf v1.3.2
github.com/golang/snappy v1.0.0
github.com/google/go-cmp v0.7.0
github.com/googleapis/gax-go/v2 v2.22.0
github.com/influxdata/influxdb v1.12.4
github.com/klauspost/compress v1.18.5
github.com/klauspost/compress v1.18.6
github.com/oklog/ulid/v2 v2.1.1
github.com/prometheus/prometheus v0.311.3
github.com/prometheus/prometheus v0.312.0
github.com/urfave/cli/v2 v2.27.7
github.com/valyala/fastjson v1.6.10
github.com/valyala/fastrand v1.1.0
@@ -34,46 +38,46 @@ require (
github.com/valyala/gozstd v1.24.0
github.com/valyala/histogram v1.2.0
github.com/valyala/quicktemplate v1.8.0
golang.org/x/net v0.55.0
golang.org/x/net v0.56.0
golang.org/x/oauth2 v0.36.0
golang.org/x/sys v0.45.0
google.golang.org/api v0.276.0
golang.org/x/sys v0.46.0
google.golang.org/api v0.284.0
gopkg.in/yaml.v2 v2.4.0
)
require (
cel.dev/expr v0.25.1 // indirect
cel.dev/expr v0.25.2 // indirect
cloud.google.com/go v0.123.0 // indirect
cloud.google.com/go/auth v0.20.0 // indirect
cloud.google.com/go/auth/oauth2adapt v0.2.8 // indirect
cloud.google.com/go/compute/metadata v0.9.0 // indirect
cloud.google.com/go/iam v1.9.0 // indirect
cloud.google.com/go/monitoring v1.27.0 // indirect
cloud.google.com/go/iam v1.11.0 // indirect
cloud.google.com/go/monitoring v1.29.0 // indirect
github.com/Azure/azure-sdk-for-go/sdk/internal v1.12.0 // indirect
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.1 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.32.0 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.56.0 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.56.0 // indirect
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.2 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.33.0 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.57.0 // indirect
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.57.0 // indirect
github.com/VividCortex/ewma v1.2.0 // indirect
github.com/alecthomas/units v0.0.0-20240927000941-0f3dac36c52b // indirect
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.8 // indirect
github.com/aws/aws-sdk-go-v2/credentials v1.19.14 // indirect
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.21 // indirect
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.21 // indirect
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.21 // indirect
github.com/aws/aws-sdk-go-v2/internal/ini v1.8.6 // indirect
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.22 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.7 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.13 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.21 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.21 // indirect
github.com/aws/aws-sdk-go-v2/service/signin v1.0.9 // indirect
github.com/aws/aws-sdk-go-v2/service/sso v1.30.15 // indirect
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.19 // indirect
github.com/aws/aws-sdk-go-v2/service/sts v1.41.10 // indirect
github.com/aws/smithy-go v1.24.3 // indirect
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.13 // indirect
github.com/aws/aws-sdk-go-v2/credentials v1.19.24 // indirect
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.29 // indirect
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.29 // indirect
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.29 // indirect
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.30 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.12 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.22 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.29 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.29 // indirect
github.com/aws/aws-sdk-go-v2/service/signin v1.2.0 // indirect
github.com/aws/aws-sdk-go-v2/service/sso v1.31.3 // indirect
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.36.6 // indirect
github.com/aws/aws-sdk-go-v2/service/sts v1.43.3 // indirect
github.com/aws/smithy-go v1.27.2 // indirect
github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/cenkalti/backoff/v5 v5.0.3 // indirect
github.com/clipperhouse/uax29/v2 v2.7.0 // indirect
github.com/cncf/xds/go v0.0.0-20260202195803-dba9d589def2 // indirect
github.com/cpuguy83/go-md2man/v2 v2.0.7 // indirect
@@ -82,8 +86,8 @@ require (
github.com/envoyproxy/go-control-plane/envoy v1.37.0 // indirect
github.com/envoyproxy/protoc-gen-validate v1.3.3 // indirect
github.com/fatih/color v1.19.0 // indirect
github.com/felixge/httpsnoop v1.0.4 // indirect
github.com/fxamacker/cbor/v2 v2.9.1 // indirect
github.com/felixge/httpsnoop v1.1.0 // indirect
github.com/fxamacker/cbor/v2 v2.9.2 // indirect
github.com/go-jose/go-jose/v4 v4.1.4 // indirect
github.com/go-logr/logr v1.4.3 // indirect
github.com/go-logr/stdr v1.2.2 // indirect
@@ -92,89 +96,90 @@ require (
github.com/golang-jwt/jwt/v5 v5.3.1 // indirect
github.com/google/s2a-go v0.1.9 // indirect
github.com/google/uuid v1.6.0 // indirect
github.com/googleapis/enterprise-certificate-proxy v0.3.14 // indirect
github.com/googleapis/enterprise-certificate-proxy v0.3.16 // indirect
github.com/grafana/regexp v0.0.0-20250905093917-f7b3be9d1853 // indirect
github.com/hashicorp/go-version v1.9.0 // indirect
github.com/jpillora/backoff v1.0.0 // indirect
github.com/json-iterator/go v1.1.12 // indirect
github.com/kamstrup/intmap v0.5.2 // indirect
github.com/knadh/koanf/maps v0.1.2 // indirect
github.com/knadh/koanf/providers/confmap v1.0.0 // indirect
github.com/knadh/koanf/v2 v2.3.4 // indirect
github.com/knadh/koanf/v2 v2.3.5 // indirect
github.com/kylelemons/godebug v1.1.0 // indirect
github.com/mattn/go-colorable v0.1.14 // indirect
github.com/mattn/go-isatty v0.0.21 // indirect
github.com/mattn/go-runewidth v0.0.23 // indirect
github.com/mattn/go-colorable v0.1.15 // indirect
github.com/mattn/go-isatty v0.0.22 // indirect
github.com/mattn/go-runewidth v0.0.24 // indirect
github.com/mitchellh/copystructure v1.2.0 // indirect
github.com/mitchellh/reflectwalk v1.0.2 // indirect
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
github.com/modern-go/reflect2 v1.0.3-0.20250322232337-35a7c28c31ee // indirect
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.150.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.150.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.150.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.154.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.154.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.154.0 // indirect
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect
github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
github.com/prometheus/client_golang v1.23.2 // indirect
github.com/prometheus/client_golang/exp v0.0.0-20260411065453-32d05ffab50a // indirect
github.com/prometheus/client_golang/exp v0.0.0-20260602051030-3537b20ac86b // indirect
github.com/prometheus/client_model v0.6.2 // indirect
github.com/prometheus/common v0.67.5 // indirect
github.com/prometheus/common v0.68.1 // indirect
github.com/prometheus/otlptranslator v1.0.0 // indirect
github.com/prometheus/procfs v0.20.1 // indirect
github.com/prometheus/sigv4 v0.4.1 // indirect
github.com/puzpuzpuz/xsync/v4 v4.4.0 // indirect
github.com/puzpuzpuz/xsync/v4 v4.5.0 // indirect
github.com/russross/blackfriday/v2 v2.1.0 // indirect
github.com/spiffe/go-spiffe/v2 v2.6.0 // indirect
github.com/spiffe/go-spiffe/v2 v2.7.0 // indirect
github.com/stretchr/testify v1.11.1 // indirect
github.com/valyala/bytebufferpool v1.0.0 // indirect
github.com/x448/float16 v0.8.4 // indirect
github.com/xrash/smetrics v0.0.0-20250705151800-55b8f293f342 // indirect
go.opentelemetry.io/auto/sdk v1.2.1 // indirect
go.opentelemetry.io/collector/component v1.56.0 // indirect
go.opentelemetry.io/collector/confmap v1.56.0 // indirect
go.opentelemetry.io/collector/confmap/xconfmap v0.150.0 // indirect
go.opentelemetry.io/collector/consumer v1.56.0 // indirect
go.opentelemetry.io/collector/featuregate v1.56.0 // indirect
go.opentelemetry.io/collector/internal/componentalias v0.150.0 // indirect
go.opentelemetry.io/collector/pdata v1.56.0 // indirect
go.opentelemetry.io/collector/pipeline v1.56.0 // indirect
go.opentelemetry.io/collector/processor v1.56.0 // indirect
go.opentelemetry.io/contrib/detectors/gcp v1.43.0 // indirect
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.68.0 // indirect
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.68.0 // indirect
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.68.0 // indirect
go.opentelemetry.io/otel v1.43.0 // indirect
go.opentelemetry.io/otel/metric v1.43.0 // indirect
go.opentelemetry.io/otel/sdk v1.43.0 // indirect
go.opentelemetry.io/otel/sdk/metric v1.43.0 // indirect
go.opentelemetry.io/otel/trace v1.43.0 // indirect
go.opentelemetry.io/collector/component v1.60.0 // indirect
go.opentelemetry.io/collector/confmap v1.60.0 // indirect
go.opentelemetry.io/collector/confmap/xconfmap v0.154.0 // indirect
go.opentelemetry.io/collector/consumer v1.60.0 // indirect
go.opentelemetry.io/collector/featuregate v1.60.0 // indirect
go.opentelemetry.io/collector/internal/componentalias v0.154.0 // indirect
go.opentelemetry.io/collector/pdata v1.60.0 // indirect
go.opentelemetry.io/collector/pipeline v1.60.0 // indirect
go.opentelemetry.io/collector/processor v1.60.0 // indirect
go.opentelemetry.io/contrib/detectors/gcp v1.44.0 // indirect
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.69.0 // indirect
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.69.0 // indirect
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.69.0 // indirect
go.opentelemetry.io/otel v1.44.0 // indirect
go.opentelemetry.io/otel/metric v1.44.0 // indirect
go.opentelemetry.io/otel/sdk v1.44.0 // indirect
go.opentelemetry.io/otel/sdk/metric v1.44.0 // indirect
go.opentelemetry.io/otel/trace v1.44.0 // indirect
go.uber.org/atomic v1.11.0 // indirect
go.uber.org/goleak v1.3.0 // indirect
go.uber.org/multierr v1.11.0 // indirect
go.uber.org/zap v1.27.1 // indirect
go.uber.org/zap v1.28.0 // indirect
go.yaml.in/yaml/v2 v2.4.4 // indirect
go.yaml.in/yaml/v3 v3.0.4 // indirect
golang.org/x/crypto v0.52.0 // indirect
golang.org/x/exp v0.0.0-20260410095643-746e56fc9e2f // indirect
golang.org/x/sync v0.20.0 // indirect
golang.org/x/term v0.43.0 // indirect
golang.org/x/text v0.37.0 // indirect
golang.org/x/crypto v0.53.0 // indirect
golang.org/x/exp v0.0.0-20260611194520-c48552f49976 // indirect
golang.org/x/sync v0.21.0 // indirect
golang.org/x/term v0.44.0 // indirect
golang.org/x/text v0.38.0 // indirect
golang.org/x/time v0.15.0 // indirect
google.golang.org/genproto v0.0.0-20260414002931-afd174a4e478 // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20260414002931-afd174a4e478 // indirect
google.golang.org/genproto/googleapis/rpc v0.0.0-20260414002931-afd174a4e478 // indirect
google.golang.org/grpc v1.80.0 // indirect
google.golang.org/protobuf v1.36.11 // indirect
google.golang.org/genproto v0.0.0-20260615183401-62b3387ff324 // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20260615183401-62b3387ff324 // indirect
google.golang.org/genproto/googleapis/rpc v0.0.0-20260615183401-62b3387ff324 // indirect
google.golang.org/grpc v1.81.1 // indirect
google.golang.org/protobuf v1.36.12-0.20260120151049-f2248ac996af // indirect
gopkg.in/inf.v0 v0.9.1 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
k8s.io/apimachinery v0.35.3 // indirect
k8s.io/client-go v0.35.3 // indirect
k8s.io/apimachinery v0.36.2 // indirect
k8s.io/client-go v0.36.2 // indirect
k8s.io/klog/v2 v2.140.0 // indirect
k8s.io/kube-openapi v0.0.0-20260414162039-ec9c827d403f // indirect
k8s.io/utils v0.0.0-20260319190234-28399d86e0b5 // indirect
k8s.io/kube-openapi v0.0.0-20260603220949-865597e52e25 // indirect
k8s.io/utils v0.0.0-20260507154919-ff6756f316d2 // indirect
sigs.k8s.io/json v0.0.0-20250730193827-2d320260d730 // indirect
sigs.k8s.io/randfill v1.0.0 // indirect
sigs.k8s.io/structured-merge-diff/v6 v6.3.2 // indirect
sigs.k8s.io/structured-merge-diff/v6 v6.4.0 // indirect
sigs.k8s.io/yaml v1.6.0 // indirect
)

566
go.sum
View File

@@ -1,5 +1,5 @@
cel.dev/expr v0.25.1 h1:1KrZg61W6TWSxuNZ37Xy49ps13NUovb66QLprthtwi4=
cel.dev/expr v0.25.1/go.mod h1:hrXvqGP6G6gyx8UAHSHJ5RGk//1Oj5nXQ2NI02Nrsg4=
cel.dev/expr v0.25.2 h1:K6j46C81hXtZQfuX60cVWQFBJahKSE2gfRbNuvr5bFs=
cel.dev/expr v0.25.2/go.mod h1:hrXvqGP6G6gyx8UAHSHJ5RGk//1Oj5nXQ2NI02Nrsg4=
cloud.google.com/go v0.123.0 h1:2NAUJwPR47q+E35uaJeYoNhuNEM9kM8SjgRgdeOJUSE=
cloud.google.com/go v0.123.0/go.mod h1:xBoMV08QcqUGuPW65Qfm1o9Y4zKZBpGS+7bImXLTAZU=
cloud.google.com/go/auth v0.20.0 h1:kXTssoVb4azsVDoUiF8KvxAqrsQcQtB53DcSgta74CA=
@@ -8,24 +8,24 @@ cloud.google.com/go/auth/oauth2adapt v0.2.8 h1:keo8NaayQZ6wimpNSmW5OPc283g65QNIi
cloud.google.com/go/auth/oauth2adapt v0.2.8/go.mod h1:XQ9y31RkqZCcwJWNSx2Xvric3RrU88hAYYbjDWYDL+c=
cloud.google.com/go/compute/metadata v0.9.0 h1:pDUj4QMoPejqq20dK0Pg2N4yG9zIkYGdBtwLoEkH9Zs=
cloud.google.com/go/compute/metadata v0.9.0/go.mod h1:E0bWwX5wTnLPedCKqk3pJmVgCBSM6qQI1yTBdEb3C10=
cloud.google.com/go/iam v1.9.0 h1:89wyjxT6DL4b5rk/Nk8eBC9DHqf+JiMstrn5IEYxFw4=
cloud.google.com/go/iam v1.9.0/go.mod h1:KP+nKGugNJW4LcLx1uEZcq1ok5sQHFaQehQNl4QDgV4=
cloud.google.com/go/logging v1.15.0 h1:6ooUEBNT6jdWh2b36+iuPn6b/R9qN/tHCbvGS5255gg=
cloud.google.com/go/logging v1.15.0/go.mod h1:ZGKnpBaURITh+g/uom2VhbiFoFWvejcrHPDhxFtU/gI=
cloud.google.com/go/longrunning v0.10.0 h1:4OWvp1BjCvoeSZTog3sRFDu6j4IrI9TI4/Y9N+8h25g=
cloud.google.com/go/longrunning v0.10.0/go.mod h1:8nqFBPOO1U/XkhWl0I19AMZEphrHi73VNABIpKYaTwM=
cloud.google.com/go/monitoring v1.27.0 h1:BhYwMqao+e5Nn7JtWMM9m6zRtKtVUK6kJWMizXChkLU=
cloud.google.com/go/monitoring v1.27.0/go.mod h1:72NOVjJXHY/HBfoLT0+qlCZBT059+9VXLeAnL2PeeVM=
cloud.google.com/go/storage v1.62.1 h1:Os0G3XbUbjZumkpDUf2Y0rLoXJTCF1kU2kWUujKYXD8=
cloud.google.com/go/storage v1.62.1/go.mod h1:cpYz/kRVZ+UQAF1uHeea10/9ewcRbxGoGNKsS9daSXA=
cloud.google.com/go/trace v1.13.0 h1:RfqsqPOiSCG8ql50UZt5F65KrVa1zbY9mJrO7xvZfbE=
cloud.google.com/go/trace v1.13.0/go.mod h1:r+bdAn16dKLSV1G2D5v3e58IlQlizfxWrUfjx7kM7X0=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.21.0 h1:fou+2+WFTib47nS+nz/ozhEBnvU96bKHy6LjRsY4E28=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.21.0/go.mod h1:t76Ruy8AHvUAC8GfMWJMa0ElSbuIcO03NLpynfbgsPA=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.13.1 h1:Hk5QBxZQC1jb2Fwj6mpzme37xbCDdNTxU7O9eb5+LB4=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.13.1/go.mod h1:IYus9qsFobWIc2YVwe/WPjcnyCkPKtnHAqUYeebc8z0=
github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.3.2 h1:yz1bePFlP5Vws5+8ez6T3HWXPmwOK7Yvq8QxDBD3SKY=
github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.3.2/go.mod h1:Pa9ZNPuoNu/GztvBSKk9J1cDJW6vk/n0zLtV4mgd8N8=
cloud.google.com/go/iam v1.11.0 h1:KieQ9Pb+LLPak1O3Rv3GgCxhnmkYf7Xyh0P5HfF1jFM=
cloud.google.com/go/iam v1.11.0/go.mod h1:KP+nKGugNJW4LcLx1uEZcq1ok5sQHFaQehQNl4QDgV4=
cloud.google.com/go/logging v1.18.0 h1:KhzZq+1cSkPH9YUaKLLhLtQxIHitVayBmk0sGfoM9+k=
cloud.google.com/go/logging v1.18.0/go.mod h1:ZGKnpBaURITh+g/uom2VhbiFoFWvejcrHPDhxFtU/gI=
cloud.google.com/go/longrunning v1.0.0 h1:lwzWEYD8+NkYV7dhexOz6kmlvajZA70+bW/xMhRVVdY=
cloud.google.com/go/longrunning v1.0.0/go.mod h1:8nqFBPOO1U/XkhWl0I19AMZEphrHi73VNABIpKYaTwM=
cloud.google.com/go/monitoring v1.29.0 h1:AHhDsFaSax1/4k+qlIDX/SDGe6hggnfXJ9dkgD9qBPY=
cloud.google.com/go/monitoring v1.29.0/go.mod h1:72NOVjJXHY/HBfoLT0+qlCZBT059+9VXLeAnL2PeeVM=
cloud.google.com/go/storage v1.62.3 h1:SZq1t23NCI+e96dH77Dg3PEfsNNEjqO8zE5AnD8gVD0=
cloud.google.com/go/storage v1.62.3/go.mod h1:cpYz/kRVZ+UQAF1uHeea10/9ewcRbxGoGNKsS9daSXA=
cloud.google.com/go/trace v1.16.0 h1:GmQovzFc5F0CNfl0VLgL64aoTtu7xsM0YajW2GlG9+E=
cloud.google.com/go/trace v1.16.0/go.mod h1:r+bdAn16dKLSV1G2D5v3e58IlQlizfxWrUfjx7kM7X0=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.22.0 h1:aokoqcHvaGjiM3VpjKDfMMnF/8epJ+Q1HLJ7CudztqE=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.22.0/go.mod h1:/WYEx9pcM9Y+Dd/APJaNlSvVSvzl54rrMdZT5+Oi2LM=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.14.0 h1:CU4+EJeJi3TKYWEcYuSdWsjzw0nVsK/H0MSQOiPcymU=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.14.0/go.mod h1:q0+UTSRvShwUCrR/s5HtyInYphN7Wvxb7snFM3u+SLA=
github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.4.0 h1:xFaZZ+IubdftrDHnGGwZ6QvQ3KHTtWl2MCK+GMt2vxs=
github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.4.0/go.mod h1:mCBhUhlMjLLJKr5aqw2TNS/VqJOie8MzWq3DAMJeKso=
github.com/Azure/azure-sdk-for-go/sdk/internal v1.12.0 h1:fhqpLE3UEXi9lPaBRpQ6XuRW0nU7hgg4zlmZZa+a9q4=
github.com/Azure/azure-sdk-for-go/sdk/internal v1.12.0/go.mod h1:7dCRMLwisfRH3dBupKeNCioWYUZ4SS09Z14H+7i8ZoY=
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.7.0 h1:LkHbJbgF3YyvC53aqYGR+wWQDn2Rdp9AQdGndf9QvY4=
@@ -34,34 +34,34 @@ github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0/go.mod h1:Y/HgrePTmGy9HjdSGTqZNa+apUpTVIEVKXJyARP2lrk=
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.8.1 h1:/Zt+cDPnpC3OVDm/JKLOs7M2DKmLRIIp3XIx9pHHiig=
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.8.1/go.mod h1:Ng3urmn6dYe8gnbCMoHHVl5APYz2txho3koEkV2o2HA=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.6.4 h1:jWQK1GI+LeGGUKBADtcH2rRqPxYB1Ljwms5gFA2LqrM=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.6.4/go.mod h1:8mwH4klAm9DUgR2EEHyEEAQlRDvLPyg5fQry3y+cDew=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.8.0 h1:irsmOWwkp0KCTTNS5e2hdFeIvSQClQo2No3IaNmL3Vw=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.8.0/go.mod h1:GWcBkQj3MqN7ozHKLaCCAuNLiXoIGv2RtanfAwSjY/Y=
github.com/AzureAD/microsoft-authentication-extensions-for-go/cache v0.1.1 h1:WJTmL004Abzc5wDB5VtZG2PJk5ndYDgVacGqfirKxjM=
github.com/AzureAD/microsoft-authentication-extensions-for-go/cache v0.1.1/go.mod h1:tCcJZ0uHAmvjsVYzEFivsRTN00oz5BEsRgQHu5JZ9WE=
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.1 h1:edShSHV3DV90+kt+CMaEXEzR9QF7wFrPJxVGz2blMIU=
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.1/go.mod h1:HKpQxkWaGLJ+D/5H8QRpyQXA1eKjxkFlOMwck5+33Jk=
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.2 h1:RHK7bS+HQMslb1sZpAokUt+zTVmue0hKSs2C791hhzU=
github.com/AzureAD/microsoft-authentication-library-for-go v1.7.2/go.mod h1:HKpQxkWaGLJ+D/5H8QRpyQXA1eKjxkFlOMwck5+33Jk=
github.com/Code-Hex/go-generics-cache v1.5.1 h1:6vhZGc5M7Y/YD8cIUcY8kcuQLB4cHR7U+0KMqAA0KcU=
github.com/Code-Hex/go-generics-cache v1.5.1/go.mod h1:qxcC9kRVrct9rHeiYpFWSoW1vxyillCVzX13KZG8dl4=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.32.0 h1:rIkQfkCOVKc1OiRCNcSDD8ml5RJlZbH/Xsq7lbpynwc=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.32.0/go.mod h1:RD2SsorTmYhF6HkTmDw7KmPYQk8OBYwTkuasChwv7R4=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.56.0 h1:O2sXMyJh8b7devAGdE+163xtRurt0RVpB6DIzX5vGfg=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.56.0/go.mod h1:hEpiGU18xf70qb3jbTcIggWAiEfX/cOIVc2OTe4OegA=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.56.0 h1:ZIT85vKP7LBS84XJ0WdJ3dPOX3iz4j3c0+lpajGQMyo=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.56.0/go.mod h1:rqP9UEhOXv9WhQ7Gjz+G5y/pf8+BJZW5/Ts0AhE0PwE=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.56.0 h1:0YP0+/ixwu+Uqeu/FGiBZNQ19huiUxxiPXIc9WsLKuQ=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.56.0/go.mod h1:6ZZMQhZKDvUvkJw2rc+oDP90tMMzuU/J+5HG1ZmPOmE=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.33.0 h1:l7+6kwRMJNwdCvYdDl7Eax+wzEYHSnNY7zrrfbhDdTA=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.33.0/go.mod h1:pJTkW8hEUIIi3Pf65lPZOnn4Y81yCllX6IWk2jNXdkM=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.57.0 h1:jLdiS1vO+XJFyDSWRHBx56r4s/NNtcl5J6KyCcWUX/w=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.57.0/go.mod h1:8lmpHY+1VRoteiOwyrQMDt1YGXOrFKCz+1wJW7n3ODY=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.57.0 h1:cSjUzZ7KU8hicTgzaSv9NmSyM9fTVK3y5lsBUl3wOis=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.57.0/go.mod h1:dzcEjy1WJ0Q4u9twNR3LcLhNoYMRCrMCMafpxa0TjPQ=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.57.0 h1:RoO5+d7uCmDqovLrHCr2/BuViUXvdcrNxyNM1pN9dDQ=
github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.57.0/go.mod h1:YqwkQPrWSC7+byyc1VlKbWLBF5JsW5IoL6xUkemYSXk=
github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY=
github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU=
github.com/VictoriaMetrics/VictoriaLogs v1.50.1-0.20260415124154-6b7a6357aec0 h1:2x1Tszv41PnCdSMumEtejz/On1RQ45kHQ+hhKT53sOk=
github.com/VictoriaMetrics/VictoriaLogs v1.50.1-0.20260415124154-6b7a6357aec0/go.mod h1:fQtmzaSUL+HJmHozeAKmnTJTOMBT+vBccv/VWQEwhUQ=
github.com/VictoriaMetrics/VictoriaLogs v1.121.1-0.20260616132739-c901a1e31cb3 h1:3eP8RRZitbga5EYiQ3IANrMPxpBwMAX4VA6akDaXwpU=
github.com/VictoriaMetrics/VictoriaLogs v1.121.1-0.20260616132739-c901a1e31cb3/go.mod h1:H4sDxcvk6OmC6zOt++IlDyrwfbn4F1eSLwMpR+kpRt8=
github.com/VictoriaMetrics/easyproto v1.2.0 h1:FJT9uNXA2isppFuJErbLqD306KoFlehl7Wn2dg/6oIE=
github.com/VictoriaMetrics/easyproto v1.2.0/go.mod h1:QlGlzaJnDfFd8Lk6Ci/fuLxfTo3/GThPs2KH23mv710=
github.com/VictoriaMetrics/fastcache v1.13.3 h1:rBabE0iIxcqKEMCwUmwHZ9dgEqXerg8FRbRDUvC7OVc=
github.com/VictoriaMetrics/fastcache v1.13.3/go.mod h1:hHXhl4DA2fTL2HTZDJFXWgW0LNjo6B+4aj2Wmng3TjU=
github.com/VictoriaMetrics/metrics v1.43.2 h1:+8pIQEGwchKS5CYFyvv3LKvNXGi7baZ9hmIV4RHqibY=
github.com/VictoriaMetrics/metrics v1.43.2/go.mod h1:xDM82ULLYCYdFRgQ2JBxi8Uf1+8En1So9YUwlGTOqTc=
github.com/VictoriaMetrics/metricsql v0.87.0 h1:Koxh3GkB/Z0f3O0bEChVFxiE4YZoxYyn5TzmGJfSfaw=
github.com/VictoriaMetrics/metricsql v0.87.0/go.mod h1:d4EisFO6ONP/HIGDYTAtwrejJBBeKGQYiRl095bS4QQ=
github.com/VictoriaMetrics/metricsql v0.87.1 h1:GdIblCDgXsrBJcBSDtFT8SLK7P+QHijdQmcr4L/f0Go=
github.com/VictoriaMetrics/metricsql v0.87.1/go.mod h1:d4EisFO6ONP/HIGDYTAtwrejJBBeKGQYiRl095bS4QQ=
github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow=
github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4=
github.com/alecthomas/units v0.0.0-20240927000941-0f3dac36c52b h1:mimo19zliBX/vSQ6PWWSL9lK8qwHozUj03+zLoEB8O0=
@@ -70,64 +70,66 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah
github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM=
github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA=
github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4=
github.com/aws/aws-sdk-go-v2 v1.41.5 h1:dj5kopbwUsVUVFgO4Fi5BIT3t4WyqIDjGKCangnV/yY=
github.com/aws/aws-sdk-go-v2 v1.41.5/go.mod h1:mwsPRE8ceUUpiTgF7QmQIJ7lgsKUPQOUl3o72QBrE1o=
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.8 h1:eBMB84YGghSocM7PsjmmPffTa+1FBUeNvGvFou6V/4o=
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.8/go.mod h1:lyw7GFp3qENLh7kwzf7iMzAxDn+NzjXEAGjKS2UOKqI=
github.com/aws/aws-sdk-go-v2/config v1.32.14 h1:opVIRo/ZbbI8OIqSOKmpFaY7IwfFUOCCXBsUpJOwDdI=
github.com/aws/aws-sdk-go-v2/config v1.32.14/go.mod h1:U4/V0uKxh0Tl5sxmCBZ3AecYny4UNlVmObYjKuuaiOo=
github.com/aws/aws-sdk-go-v2/credentials v1.19.14 h1:n+UcGWAIZHkXzYt87uMFBv/l8THYELoX6gVcUvgl6fI=
github.com/aws/aws-sdk-go-v2/credentials v1.19.14/go.mod h1:cJKuyWB59Mqi0jM3nFYQRmnHVQIcgoxjEMAbLkpr62w=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.21 h1:NUS3K4BTDArQqNu2ih7yeDLaS3bmHD0YndtA6UP884g=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.21/go.mod h1:YWNWJQNjKigKY1RHVJCuupeWDrrHjRqHm0N9rdrWzYI=
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.13 h1:uMC4oL6G3MNhodo358QEqSDjrgvzV3TUQ58nyQSGq2E=
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.13/go.mod h1:Cer86AE2686DvVUe57LPve3jUBmbujuaonSX8pNzGgw=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.21 h1:Rgg6wvjjtX8bNHcvi9OnXWwcE0a2vGpbwmtICOsvcf4=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.21/go.mod h1:A/kJFst/nm//cyqonihbdpQZwiUhhzpqTsdbhDdRF9c=
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.21 h1:PEgGVtPoB6NTpPrBgqSE5hE/o47Ij9qk/SEZFbUOe9A=
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.21/go.mod h1:p+hz+PRAYlY3zcpJhPwXlLC4C+kqn70WIHwnzAfs6ps=
github.com/aws/aws-sdk-go-v2/internal/ini v1.8.6 h1:qYQ4pzQ2Oz6WpQ8T3HvGHnZydA72MnLuFK9tJwmrbHw=
github.com/aws/aws-sdk-go-v2/internal/ini v1.8.6/go.mod h1:O3h0IK87yXci+kg6flUKzJnWeziQUKciKrLjcatSNcY=
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.22 h1:rWyie/PxDRIdhNf4DzRk0lvjVOqFJuNnO8WwaIRVxzQ=
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.22/go.mod h1:zd/JsJ4P7oGfUhXn1VyLqaRZwPmZwg44Jf2dS84Dm3Y=
github.com/aws/aws-sdk-go-v2/service/ec2 v1.296.0 h1:98Miqj16un1WLNyM1RjVDhXYumhqZrQfAeG8i4jPG6o=
github.com/aws/aws-sdk-go-v2/service/ec2 v1.296.0/go.mod h1:T6ndRfdhnXLIY5oKBHjYZDVj706los2zGdpThppquvA=
github.com/aws/aws-sdk-go-v2/service/ecs v1.74.0 h1:YS5TXaEvzDb+sV+wdQFUtuCAk0GeFR9Ai6HFdxpz6q8=
github.com/aws/aws-sdk-go-v2/service/ecs v1.74.0/go.mod h1:10kBgdaNJz0FO/+JWDUH+0rtSjkn5yafgavDDmmhFzs=
github.com/aws/aws-sdk-go-v2/service/elasticache v1.51.12 h1:S066ajzfPRCSW4lsSHOYglne6SNi2CHt1u5omzW1RBg=
github.com/aws/aws-sdk-go-v2/service/elasticache v1.51.12/go.mod h1:86SE4NcXxbxr8KTG3yOyDmd4HyiFmKl8TexXnhYJ+Bw=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.7 h1:5EniKhLZe4xzL7a+fU3C2tfUN4nWIqlLesfrjkuPFTY=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.7/go.mod h1:x0nZssQ3qZSnIcePWLvcoFisRXJzcTVvYpAAdYX8+GI=
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.13 h1:JRaIgADQS/U6uXDqlPiefP32yXTda7Kqfx+LgspooZM=
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.13/go.mod h1:CEuVn5WqOMilYl+tbccq8+N2ieCy0gVn3OtRb0vBNNM=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.21 h1:c31//R3xgIJMSC8S6hEVq+38DcvUlgFY0FM6mSI5oto=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.21/go.mod h1:r6+pf23ouCB718FUxaqzZdbpYFyDtehyZcmP5KL9FkA=
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.21 h1:ZlvrNcHSFFWURB8avufQq9gFsheUgjVD9536obIknfM=
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.21/go.mod h1:cv3TNhVrssKR0O/xxLJVRfd2oazSnZnkUeTf6ctUwfQ=
github.com/aws/aws-sdk-go-v2/service/kafka v1.49.1 h1:BgBatWcQIFqF1l6KGHjv66V0d/ISnWrTwxDx/Jf6EJM=
github.com/aws/aws-sdk-go-v2/service/kafka v1.49.1/go.mod h1:pMpys+PlrN//vj8j5s0oOAMJjauj81VkHzIZxPVWOro=
github.com/aws/aws-sdk-go-v2/service/lightsail v1.51.0 h1:cg6PxzoIide2wiEyLfikOFN+XwHafwR8p5+L9U1E8dQ=
github.com/aws/aws-sdk-go-v2/service/lightsail v1.51.0/go.mod h1:YvX7hjUWecrKX8fBkbEncyddEW85xjNH+u5JHioITOw=
github.com/aws/aws-sdk-go-v2/service/rds v1.117.0 h1:T1Xe9sYxSUUQOvd1RsFeVk/IXFPdqSiN0atXu/Hy/8A=
github.com/aws/aws-sdk-go-v2/service/rds v1.117.0/go.mod h1:QbXW4coAMakHQhf1qhE0eVVCen9gwB/Kvn+HHHKhpGY=
github.com/aws/aws-sdk-go-v2/service/s3 v1.99.0 h1:hlSuz394kV0vhv9drL5lhuEFbEOEP1VyQpy15qWh1Pk=
github.com/aws/aws-sdk-go-v2/service/s3 v1.99.0/go.mod h1:uoA43SdFwacedBfSgfFSjjCvYe8aYBS7EnU5GZ/YKMM=
github.com/aws/aws-sdk-go-v2/service/signin v1.0.9 h1:QKZH0S178gCmFEgst8hN0mCX1KxLgHBKKY/CLqwP8lg=
github.com/aws/aws-sdk-go-v2/service/signin v1.0.9/go.mod h1:7yuQJoT+OoH8aqIxw9vwF+8KpvLZ8AWmvmUWHsGQZvI=
github.com/aws/aws-sdk-go-v2/service/sso v1.30.15 h1:lFd1+ZSEYJZYvv9d6kXzhkZu07si3f+GQ1AaYwa2LUM=
github.com/aws/aws-sdk-go-v2/service/sso v1.30.15/go.mod h1:WSvS1NLr7JaPunCXqpJnWk1Bjo7IxzZXrZi1QQCkuqM=
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.19 h1:dzztQ1YmfPrxdrOiuZRMF6fuOwWlWpD2StNLTceKpys=
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.19/go.mod h1:YO8TrYtFdl5w/4vmjL8zaBSsiNp3w0L1FfKVKenZT7w=
github.com/aws/aws-sdk-go-v2/service/sts v1.41.10 h1:p8ogvvLugcR/zLBXTXrTkj0RYBUdErbMnAFFp12Lm/U=
github.com/aws/aws-sdk-go-v2/service/sts v1.41.10/go.mod h1:60dv0eZJfeVXfbT1tFJinbHrDfSJ2GZl4Q//OSSNAVw=
github.com/aws/smithy-go v1.24.3 h1:XgOAaUgx+HhVBoP4v8n6HCQoTRDhoMghKqw4LNHsDNg=
github.com/aws/smithy-go v1.24.3/go.mod h1:YE2RhdIuDbA5E5bTdciG9KrW3+TiEONeUWCqxX9i1Fc=
github.com/aws/aws-sdk-go v1.55.8 h1:JRmEUbU52aJQZ2AjX4q4Wu7t4uZjOu71uyNmaWlUkJQ=
github.com/aws/aws-sdk-go v1.55.8/go.mod h1:ZkViS9AqA6otK+JBBNH2++sx1sgxrPKcSzPPvQkUtXk=
github.com/aws/aws-sdk-go-v2 v1.42.0 h1:XvXMJTkFQtpBKIWZnmr9ZEOc2InWM2yldjXEJ/bymhA=
github.com/aws/aws-sdk-go-v2 v1.42.0/go.mod h1:27+ACypSLljLAEKsCYOmrjKh83vuTRkuAe9Uv/3A4bg=
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.13 h1:p1BBrg/Hhp6uK7zpejeI8QFXHJeC/mynzi04Sl03k9g=
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.13/go.mod h1:8cIfkE9MDhkRZGpQ22aV6/lkYeYSozpz16Smrs5x4Ls=
github.com/aws/aws-sdk-go-v2/config v1.32.25 h1:ACCejvStYoilgwrfegSt5ZntCbPrk52qfwyNcnl3omM=
github.com/aws/aws-sdk-go-v2/config v1.32.25/go.mod h1:LJyU8sDRbXUxFn8xMJIGP+v9QYYwveNLI8a/giAOiAs=
github.com/aws/aws-sdk-go-v2/credentials v1.19.24 h1:2hQqYCV9yqyePQ9o6dCrZc/zO8U3TwPr9mIKlZnPu/I=
github.com/aws/aws-sdk-go-v2/credentials v1.19.24/go.mod h1:IDwpACtwqHLISdzfwUUNq4P9DsB/h5BLg4FwJPNfqFY=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.29 h1:r6qZHbT+wxgWO/e9vYNUEtg7lv5+UN3pRqKhLXvnArg=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.29/go.mod h1:QRnaRcTVGKPGRy8w78HMQtKUGRYcnMZAANATkeVA6Mo=
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.27 h1:gb+HtIZdwcIoLxv/xwGumQr1DmGmGGCQnjKKVVSMYsU=
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.22.27/go.mod h1:2b/8jZl/qwUMBZpSAcxX+IdM3zj6RUyfnB2IdLt9I+I=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.29 h1:f3vKqSo13fhTYb+JEcXwXefZQE26I1FB5eTSniU67ko=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.29/go.mod h1:MzoLFUArKGpGD+ukmPiTPG1X5x4o6M2kq4v2dr1FiEc=
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.29 h1:RdwIf/CuUsvJX3RgJagbOyotl/cxoLY4xviKuE7p2GY=
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.29/go.mod h1:71wt8W2EgswdZy9Mf9KNnzxZ3TiZlv4caKghPktDOkA=
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.30 h1:VTGy885W5DKBxWRUJbym9hytNaYzsyaPkCHGRRMAOhU=
github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.30/go.mod h1:AS0HycUvJRFvTt613AYDOgO2jzw+00cVSMny8XB3yMY=
github.com/aws/aws-sdk-go-v2/service/ec2 v1.304.0 h1:wZthLlYdKxBo7NpWLbl0A/8DB/QNDB+8RJa9WboK9Q0=
github.com/aws/aws-sdk-go-v2/service/ec2 v1.304.0/go.mod h1:Y95W0Hm6FYLPa6o0hbnJ+sWgmdc4ifcLFjGkdobWVhY=
github.com/aws/aws-sdk-go-v2/service/ecs v1.81.0 h1:2Sp9EwK7giQpJnQ54k0zdUh6aykmmbpEurEEygr104c=
github.com/aws/aws-sdk-go-v2/service/ecs v1.81.0/go.mod h1:TIKZ9zIFS6W2k9FeW+r5sGVnlxp+aUt9oQ/St3Suj1o=
github.com/aws/aws-sdk-go-v2/service/elasticache v1.52.2 h1:5wbCUfyxXcjIqesyVfJBBJs0bDMyejthtHyy48mfZCI=
github.com/aws/aws-sdk-go-v2/service/elasticache v1.52.2/go.mod h1:o4vQxDt6oteknUjkXIEskp0ccy+93NRTPKXw3HlVMFE=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.12 h1:ZD2+BSw9vFsNlKYIasSNt3uDbjqqXIBcM13UJv/Lx2k=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.12/go.mod h1:Ms4zlcVBbXbiP7EVLhl+lgjvA/a7YphqQ3Ih3174EmI=
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.22 h1:V51LGlOq/1VsDsHUdoklAQi7rMmx4qQubvFYAlP2254=
github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.22/go.mod h1:4Pzhyz8hJOm2bepgl+NjvRx8vlUFAIIvJnZ/MkcNPpU=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.29 h1:DRebniUGZ2MqiiIVmQJ04vIXr918hubdHMnarSLEWyU=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.29/go.mod h1:LfRkPCD8YHDM2E5eTkos2UpwYeZnBcVarTa8L59bJHA=
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.29 h1:hiME6pBzC7OTl9LMtlyTWBuEl1f4QBcUmFDKC7MLXtc=
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.29/go.mod h1:G7RP+uhagpKtKhd1BM9N6JQqjCcGEU47K5lBVZQyRQw=
github.com/aws/aws-sdk-go-v2/service/kafka v1.52.0 h1:jalIJqKvZMvJRvs6ABLX+FhHz8E9pjU03Pyml4D9r3k=
github.com/aws/aws-sdk-go-v2/service/kafka v1.52.0/go.mod h1:pW4pYNuVeScl13yqwsjLY0F/7g2YD8E0AvR6SOQsJZE=
github.com/aws/aws-sdk-go-v2/service/lightsail v1.54.0 h1:07DKnL5eKSel3XEM2UxlD/z9zUZZ6XMHLGDXkAdY4u8=
github.com/aws/aws-sdk-go-v2/service/lightsail v1.54.0/go.mod h1:Etcg8xorq1b0g0V2KMNgFjubYITZseJv08qtX/3szko=
github.com/aws/aws-sdk-go-v2/service/rds v1.118.2 h1:pkEeQneYFpTAnGhyqSbyp/DlCPPJTGt0GkWahlLYzMA=
github.com/aws/aws-sdk-go-v2/service/rds v1.118.2/go.mod h1:7gS+cGrKF0mH253QHFlStmx79ws+DlNk+04ZRfmw3U0=
github.com/aws/aws-sdk-go-v2/service/s3 v1.103.3 h1:JRseEu/vIDMaWis4bSw0QbXL+cvIGc1XnX076H5ZXLE=
github.com/aws/aws-sdk-go-v2/service/s3 v1.103.3/go.mod h1:77ZAgynvx1txMvDG8gGWoWkO1augYDxkp9JElWFgjQU=
github.com/aws/aws-sdk-go-v2/service/signin v1.2.0 h1:3nXpRcFwRCW8n7HgO2QGy0Dc20eQNfBuUemGQhpF8m8=
github.com/aws/aws-sdk-go-v2/service/signin v1.2.0/go.mod h1:LxYujSTLPRlp2vTtcUO/+1ilrew8ytt6SvQyOgejzFQ=
github.com/aws/aws-sdk-go-v2/service/sso v1.31.3 h1:ey1XLTYXb9PcLt4535632o5kCGXNXEhNb620Dqwuylo=
github.com/aws/aws-sdk-go-v2/service/sso v1.31.3/go.mod h1:Lk7PlmoTYryQmyBG0EXqj5BcUbj3whXdU2s3yGI3EAc=
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.36.6 h1:yLr03zQE/5Eu5l3QU0Si+xMbLMbSDF2YXsigqXngs6g=
github.com/aws/aws-sdk-go-v2/service/ssooidc v1.36.6/go.mod h1:Q5N6icH+KJZDLh+ESNwzdv6cZ6vLFF/egy3IOxWhmz4=
github.com/aws/aws-sdk-go-v2/service/sts v1.43.3 h1:VrIhKRCSK1umelSgB9RghvA9RTUYeQffyAS5ApXehNI=
github.com/aws/aws-sdk-go-v2/service/sts v1.43.3/go.mod h1:r8wkDOuLaaMFqFiYAb8dGY2A3gJCOujMc6CFOVC4Zhc=
github.com/aws/smithy-go v1.27.2 h1:y9NPmSE6am6LjEFPfqHqG/jJk7AauQvhCJONKh7kpzk=
github.com/aws/smithy-go v1.27.2/go.mod h1:YE2RhdIuDbA5E5bTdciG9KrW3+TiEONeUWCqxX9i1Fc=
github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 h1:6df1vn4bBlDDo4tARvBm7l6KA9iVMnE3NWizDeWSrps=
github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3/go.mod h1:CIWtjkly68+yqLPbvwwR/fjNJA/idrtULjZWh2v1ys0=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
github.com/bmatcuk/doublestar/v4 v4.10.0 h1:zU9WiOla1YA122oLM6i4EXvGW62DvKZVxIe6TYWexEs=
github.com/bmatcuk/doublestar/v4 v4.10.0/go.mod h1:xBQ8jztBU6kakFMg+8WGxn0c6z1fTSPVIjEY1Wr7jzc=
github.com/cenkalti/backoff/v5 v5.0.3 h1:ZN+IMa753KfX5hd8vVaMixjnqRZ3y8CuJKRKj1xcsSM=
github.com/cenkalti/backoff/v5 v5.0.3/go.mod h1:rkhZdG3JZukswDf7f0cwqPNk4K0sa+F97BxZthm/crw=
github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs=
github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/cheggaaa/pb/v3 v3.1.7 h1:2FsIW307kt7A/rz/ZI2lvPO+v3wKazzE4K/0LtTWsOI=
@@ -148,20 +150,20 @@ github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE=
github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgziApxA=
github.com/digitalocean/godo v1.178.0 h1:+B4xGOaoFwwwpM7TKhoyGHdmFg5eF9zDB1YfOLvNJ2E=
github.com/digitalocean/godo v1.178.0/go.mod h1:xQsWpVCCbkDrWisHA72hPzPlnC+4W5w/McZY5ij9uvU=
github.com/dgryski/go-metro v0.0.0-20250106013310-edb8663e5e33 h1:ucRHb6/lvW/+mTEIGbvhcYU3S8+uSNkuMjx/qZFfhtM=
github.com/dgryski/go-metro v0.0.0-20250106013310-edb8663e5e33/go.mod h1:c9O8+fpSOX1DM8cPNSkX/qsBWdkD4yd2dpciOWQjpBw=
github.com/digitalocean/godo v1.193.0 h1:CSbbUl5LufT75KPNvex3vDnBYjY2RfJWs7T3Ac7dHpA=
github.com/digitalocean/godo v1.193.0/go.mod h1:xQsWpVCCbkDrWisHA72hPzPlnC+4W5w/McZY5ij9uvU=
github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5QvfrDyIgxBk=
github.com/distribution/reference v0.6.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E=
github.com/docker/docker v28.5.2+incompatible h1:DBX0Y0zAjZbSrm1uzOkdr1onVghKaftjlSWt4AFexzM=
github.com/docker/docker v28.5.2+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk=
github.com/docker/go-connections v0.6.0 h1:LlMG9azAe1TqfR7sO+NJttz1gy6KO7VJBh+pMmjSD94=
github.com/docker/go-connections v0.6.0/go.mod h1:AahvXYshr6JgfUJGdDCs2b5EZG/vmaMAntpSFH5BFKE=
github.com/docker/go-connections v0.7.0 h1:6SsRfJddP22WMrCkj19x9WKjEDTB+ahsdiGYf0mN39c=
github.com/docker/go-connections v0.7.0/go.mod h1:no1qkHdjq7kLMGUXYAduOhYPSJxxvgWBh7ogVvptn3Q=
github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4=
github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
github.com/edsrzf/mmap-go v1.2.0 h1:hXLYlkbaPzt1SaQk+anYwKSRNhufIDCchSPkUD6dD84=
github.com/edsrzf/mmap-go v1.2.0/go.mod h1:19H/e8pUPLicwkyNgOykDXkJ9F0MHE+Z52B8EIth78Q=
github.com/emicklei/go-restful/v3 v3.12.2 h1:DhwDP0vY3k8ZzE0RunuJy8GhNpPL6zqLkDf9B/a0/xU=
github.com/emicklei/go-restful/v3 v3.12.2/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc=
github.com/edsrzf/mmap-go v1.2.1-0.20241212181136-fad1cd13edbd h1:I4PrRZuNMeDP3VbFrak4QsqwO5tWkQf0tqrrr1L2DsU=
github.com/edsrzf/mmap-go v1.2.1-0.20241212181136-fad1cd13edbd/go.mod h1:19H/e8pUPLicwkyNgOykDXkJ9F0MHE+Z52B8EIth78Q=
github.com/emicklei/go-restful/v3 v3.13.0 h1:C4Bl2xDndpU6nJ4bc1jXd+uTmYPVUwkD6bFY/oTyCes=
github.com/emicklei/go-restful/v3 v3.13.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc=
github.com/envoyproxy/go-control-plane v0.14.0 h1:hbG2kr4RuFj222B6+7T83thSPqLjwBIfQawTkC++2HA=
github.com/envoyproxy/go-control-plane v0.14.0/go.mod h1:NcS5X47pLl/hfqxU70yPwL9ZMkUlwlKxtAohpi2wBEU=
github.com/envoyproxy/go-control-plane/envoy v1.37.0 h1:u3riX6BoYRfF4Dr7dwSOroNfdSbEPe9Yyl09/B6wBrQ=
@@ -174,12 +176,12 @@ github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb h1:IT4JYU7k4ikYg1S
github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb/go.mod h1:bH6Xx7IW64qjjJq8M2u4dxNaBiDfKK+z/3eGDpXEQhc=
github.com/fatih/color v1.19.0 h1:Zp3PiM21/9Ld6FzSKyL5c/BULoe/ONr9KlbYVOfG8+w=
github.com/fatih/color v1.19.0/go.mod h1:zNk67I0ZUT1bEGsSGyCZYZNrHuTkJJB+r6Q9VuMi0LE=
github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg=
github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
github.com/fsnotify/fsnotify v1.9.0 h1:2Ml+OJNzbYCTzsxtv8vKSFD9PbJjmhYF14k/jKC7S9k=
github.com/fsnotify/fsnotify v1.9.0/go.mod h1:8jBTzvmWwFyi3Pb8djgCCO5IBqzKJ/Jwo8TRcHyHii0=
github.com/fxamacker/cbor/v2 v2.9.1 h1:2rWm8B193Ll4VdjsJY28jxs70IdDsHRWgQYAI80+rMQ=
github.com/fxamacker/cbor/v2 v2.9.1/go.mod h1:vM4b+DJCtHn+zz7h3FFp/hDAI9WNWCsZj23V5ytsSxQ=
github.com/felixge/httpsnoop v1.1.0 h1:3YtUj32ZZkqZtt3sZZsClsymw/QDuVfpNhoA31zeORc=
github.com/felixge/httpsnoop v1.1.0/go.mod h1:Zqxgdd+1Rkcz8euOqdr7lqgCRJztwr5hp9vDSi5UZCE=
github.com/fsnotify/fsnotify v1.10.1 h1:b0/UzAf9yR5rhf3RPm9gf3ehBPpf0oZKIjtpKrx59Ho=
github.com/fsnotify/fsnotify v1.10.1/go.mod h1:TLheqan6HD6GBK6PrDWyDPBaEV8LspOxvPSjC+bVfgo=
github.com/fxamacker/cbor/v2 v2.9.2 h1:X4Ksno9+x3cz0TZv69ec1hxP/+tymuR8PXQJyDwfh78=
github.com/fxamacker/cbor/v2 v2.9.2/go.mod h1:vM4b+DJCtHn+zz7h3FFp/hDAI9WNWCsZj23V5ytsSxQ=
github.com/go-jose/go-jose/v4 v4.1.4 h1:moDMcTHmvE6Groj34emNPLs/qtYXRVcd6S7NHbHz3kA=
github.com/go-jose/go-jose/v4 v4.1.4/go.mod h1:x4oUasVrzR7071A4TnHLGSPpNOm2a21K9Kf04k1rs08=
github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
@@ -187,34 +189,34 @@ github.com/go-logr/logr v1.4.3 h1:CjnDlHq8ikf6E492q6eKboGOC0T8CDaOvkHCIg8idEI=
github.com/go-logr/logr v1.4.3/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
github.com/go-openapi/jsonpointer v0.22.5 h1:8on/0Yp4uTb9f4XvTrM2+1CPrV05QPZXu+rvu2o9jcA=
github.com/go-openapi/jsonpointer v0.22.5/go.mod h1:gyUR3sCvGSWchA2sUBJGluYMbe1zazrYWIkWPjjMUY0=
github.com/go-openapi/jsonreference v0.21.4 h1:24qaE2y9bx/q3uRK/qN+TDwbok1NhbSmGjjySRCHtC8=
github.com/go-openapi/jsonreference v0.21.4/go.mod h1:rIENPTjDbLpzQmQWCj5kKj3ZlmEh+EFVbz3RTUh30/4=
github.com/go-openapi/swag v0.25.4 h1:OyUPUFYDPDBMkqyxOTkqDYFnrhuhi9NR6QVUvIochMU=
github.com/go-openapi/swag v0.25.4/go.mod h1:zNfJ9WZABGHCFg2RnY0S4IOkAcVTzJ6z2Bi+Q4i6qFQ=
github.com/go-openapi/swag/cmdutils v0.25.4 h1:8rYhB5n6WawR192/BfUu2iVlxqVR9aRgGJP6WaBoW+4=
github.com/go-openapi/swag/cmdutils v0.25.4/go.mod h1:pdae/AFo6WxLl5L0rq87eRzVPm/XRHM3MoYgRMvG4A0=
github.com/go-openapi/swag/conv v0.25.4 h1:/Dd7p0LZXczgUcC/Ikm1+YqVzkEeCc9LnOWjfkpkfe4=
github.com/go-openapi/swag/conv v0.25.4/go.mod h1:3LXfie/lwoAv0NHoEuY1hjoFAYkvlqI/Bn5EQDD3PPU=
github.com/go-openapi/swag/fileutils v0.25.4 h1:2oI0XNW5y6UWZTC7vAxC8hmsK/tOkWXHJQH4lKjqw+Y=
github.com/go-openapi/swag/fileutils v0.25.4/go.mod h1:cdOT/PKbwcysVQ9Tpr0q20lQKH7MGhOEb6EwmHOirUk=
github.com/go-openapi/swag/jsonname v0.25.5 h1:8p150i44rv/Drip4vWI3kGi9+4W9TdI3US3uUYSFhSo=
github.com/go-openapi/swag/jsonname v0.25.5/go.mod h1:jNqqikyiAK56uS7n8sLkdaNY/uq6+D2m2LANat09pKU=
github.com/go-openapi/swag/jsonutils v0.25.4 h1:VSchfbGhD4UTf4vCdR2F4TLBdLwHyUDTd1/q4i+jGZA=
github.com/go-openapi/swag/jsonutils v0.25.4/go.mod h1:7OYGXpvVFPn4PpaSdPHJBtF0iGnbEaTk8AvBkoWnaAY=
github.com/go-openapi/swag/loading v0.25.4 h1:jN4MvLj0X6yhCDduRsxDDw1aHe+ZWoLjW+9ZQWIKn2s=
github.com/go-openapi/swag/loading v0.25.4/go.mod h1:rpUM1ZiyEP9+mNLIQUdMiD7dCETXvkkC30z53i+ftTE=
github.com/go-openapi/swag/mangling v0.25.4 h1:2b9kBJk9JvPgxr36V23FxJLdwBrpijI26Bx5JH4Hp48=
github.com/go-openapi/swag/mangling v0.25.4/go.mod h1:6dxwu6QyORHpIIApsdZgb6wBk/DPU15MdyYj/ikn0Hg=
github.com/go-openapi/swag/netutils v0.25.4 h1:Gqe6K71bGRb3ZQLusdI8p/y1KLgV4M/k+/HzVSqT8H0=
github.com/go-openapi/swag/netutils v0.25.4/go.mod h1:m2W8dtdaoX7oj9rEttLyTeEFFEBvnAx9qHd5nJEBzYg=
github.com/go-openapi/swag/stringutils v0.25.4 h1:O6dU1Rd8bej4HPA3/CLPciNBBDwZj9HiEpdVsb8B5A8=
github.com/go-openapi/swag/stringutils v0.25.4/go.mod h1:GTsRvhJW5xM5gkgiFe0fV3PUlFm0dr8vki6/VSRaZK0=
github.com/go-openapi/swag/typeutils v0.25.4 h1:1/fbZOUN472NTc39zpa+YGHn3jzHWhv42wAJSN91wRw=
github.com/go-openapi/swag/typeutils v0.25.4/go.mod h1:Ou7g//Wx8tTLS9vG0UmzfCsjZjKhpjxayRKTHXf2pTE=
github.com/go-openapi/swag/yamlutils v0.25.4 h1:6jdaeSItEUb7ioS9lFoCZ65Cne1/RZtPBZ9A56h92Sw=
github.com/go-openapi/swag/yamlutils v0.25.4/go.mod h1:MNzq1ulQu+yd8Kl7wPOut/YHAAU/H6hL91fF+E2RFwc=
github.com/go-openapi/jsonpointer v0.23.1 h1:1HBACs7XIwR2RcmItfdSFlALhGbe6S92p0ry4d1GWg4=
github.com/go-openapi/jsonpointer v0.23.1/go.mod h1:iWRmZTrGn7XwYhtPt/fvdSFj1OfNBngqRT2UG3BxSqY=
github.com/go-openapi/jsonreference v0.21.5 h1:6uCGVXU/aNF13AQNggxfysJ+5ZcU4nEAe+pJyVWRdiE=
github.com/go-openapi/jsonreference v0.21.5/go.mod h1:u25Bw85sX4E2jzFodh1FOKMTZLcfifd1Q+iKKOUxExw=
github.com/go-openapi/swag v0.25.5 h1:pNkwbUEeGwMtcgxDr+2GBPAk4kT+kJ+AaB+TMKAg+TU=
github.com/go-openapi/swag v0.25.5/go.mod h1:B3RT6l8q7X803JRxa2e59tHOiZlX1t8viplOcs9CwTA=
github.com/go-openapi/swag/cmdutils v0.25.5 h1:yh5hHrpgsw4NwM9KAEtaDTXILYzdXh/I8Whhx9hKj7c=
github.com/go-openapi/swag/cmdutils v0.25.5/go.mod h1:pdae/AFo6WxLl5L0rq87eRzVPm/XRHM3MoYgRMvG4A0=
github.com/go-openapi/swag/conv v0.25.5 h1:wAXBYEXJjoKwE5+vc9YHhpQOFj2JYBMF2DUi+tGu97g=
github.com/go-openapi/swag/conv v0.25.5/go.mod h1:CuJ1eWvh1c4ORKx7unQnFGyvBbNlRKbnRyAvDvzWA4k=
github.com/go-openapi/swag/fileutils v0.25.5 h1:B6JTdOcs2c0dBIs9HnkyTW+5gC+8NIhVBUwERkFhMWk=
github.com/go-openapi/swag/fileutils v0.25.5/go.mod h1:V3cT9UdMQIaH4WiTrUc9EPtVA4txS0TOmRURmhGF4kc=
github.com/go-openapi/swag/jsonname v0.26.0 h1:gV1NFX9M8avo0YSpmWogqfQISigCmpaiNci8cGECU5w=
github.com/go-openapi/swag/jsonname v0.26.0/go.mod h1:urBBR8bZNoDYGr653ynhIx+gTeIz0ARZxHkAPktJK2M=
github.com/go-openapi/swag/jsonutils v0.25.5 h1:XUZF8awQr75MXeC+/iaw5usY/iM7nXPDwdG3Jbl9vYo=
github.com/go-openapi/swag/jsonutils v0.25.5/go.mod h1:48FXUaz8YsDAA9s5AnaUvAmry1UcLcNVWUjY42XkrN4=
github.com/go-openapi/swag/loading v0.25.5 h1:odQ/umlIZ1ZVRteI6ckSrvP6e2w9UTF5qgNdemJHjuU=
github.com/go-openapi/swag/loading v0.25.5/go.mod h1:I8A8RaaQ4DApxhPSWLNYWh9NvmX2YKMoB9nwvv6oW6g=
github.com/go-openapi/swag/mangling v0.25.5 h1:hyrnvbQRS7vKePQPHHDso+k6CGn5ZBs5232UqWZmJZw=
github.com/go-openapi/swag/mangling v0.25.5/go.mod h1:6hadXM/o312N/h98RwByLg088U61TPGiltQn71Iw0NY=
github.com/go-openapi/swag/netutils v0.25.5 h1:LZq2Xc2QI8+7838elRAaPCeqJnHODfSyOa7ZGfxDKlU=
github.com/go-openapi/swag/netutils v0.25.5/go.mod h1:lHbtmj4m57APG/8H7ZcMMSWzNqIQcu0RFiXrPUara14=
github.com/go-openapi/swag/stringutils v0.25.5 h1:NVkoDOA8YBgtAR/zvCx5rhJKtZF3IzXcDdwOsYzrB6M=
github.com/go-openapi/swag/stringutils v0.25.5/go.mod h1:PKK8EZdu4QJq8iezt17HM8RXnLAzY7gW0O1KKarrZII=
github.com/go-openapi/swag/typeutils v0.25.5 h1:EFJ+PCga2HfHGdo8s8VJXEVbeXRCYwzzr9u4rJk7L7E=
github.com/go-openapi/swag/typeutils v0.25.5/go.mod h1:itmFmScAYE1bSD8C4rS0W+0InZUBrB2xSPbWt6DLGuc=
github.com/go-openapi/swag/yamlutils v0.25.5 h1:kASCIS+oIeoc55j28T4o8KwlV2S4ZLPT6G0iq2SSbVQ=
github.com/go-openapi/swag/yamlutils v0.25.5/go.mod h1:Gek1/SjjfbYvM+Iq4QGwa/2lEXde9n2j4a3wI3pNuOQ=
github.com/go-resty/resty/v2 v2.17.2 h1:FQW5oHYcIlkCNrMD2lloGScxcHJ0gkjshV3qcQAyHQk=
github.com/go-resty/resty/v2 v2.17.2/go.mod h1:kCKZ3wWmwJaNc7S29BRtUhJwy7iqmn+2mLtQrOyQlVA=
github.com/go-viper/mapstructure/v2 v2.5.0 h1:vM5IJoUAy3d7zRSVtIwQgBj7BiWtMPfmPEgAXnvj1Ro=
@@ -244,12 +246,12 @@ github.com/google/s2a-go v0.1.9 h1:LGD7gtMgezd8a/Xak7mEWL0PjoTQFvpRudN895yqKW0=
github.com/google/s2a-go v0.1.9/go.mod h1:YA0Ei2ZQL3acow2O62kdp9UlnvMmU7kA6Eutn0dXayM=
github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=
github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/enterprise-certificate-proxy v0.3.14 h1:yh8ncqsbUY4shRD5dA6RlzjJaT4hi3kII+zYw8wmLb8=
github.com/googleapis/enterprise-certificate-proxy v0.3.14/go.mod h1:vqVt9yG9480NtzREnTlmGSBmFrA+bzb0yl0TxoBQXOg=
github.com/googleapis/enterprise-certificate-proxy v0.3.16 h1:F/VPrx0YPBdksZJQdCAp0WUsqnNmZpUZszzfYt0M5Dw=
github.com/googleapis/enterprise-certificate-proxy v0.3.16/go.mod h1:9Yb0eAkH/Xqhvv3zbeKf/+wMJqCeocWc6KIhDvEAuYE=
github.com/googleapis/gax-go/v2 v2.22.0 h1:PjIWBpgGIVKGoCXuiCoP64altEJCj3/Ei+kSU5vlZD4=
github.com/googleapis/gax-go/v2 v2.22.0/go.mod h1:irWBbALSr0Sk3qlqb9SyJ1h68WjgeFuiOzI4Rqw5+aY=
github.com/gophercloud/gophercloud/v2 v2.11.1 h1:jCs4vLH8sJgRqrPzqVfWgl7uI6JnIIlsgeIRM0uHjxY=
github.com/gophercloud/gophercloud/v2 v2.11.1/go.mod h1:Rm0YvKQ4QYX2rY9XaDKnjRzSGwlG5ge4h6ABYnmkKQM=
github.com/gophercloud/gophercloud/v2 v2.12.0 h1:Gxmc/Bog1UDKkxTcQW7MSPTDviJXpLeEgVeN5KrxoCo=
github.com/gophercloud/gophercloud/v2 v2.12.0/go.mod h1:H7TTOxbLy8RIaHSNhI2GCrWIzw4Xpw8Xn2mBhCUT5kA=
github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674 h1:JeSE6pjso5THxAzdVpqr6/geYxZytqFMBCOtn/ujyeo=
github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674/go.mod h1:r4w70xmWCQKmi1ONH4KIaBptdivuRPyosB9RmPlGEwA=
github.com/grafana/regexp v0.0.0-20250905093917-f7b3be9d1853 h1:cLN4IBkmkYZNnk7EAJ0BHIethd+J6LqxFNw5mSiI2bM=
@@ -276,32 +278,36 @@ github.com/hashicorp/go-version v1.9.0 h1:CeOIz6k+LoN3qX9Z0tyQrPtiB1DFYRPfCIBtaX
github.com/hashicorp/go-version v1.9.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA=
github.com/hashicorp/golang-lru v0.6.0 h1:uL2shRDx7RTrOrTCUZEGP/wJUFiUI8QT6E7z5o8jga4=
github.com/hashicorp/golang-lru v0.6.0/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
github.com/hashicorp/nomad/api v0.0.0-20260324203407-b27b0c2e019a h1:HGwfgBNl90YBiHdbzZ/+8aMxO1UL9B/yNTAXa8iB8z8=
github.com/hashicorp/nomad/api v0.0.0-20260324203407-b27b0c2e019a/go.mod h1:KkLNLU0Nyfh5jWsFoF/PsmMbKpRIAoIV4lmQoJWgKCk=
github.com/hashicorp/nomad/api v0.0.0-20260528135333-5b027732945f h1:sdf4a6FF3tC1/c0buuizLAwZa/xLu4gWD87qWrzQLvo=
github.com/hashicorp/nomad/api v0.0.0-20260528135333-5b027732945f/go.mod h1:Kr8imJwigbQ/50BqVae2+JL+AyX+FnzbnuCoIFb6iYg=
github.com/hashicorp/serf v0.10.1 h1:Z1H2J60yRKvfDYAOZLd2MU0ND4AH/WDz7xYHDWQsIPY=
github.com/hashicorp/serf v0.10.1/go.mod h1:yL2t6BqATOLGc5HF7qbFkTfXoPIY0WZdWHfEvMqbG+4=
github.com/hetznercloud/hcloud-go/v2 v2.36.0 h1:HlLL/aaVXUulqe+rsjoJmrxKhPi1MflL5O9iq5QEtvo=
github.com/hetznercloud/hcloud-go/v2 v2.36.0/go.mod h1:MnN/QJEa/RYNQiiVoJjNHPntM7Z1wlYPgJ2HA40/cDE=
github.com/hetznercloud/hcloud-go/v2 v2.41.2 h1:fO5zsMgp5oejrtnFj8mYuqlp+iMuirpaKv4b5FYNRdQ=
github.com/hetznercloud/hcloud-go/v2 v2.41.2/go.mod h1:9OGvC//jbHE4sv2Oyo0bQ2vEWuUMKYoNMyj9Qxz2qcc=
github.com/influxdata/influxdb v1.12.4 h1:vn/1rvFYkYpg9efRw79+PUPPnMX7HwyJV+hDIB9IrOQ=
github.com/influxdata/influxdb v1.12.4/go.mod h1:czsGl4TCm2kWtzEHsGh74Nye77o/KgmKsLtF4/L9QVc=
github.com/ionos-cloud/sdk-go/v6 v6.3.6 h1:l/TtKgdQ1wUH3DDe2SfFD78AW+TJWdEbDpQhHkWd6CM=
github.com/ionos-cloud/sdk-go/v6 v6.3.6/go.mod h1:nUGHP4kZHAZngCVr4v6C8nuargFrtvt7GrzH/hqn7c4=
github.com/ionos-cloud/sdk-go/v6 v6.3.7 h1:t773JkC/asnyVqeQ+OvN9WCRZuosSoPtJfyM82EFCWY=
github.com/ionos-cloud/sdk-go/v6 v6.3.7/go.mod h1:nUGHP4kZHAZngCVr4v6C8nuargFrtvt7GrzH/hqn7c4=
github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg=
github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA=
github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM=
github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo=
github.com/kamstrup/intmap v0.5.2 h1:qnwBm1mh4XAnW9W9Ue9tZtTff8pS6+s6iKF6JRIV2Dk=
github.com/kamstrup/intmap v0.5.2/go.mod h1:gWUVWHKzWj8xpJVFf5GC0O26bWmv3GqdnIX/LMT6Aq4=
github.com/keybase/go-keychain v0.0.1 h1:way+bWYa6lDppZoZcgMbYsvC7GxljxrskdNInRtuthU=
github.com/keybase/go-keychain v0.0.1/go.mod h1:PdEILRW3i9D8JcdM+FmY6RwkHGnhHxXwkPPMeUgOK1k=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
github.com/klauspost/compress v1.18.5 h1:/h1gH5Ce+VWNLSWqPzOVn6XBO+vJbCNGvjoaGBFW2IE=
github.com/klauspost/compress v1.18.5/go.mod h1:cwPg85FWrGar70rWktvGQj8/hthj3wpl0PGDogxkrSQ=
github.com/klauspost/compress v1.18.6 h1:2jupLlAwFm95+YDR+NwD2MEfFO9d4z4Prjl1XXDjuao=
github.com/klauspost/compress v1.18.6/go.mod h1:cwPg85FWrGar70rWktvGQj8/hthj3wpl0PGDogxkrSQ=
github.com/knadh/koanf/maps v0.1.2 h1:RBfmAW5CnZT+PJ1CVc1QSJKf4Xu9kxfQgYVQSu8hpbo=
github.com/knadh/koanf/maps v0.1.2/go.mod h1:npD/QZY3V6ghQDdcQzl1W4ICNVTkohC8E73eI2xW4yI=
github.com/knadh/koanf/providers/confmap v1.0.0 h1:mHKLJTE7iXEys6deO5p6olAiZdG5zwp8Aebir+/EaRE=
github.com/knadh/koanf/providers/confmap v1.0.0/go.mod h1:txHYHiI2hAtF0/0sCmcuol4IDcuQbKTybiB1nOcUo1A=
github.com/knadh/koanf/v2 v2.3.4 h1:fnynNSDlujWE+v83hAp8wKr/cdoxHLO0629SN+U8Urc=
github.com/knadh/koanf/v2 v2.3.4/go.mod h1:gRb40VRAbd4iJMYYD5IxZ6hfuopFcXBpc9bbQpZwo28=
github.com/knadh/koanf/v2 v2.3.5 h1:2dXJUYaKGm4SGYeoAtBviq9+02JZo/pxQ2ssOd60rJg=
github.com/knadh/koanf/v2 v2.3.5/go.mod h1:gRb40VRAbd4iJMYYD5IxZ6hfuopFcXBpc9bbQpZwo28=
github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b h1:udzkj9S/zlT5X367kqJis0QP7YMxobob6zhzq6Yre00=
github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b/go.mod h1:pcaDhQK0/NJZEvtCO0qQPPropqV0sJOJ6YW7X+9kRwM=
github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
@@ -310,14 +316,16 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc=
github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw=
github.com/linode/linodego v1.66.0 h1:rK8QJFaV53LWOEJvb/evhTg/dP5ElvtuZmx4iv4RJds=
github.com/linode/linodego v1.66.0/go.mod h1:12ykGs9qsvxE+OU3SXuW2w+DTruWF35FPlXC7gGk2tU=
github.com/mattn/go-colorable v0.1.14 h1:9A9LHSqF/7dyVVX6g0U9cwm9pG3kP9gSzcuIPHPsaIE=
github.com/mattn/go-colorable v0.1.14/go.mod h1:6LmQG8QLFO4G5z1gPvYEzlUgJ2wF+stgPZH1UqBm1s8=
github.com/mattn/go-isatty v0.0.21 h1:xYae+lCNBP7QuW4PUnNG61ffM4hVIfm+zUzDuSzYLGs=
github.com/mattn/go-isatty v0.0.21/go.mod h1:ZXfXG4SQHsB/w3ZeOYbR0PrPwLy+n6xiMrJlRFqopa4=
github.com/mattn/go-runewidth v0.0.23 h1:7ykA0T0jkPpzSvMS5i9uoNn2Xy3R383f9HDx3RybWcw=
github.com/mattn/go-runewidth v0.0.23/go.mod h1:XBkDxAl56ILZc9knddidhrOlY5R/pDhgLpndooCuJAs=
github.com/linode/linodego v1.69.1 h1:f45N2MHR/oece2/ktTTCYmrlfse4//k3NgwcF5zbGZ0=
github.com/linode/linodego v1.69.1/go.mod h1:Fha0NYsQSx5VZK1HQNJY/z/dIxxkFp+vb5veawbmAUw=
github.com/makasim/hyperloglog v0.0.10-reuse-memory h1:tqMXSDlkVujI/aGYUm6uwt4lRUQcne22MOLcJBgLAGc=
github.com/makasim/hyperloglog v0.0.10-reuse-memory/go.mod h1:YjX/dQqCR/7QYX0g8mu8UZAjpIenz1FKM71UEsjFoTo=
github.com/mattn/go-colorable v0.1.15 h1:+u9SLTRGnXv73cEsnsmoZBom+dMU88B2M0aDcWy0/jY=
github.com/mattn/go-colorable v0.1.15/go.mod h1:6LmQG8QLFO4G5z1gPvYEzlUgJ2wF+stgPZH1UqBm1s8=
github.com/mattn/go-isatty v0.0.22 h1:j8l17JJ9i6VGPUFUYoTUKPSgKe/83EYU2zBC7YNKMw4=
github.com/mattn/go-isatty v0.0.22/go.mod h1:ZXfXG4SQHsB/w3ZeOYbR0PrPwLy+n6xiMrJlRFqopa4=
github.com/mattn/go-runewidth v0.0.24 h1:cpokDiIn0MGnhdHwuWnJBITySJ20QyNGnY2kR/ay2DU=
github.com/mattn/go-runewidth v0.0.24/go.mod h1:XBkDxAl56ILZc9knddidhrOlY5R/pDhgLpndooCuJAs=
github.com/miekg/dns v1.1.72 h1:vhmr+TF2A3tuoGNkLDFK9zi36F2LS+hKTRW0Uf8kbzI=
github.com/miekg/dns v1.1.72/go.mod h1:+EuEPhdHOsfk6Wk5TT2CzssZdqkmFhf8r+aVyDEToIs=
github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw=
@@ -330,6 +338,10 @@ github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zx
github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw=
github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0=
github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo=
github.com/moby/moby/api v1.54.2 h1:wiat9QAhnDQjA7wk1kh/TqHz2I1uUA7M7t9SAl/JNXg=
github.com/moby/moby/api v1.54.2/go.mod h1:+RQ6wluLwtYaTd1WnPLykIDPekkuyD/ROWQClE83pzs=
github.com/moby/moby/client v0.4.1 h1:DMQgisVoMkmMs7fp3ROSdiBnoAu8+vo3GggFl06M/wY=
github.com/moby/moby/client v0.4.1/go.mod h1:z52C9O2POPOsnxZAy//WtKcQ32P+jT/NGeXu/7nfjGQ=
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
@@ -342,23 +354,23 @@ github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/oklog/ulid/v2 v2.1.1 h1:suPZ4ARWLOJLegGFiZZ1dFAkqzhMjL3J1TzI+5wHz8s=
github.com/oklog/ulid/v2 v2.1.1/go.mod h1:rcEKHmBBKfef9DhnvX7y1HZBYxjXb0cP5ExxNsTT1QQ=
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.150.0 h1:kRIR4xDIE94IGdHMrYlfMjzlxInL8f9UO9jz+R4iHEc=
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.150.0/go.mod h1:aOBH3hR2zVgVFUbpTSjxuHTCkx+rvg7OWDNnTYUDB3w=
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.150.0 h1:eMU3aV8tUWhTlZwrW/eHbw3JCxsUDXqLKqswqj/vtlQ=
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.150.0/go.mod h1:qYh2PcJsgK0qJ0+vjLYow+E2DAJ7o1MuZ3r0hakgxiE=
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.150.0 h1:Jg2Ets7VwgAZUmnaptx7TJTnjPG7nYZhGOeaveTpC/8=
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.150.0/go.mod h1:SdJHLLrcNFtcPIY4NUTLyx4+0gZZRyOTJfrVLUVxILQ=
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.154.0 h1:WS8HkUa6p8iVJ2v0mmGEK1a9R2b+Uro6tSG+4IfX6rk=
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.154.0/go.mod h1:9QPTx+XgZE7ktvh5jT5TvSisIkh2Fwc7mrfuf6+j2/U=
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.154.0 h1:Kda+8F8o5QATBLP5K2MKmI2t7ddr7sBaV0EhZpjlvB0=
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.154.0/go.mod h1:iVnoGSVXYhnyuQ6TQNhBIHqtu7h0LTXbSyWy584eBjg=
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.154.0 h1:U/MRkEeVwZ3zl8hOlUBP/Q/RMgLfMbTHQoATlLXhI4I=
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.154.0/go.mod h1:dFTV2c6rjph2ZMtkq9xHN5QuYbUSQ+o/25UQfIY3QUQ=
github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U=
github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM=
github.com/opencontainers/image-spec v1.1.1 h1:y0fUlFfIZhPF1W537XOLg0/fcx6zcHCJwooC2xJA040=
github.com/opencontainers/image-spec v1.1.1/go.mod h1:qpqAh3Dmcf36wStyyWU+kCeDgrGnAve2nCC8+7h8Q0M=
github.com/outscale/osc-sdk-go/v2 v2.34.0 h1:hHH5W9Fmgt6b8nGUmDyu4vVP+zqJ+W0zflzjgsGEGUQ=
github.com/outscale/osc-sdk-go/v2 v2.34.0/go.mod h1:6J8WRznaSIEXXVHhhTXisGJQgvE5fYzbf8hAw7YIGfQ=
github.com/ovh/go-ovh v1.9.0 h1:6K8VoL3BYjVV3In9tPJUdT7qMx9h0GExN9EXx1r2kKE=
github.com/ovh/go-ovh v1.9.0/go.mod h1:cTVDnl94z4tl8pP1uZ/8jlVxntjSIf09bNcQ5TJSC7c=
github.com/pborman/getopt v0.0.0-20170112200414-7148bc3a4c30/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o=
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ=
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU=
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 h1:GFCKgmp0tecUJ0sJuv4pzYCqS9+RGSn52M3FUwPs+uo=
github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1:t/avpk3KcrXxUnYOhZhMXJlSEyie6gQbtLq5NM3loB8=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
@@ -366,22 +378,22 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRI
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/prometheus/client_golang v1.23.2 h1:Je96obch5RDVy3FDMndoUsjAhG5Edi49h0RJWRi/o0o=
github.com/prometheus/client_golang v1.23.2/go.mod h1:Tb1a6LWHB3/SPIzCoaDXI4I8UHKeFTEQ1YCr+0Gyqmg=
github.com/prometheus/client_golang/exp v0.0.0-20260411065453-32d05ffab50a h1:HhtTz23wnlOur6WC5/j6+MXPweyJrkarM0sj+sg6OKI=
github.com/prometheus/client_golang/exp v0.0.0-20260411065453-32d05ffab50a/go.mod h1:xA9/W/d/j+zeg2Kp3UgOF+E2rbq7KNVa5PLJPJBE0lw=
github.com/prometheus/client_golang/exp v0.0.0-20260602051030-3537b20ac86b h1:633sracZPrB7O7T6r5skFtwqXDOrXlQkE9Wr5DnYVJE=
github.com/prometheus/client_golang/exp v0.0.0-20260602051030-3537b20ac86b/go.mod h1:7hAEIbflIgnK0HubVroVy6UgJYYKryF6p3mP/dcyay8=
github.com/prometheus/client_model v0.6.2 h1:oBsgwpGs7iVziMvrGhE53c/GrLUsZdHnqNwqPLxwZyk=
github.com/prometheus/client_model v0.6.2/go.mod h1:y3m2F6Gdpfy6Ut/GBsUqTWZqCUvMVzSfMLjcu6wAwpE=
github.com/prometheus/common v0.67.5 h1:pIgK94WWlQt1WLwAC5j2ynLaBRDiinoAb86HZHTUGI4=
github.com/prometheus/common v0.67.5/go.mod h1:SjE/0MzDEEAyrdr5Gqc6G+sXI67maCxzaT3A2+HqjUw=
github.com/prometheus/common v0.68.1 h1:omjRRl4QP4komogpXuhfeOiisQg7xdy8VM1UY+pStaY=
github.com/prometheus/common v0.68.1/go.mod h1:ZzL3f6u94qUxh9p+tJTrF+FvBS1XXbbRAZCQkytAL0Y=
github.com/prometheus/otlptranslator v1.0.0 h1:s0LJW/iN9dkIH+EnhiD3BlkkP5QVIUVEoIwkU+A6qos=
github.com/prometheus/otlptranslator v1.0.0/go.mod h1:vRYWnXvI6aWGpsdY/mOT/cbeVRBlPWtBNDb7kGR3uKM=
github.com/prometheus/procfs v0.20.1 h1:XwbrGOIplXW/AU3YhIhLODXMJYyC1isLFfYCsTEycfc=
github.com/prometheus/procfs v0.20.1/go.mod h1:o9EMBZGRyvDrSPH1RqdxhojkuXstoe4UlK79eF5TGGo=
github.com/prometheus/prometheus v0.311.3 h1:3IrVxQv6v5i/ZCGi6OrYeBhtCwaPTn6Z3DYruXoYm3M=
github.com/prometheus/prometheus v0.311.3/go.mod h1:gjsCxTKtHO1Q8T9333u1s+lUR1OjPyM7ruuGH8RvVyo=
github.com/prometheus/prometheus v0.312.0 h1:f9jdv2fQhQ1fks9a9YwlGZrKr4hih0rRP/rh0mu3Q18=
github.com/prometheus/prometheus v0.312.0/go.mod h1:8oAYd2XPgHXLP4fFKam594R/ZLlPicrrBkVdaWt74Sw=
github.com/prometheus/sigv4 v0.4.1 h1:EIc3j+8NBea9u1iV6O5ZAN8uvPq2xOIUPcqCTivHuXs=
github.com/prometheus/sigv4 v0.4.1/go.mod h1:eu+ZbRvsc5TPiHwqh77OWuCnWK73IdkETYY46P4dXOU=
github.com/puzpuzpuz/xsync/v4 v4.4.0 h1:vlSN6/CkEY0pY8KaB0yqo/pCLZvp9nhdbBdjipT4gWo=
github.com/puzpuzpuz/xsync/v4 v4.4.0/go.mod h1:VJDmTCJMBt8igNxnkQd86r+8KUeN1quSfNKu5bLYFQo=
github.com/puzpuzpuz/xsync/v4 v4.5.0 h1:vOSWu6b57/emh+L/Cw0BeQfvxa/cogFywXHeGUxQxAg=
github.com/puzpuzpuz/xsync/v4 v4.5.0/go.mod h1:VJDmTCJMBt8igNxnkQd86r+8KUeN1quSfNKu5bLYFQo=
github.com/rogpeppe/go-internal v1.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ=
github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc=
github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk=
@@ -390,10 +402,10 @@ github.com/scaleway/scaleway-sdk-go v1.0.0-beta.36 h1:ObX9hZmK+VmijreZO/8x9pQ8/P
github.com/scaleway/scaleway-sdk-go v1.0.0-beta.36/go.mod h1:LEsDu4BubxK7/cWhtlQWfuxwL4rf/2UEpxXz1o1EMtM=
github.com/spf13/pflag v1.0.10 h1:4EBh2KAYBwaONj6b2Ye1GiHfwjqyROoF4RwYO+vPwFk=
github.com/spf13/pflag v1.0.10/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
github.com/spiffe/go-spiffe/v2 v2.6.0 h1:l+DolpxNWYgruGQVV0xsfeya3CsC7m8iBzDnMpsbLuo=
github.com/spiffe/go-spiffe/v2 v2.6.0/go.mod h1:gm2SeUoMZEtpnzPNs2Csc0D/gX33k1xIx7lEzqblHEs=
github.com/stackitcloud/stackit-sdk-go/core v0.23.0 h1:zPrOhf3Xe47rKRs1fg/AqKYUiJJRYjdcv+3qsS50mEs=
github.com/stackitcloud/stackit-sdk-go/core v0.23.0/go.mod h1:osMglDby4csGZ5sIfhNyYq1bS1TxIdPY88+skE/kkmI=
github.com/spiffe/go-spiffe/v2 v2.7.0 h1:uXe1MflJoHw58wAUvxVlcM7WpKtijWG7I1UidcGh6g4=
github.com/spiffe/go-spiffe/v2 v2.7.0/go.mod h1:47Q0Q9/AqGha8QLHp+kxpH4Wca7X7EnOtlIJy3mxZ3U=
github.com/stackitcloud/stackit-sdk-go/core v0.26.0 h1:jQEb9gkehfp6VCP6TcYk7BI10cz4l0KM2L6hqYBH2QA=
github.com/stackitcloud/stackit-sdk-go/core v0.26.0/go.mod h1:WU1hhxnjXw2EV7CYa1nlEvNpMiRY6CvmIOaHuL3pOaA=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
@@ -422,8 +434,8 @@ github.com/valyala/histogram v1.2.0 h1:wyYGAZZt3CpwUiIb9AU/Zbllg1llXyrtApRS815OL
github.com/valyala/histogram v1.2.0/go.mod h1:Hb4kBwb4UxsaNbbbh+RRz8ZR6pdodR57tzWUS3BUzXY=
github.com/valyala/quicktemplate v1.8.0 h1:zU0tjbIqTRgKQzFY1L42zq0qR3eh4WoQQdIdqCysW5k=
github.com/valyala/quicktemplate v1.8.0/go.mod h1:qIqW8/igXt8fdrUln5kOSb+KWMaJ4Y8QUsfd1k6L2jM=
github.com/vultr/govultr/v3 v3.28.1 h1:KR3LhppYARlBujY7+dcrE7YKL0Yo9qXL+msxykKQrLI=
github.com/vultr/govultr/v3 v3.28.1/go.mod h1:2zyUw9yADQaGwKnwDesmIOlBNLrm7edsCfWHFJpWKf8=
github.com/vultr/govultr/v3 v3.31.2 h1:2l3/KDvfemG+4azw4LLquJoh9mFOAVEdBXtPPzix3ac=
github.com/vultr/govultr/v3 v3.31.2/go.mod h1:2zyUw9yADQaGwKnwDesmIOlBNLrm7edsCfWHFJpWKf8=
github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM=
github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg=
github.com/xrash/smetrics v0.0.0-20250705151800-55b8f293f342 h1:FnBeRrxr7OU4VvAzt5X7s6266i6cSVkkFPS0TuXWbIg=
@@ -432,62 +444,64 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de
github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
go.opentelemetry.io/auto/sdk v1.2.1 h1:jXsnJ4Lmnqd11kwkBV2LgLoFMZKizbCi5fNZ/ipaZ64=
go.opentelemetry.io/auto/sdk v1.2.1/go.mod h1:KRTj+aOaElaLi+wW1kO/DZRXwkF4C5xPbEe3ZiIhN7Y=
go.opentelemetry.io/collector/component v1.56.0 h1:fOCs36Dxg95w2RQCVI2i5IsHc5IbZ99vmbipK9FM7pQ=
go.opentelemetry.io/collector/component v1.56.0/go.mod h1:MkAjcSc2T0BiYf/uARZdTlfnxBB9BwmvY6v08D+qeY4=
go.opentelemetry.io/collector/component/componentstatus v0.150.0 h1:Jy/9quAWwDN9BqMEzZn0BEzVxxWARa1b/wvBQW6yPxs=
go.opentelemetry.io/collector/component/componentstatus v0.150.0/go.mod h1:FFmyHgPqwtvkezi9Z9NYgXxY0m3N0oUMMd/HIAEq8vY=
go.opentelemetry.io/collector/component/componenttest v0.150.0 h1:pT7avT/Pfn8tAOOlmFWgtOaGvXY0nxSwrivnhOl/LH0=
go.opentelemetry.io/collector/component/componenttest v0.150.0/go.mod h1:D+7mfbcZ/TfneQRZNtVwH+/YKQdalc1joa9NhH1BGPk=
go.opentelemetry.io/collector/confmap v1.56.0 h1:YjLll5L77Z3up94t/pdOMaH35kwd28EtjBORewfIjmA=
go.opentelemetry.io/collector/confmap v1.56.0/go.mod h1:iprN8aL/euBXig6bpLZSZqi+8CZIgE9/Pm6y3qb1QWY=
go.opentelemetry.io/collector/confmap/xconfmap v0.150.0 h1:PR+c4/Ly4Plx862jJ1Cg+HFewMrHsWaN9eKxrYBhtK4=
go.opentelemetry.io/collector/confmap/xconfmap v0.150.0/go.mod h1:WDLyne6Zmoi5OZ46Hfg4z/5KhsBG1mFuYjoK20VcDcA=
go.opentelemetry.io/collector/consumer v1.56.0 h1:olhuaTI3cic6VfcraXt3qqsv1v4Qxf55gHxOO1uIVXw=
go.opentelemetry.io/collector/consumer v1.56.0/go.mod h1:FpnfeTLQAdcOtzrkQ36Z+E5aconIymkv9xpJuAdLvy0=
go.opentelemetry.io/collector/consumer/consumertest v0.150.0 h1:DQtVy0BUTQqHKKOyM0hYnxV8H2kKHjayc8aMMa2fow0=
go.opentelemetry.io/collector/consumer/consumertest v0.150.0/go.mod h1:2mgIllFOgoq+SQ7QfXzaZn65pa6OZWobcy3yj+Ik9Ug=
go.opentelemetry.io/collector/consumer/xconsumer v0.150.0 h1:URO73bAV00wTH9bJeloqaiLgS3Q80GNci+nm1iZ3W6Q=
go.opentelemetry.io/collector/consumer/xconsumer v0.150.0/go.mod h1:BMcOInfcRUpVZ2R4qa3vNglvU6mWL+0dhAayH87YSB8=
go.opentelemetry.io/collector/featuregate v1.56.0 h1:NjcbOZkdCSXddAJmFLdO+pv1gmAgrU6sC5PBga2KlKI=
go.opentelemetry.io/collector/featuregate v1.56.0/go.mod h1:4ga1QBMPEejXXmpyJS8lmaRpknJ3Lb9Bvk6e420bUFU=
go.opentelemetry.io/collector/internal/componentalias v0.150.0 h1:qvcJr0m/fFgsc3x6Oya3RNDOZp/WyfmOKIv9jtvoLYw=
go.opentelemetry.io/collector/internal/componentalias v0.150.0/go.mod h1:abuQP8ELgPpCSq6xbHM1b2hPOGqaKxUeLgHHdU/XGP0=
go.opentelemetry.io/collector/internal/testutil v0.150.0 h1:J4PLQGPfbLVaL5eI1aMc0m0TMixV9wzBhNhoHU00J0I=
go.opentelemetry.io/collector/internal/testutil v0.150.0/go.mod h1:Jkjs6rkqs973LqgZ0Fe3zrokQRKULYXPIf4HuqStiEE=
go.opentelemetry.io/collector/pdata v1.56.0 h1:W+QAfN2Iz8SNss1T5JNzRWFnw+7oP1vXBQH9ZuOJkXY=
go.opentelemetry.io/collector/pdata v1.56.0/go.mod h1:usR9utboXufbD1rp1oJy+3smQXXpZ+CsI3WN7QsiOs0=
go.opentelemetry.io/collector/pdata/pprofile v0.150.0 h1:Ae+FxmYXDdcqeLqIAdNSO3YGxco7RS2mIMTdjvavfso=
go.opentelemetry.io/collector/pdata/pprofile v0.150.0/go.mod h1:tEBeGysY/LpIh39NLoQQl3qmUBOF9wyH5p/fmn7smzM=
go.opentelemetry.io/collector/pdata/testdata v0.150.0 h1:nZE3UNuDYd9lfXTk/n5UplPwXBD4tptDIZH5PvWhHKQ=
go.opentelemetry.io/collector/pdata/testdata v0.150.0/go.mod h1:RPOOH2KNevfhu7adoEXVTNtPPZsHwbrSOQKeFZE/220=
go.opentelemetry.io/collector/pipeline v1.56.0 h1:KfyCes/EPC2hpBhU28z9WnJzSRlBYS5FfMHOYAXHbXw=
go.opentelemetry.io/collector/pipeline v1.56.0/go.mod h1:RD90NG3Jbk965Xaqym3JyHkuol4uZJjQVUkD9ddXJIs=
go.opentelemetry.io/collector/processor v1.56.0 h1:5UGXZorhoWg0gnhvDZWYIOUp2dEWfWH28vE+/wV3cMA=
go.opentelemetry.io/collector/processor v1.56.0/go.mod h1:ptpRRg6r9YdXGSKdQcsM6ePH6ZkkkMnlNY8OxIC4Q7c=
go.opentelemetry.io/collector/processor/processortest v0.150.0 h1:M3p/ZcAAKnh/3aZgTgXngWU+9C0Yx/sQLgZNPBP/ZSo=
go.opentelemetry.io/collector/processor/processortest v0.150.0/go.mod h1:szMO3iQ+CDQLCq6y8+deTUc6FbLKi9zNqwtm5+umZk8=
go.opentelemetry.io/collector/processor/xprocessor v0.150.0 h1:tyIM+WT6NRRkynTGtPwPYDRvWlv3YVt9PDteLscW2mU=
go.opentelemetry.io/collector/processor/xprocessor v0.150.0/go.mod h1:m7uoLC+a9G3EOr9v2+fItUMV0EOFrL+RUkPAspBLK1w=
go.opentelemetry.io/contrib/detectors/gcp v1.43.0 h1:62yY3dT7/ShwOxzA0RsKRgshBmfElKI4d/Myu2OxDFU=
go.opentelemetry.io/contrib/detectors/gcp v1.43.0/go.mod h1:RyaZMFY7yi1kAs45S6mbFGz8O8rqB0dTY14uzvG4LCs=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.68.0 h1:0Qx7VGBacMm9ZENQ7TnNObTYI4ShC+lHI16seduaxZo=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.68.0/go.mod h1:Sje3i3MjSPKTSPvVWCaL8ugBzJwik3u4smCjUeuupqg=
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.68.0 h1:cuXaPAfIoJKsYjBjPSb2nKZEmgM43zVr25l37IxhKME=
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.68.0/go.mod h1:BuzhPofpCzlDi/Q/Xjg54M4/3oWqqyDe2Zeq7A2I0QE=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.68.0 h1:CqXxU8VOmDefoh0+ztfGaymYbhdB/tT3zs79QaZTNGY=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.68.0/go.mod h1:BuhAPThV8PBHBvg8ZzZ/Ok3idOdhWIodywz2xEcRbJo=
go.opentelemetry.io/otel v1.43.0 h1:mYIM03dnh5zfN7HautFE4ieIig9amkNANT+xcVxAj9I=
go.opentelemetry.io/otel v1.43.0/go.mod h1:JuG+u74mvjvcm8vj8pI5XiHy1zDeoCS2LB1spIq7Ay0=
go.opentelemetry.io/collector/component v1.60.0 h1:LpIjHMn7OOjUsFR84ROc2kqPbP1xnKyDCGi7ZVqEaKU=
go.opentelemetry.io/collector/component v1.60.0/go.mod h1:Rag+NNgiGIkcGYlcTfJtMh2l0T5XS1KNv9Wjw9yofAk=
go.opentelemetry.io/collector/component/componentstatus v0.154.0 h1:4ifSCy2Y332iZ5AldHt9ujVjY6XKxhVe/hND4TSDarg=
go.opentelemetry.io/collector/component/componentstatus v0.154.0/go.mod h1:ZsBIax7tvvODn0XqTyhTfKZjm96zVKnLUKvlN8SHFjo=
go.opentelemetry.io/collector/component/componenttest v0.154.0 h1:uH06tUatG4S45A/f3sFENMMAMzWURmgxKK3MAbVZAUI=
go.opentelemetry.io/collector/component/componenttest v0.154.0/go.mod h1:SQ1JRosjFAZ7kN2yNHNcNakOliqrP0QxglKcYyUrUpQ=
go.opentelemetry.io/collector/confmap v1.60.0 h1:TEBi/N3kac/JI4VTEq9LjqRCFdF2JS2MHOCEiHq8GSM=
go.opentelemetry.io/collector/confmap v1.60.0/go.mod h1:Z693ETewV4n8JsOO2jp/iLe1PGGpFCIzuNsF1xLeiSY=
go.opentelemetry.io/collector/confmap/xconfmap v0.154.0 h1:tarvY9S02jkYNYW/4+yD02RRatwJAojMD430Bs4JD/4=
go.opentelemetry.io/collector/confmap/xconfmap v0.154.0/go.mod h1:zcVRrY1gS8qVwBrTrhzVI67tMAUu5BONTsIXzjXu1Ho=
go.opentelemetry.io/collector/consumer v1.60.0 h1:SWP/0HvDnWiiy/4S366CiatAZ4gFl410UmggrZEcWVg=
go.opentelemetry.io/collector/consumer v1.60.0/go.mod h1:nkp1NBtKQzme7WFF7fkgRgDlQLs49VIMOn8rO0jfmYU=
go.opentelemetry.io/collector/consumer/consumertest v0.154.0 h1:G9gFP86ZsglC3mTLA6cqOrW5lvdcEBJrVgHtThE+Sc4=
go.opentelemetry.io/collector/consumer/consumertest v0.154.0/go.mod h1:FRLGgy8gFYjm3A+yby1bctz5ZIAn6EUOpuV49KnKbFY=
go.opentelemetry.io/collector/consumer/xconsumer v0.154.0 h1:I3rB+S5ORE1XLzqopFXvP6UmYrsj5n1tFlcEAPg96Zw=
go.opentelemetry.io/collector/consumer/xconsumer v0.154.0/go.mod h1:WNT9BoyLE/nE5N6WEL4c1GXcfGcRUmSTCSr6e/tyfO4=
go.opentelemetry.io/collector/featuregate v1.60.0 h1:/HxHB8hq4N5Fhq5N0C8G6xbXTHxnGcWIryyJzmP7pdc=
go.opentelemetry.io/collector/featuregate v1.60.0/go.mod h1:4ga1QBMPEejXXmpyJS8lmaRpknJ3Lb9Bvk6e420bUFU=
go.opentelemetry.io/collector/internal/componentalias v0.154.0 h1:g0y8F/qez9cbsgF5+/uU6YC6l5oXVkccIhsXVHmF3xQ=
go.opentelemetry.io/collector/internal/componentalias v0.154.0/go.mod h1:F2tudJ/Zcm8w8b768sU65nZc4q2rgY1MhfX5FxDeUgA=
go.opentelemetry.io/collector/internal/testutil v0.154.0 h1:iUYHOM8+wONW01A4jFnzauanOYGVBGchKWWtm51is6c=
go.opentelemetry.io/collector/internal/testutil v0.154.0/go.mod h1:Jkjs6rkqs973LqgZ0Fe3zrokQRKULYXPIf4HuqStiEE=
go.opentelemetry.io/collector/pdata v1.60.0 h1:YcGMHzeJucHen41AoR4mxHro8reUr9SVqt7P0KacKzQ=
go.opentelemetry.io/collector/pdata v1.60.0/go.mod h1:Ca8VgZX2wOr6wW4nihPWaCpkJVvzeo6Txa7BJ7/WO90=
go.opentelemetry.io/collector/pdata/pprofile v0.154.0 h1:dWrHnKBzzMhkZXfKmSuFpGVAApSUcrQ+mBFzAsO6/8s=
go.opentelemetry.io/collector/pdata/pprofile v0.154.0/go.mod h1:BE9oOmAEHVqE+yHRe5Z3qz7co+2SU249DIxVGPRsYf8=
go.opentelemetry.io/collector/pdata/testdata v0.154.0 h1:PSc3gogHpJoVHenvMhcxkOPTnEKpaykURxtSNyVXYK4=
go.opentelemetry.io/collector/pdata/testdata v0.154.0/go.mod h1:zIT+sag/xmSM6VAMhv2tnEzlQF9n266OcQm4V6roWdU=
go.opentelemetry.io/collector/pipeline v1.60.0 h1:ZLk/8K/Xzz+JRBWLmqLlVMwEWVnQvmly6nWeKs+lh6s=
go.opentelemetry.io/collector/pipeline v1.60.0/go.mod h1:RD90NG3Jbk965Xaqym3JyHkuol4uZJjQVUkD9ddXJIs=
go.opentelemetry.io/collector/processor v1.60.0 h1:B3YgiKa+4tMuJ6v4bSaKUtTCwNRzugbEDei8j7jiPpI=
go.opentelemetry.io/collector/processor v1.60.0/go.mod h1:ZRNUW8FHZ+0CW+HoIG0/h+fQq8aYjMz9ccy2w2jguag=
go.opentelemetry.io/collector/processor/processortest v0.154.0 h1:2Lu7JGqH3fzg9BE0rmzBwCQB7oRWzM8fs+X5SSZO/4M=
go.opentelemetry.io/collector/processor/processortest v0.154.0/go.mod h1:E813PIbkBcwgoDnZ9cjuw70MUNmqxAHIvmDC8gOZiP8=
go.opentelemetry.io/collector/processor/xprocessor v0.154.0 h1:ert+SRk5DPSqIxqpOEnywrwVLYSvqEvXwy60F94VtFE=
go.opentelemetry.io/collector/processor/xprocessor v0.154.0/go.mod h1:93XyfiqPYokF1i8NQvWsKggt5Si5qZvOcZ2P0l+uxII=
go.opentelemetry.io/contrib/detectors/gcp v1.44.0 h1:NmLfL734pJhM0JKaYd2Y28+nY9dPRWYAAbxhRCrKXPw=
go.opentelemetry.io/contrib/detectors/gcp v1.44.0/go.mod h1:tNAsgd8avTGke1+MndXlU5Cru4PQ9Ai/cCNWQv/ZJ/s=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.69.0 h1:2yEATaop1/a1I4psnSLgWVPLWwCzkqWakgJy7xTDVy0=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.69.0/go.mod h1:D7J12YRapIekYyPWgGPlA/23pRmpSEZC5xJC/TTLI9U=
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.69.0 h1:MCcYL7J6Vt/X0kjqbMZkekCmwsurbQRbL69vkiye2lk=
go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.69.0/go.mod h1:3jnStNwSufK+f5ktjL4EPcwtig4rtd81NS70lqHuXl8=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.69.0 h1:8tvICD4vSTOOsNrsI4Ljf6C+6UKvpTEH5XY3JMoyPoo=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.69.0/go.mod h1:z9+yiacE0IHRqM4qFfkbt/JYlmYXgss8GY/jXoNuPJI=
go.opentelemetry.io/otel v1.44.0 h1:JjwHmHpA4iZ3wBxluu2fbbE7j4kqlE8jXyAyPXH7HqU=
go.opentelemetry.io/otel v1.44.0/go.mod h1:BMgjTHL9WPRlRjL2oZCBTL4whCGtXch2H4BhOPIAyYc=
go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.43.0 h1:TC+BewnDpeiAmcscXbGMfxkO+mwYUwE/VySwvw88PfA=
go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.43.0/go.mod h1:J/ZyF4vfPwsSr9xJSPyQ4LqtcTPULFR64KwTikGLe+A=
go.opentelemetry.io/otel/metric v1.43.0 h1:d7638QeInOnuwOONPp4JAOGfbCEpYb+K6DVWvdxGzgM=
go.opentelemetry.io/otel/metric v1.43.0/go.mod h1:RDnPtIxvqlgO8GRW18W6Z/4P462ldprJtfxHxyKd2PY=
go.opentelemetry.io/otel/sdk v1.43.0 h1:pi5mE86i5rTeLXqoF/hhiBtUNcrAGHLKQdhg4h4V9Dg=
go.opentelemetry.io/otel/sdk v1.43.0/go.mod h1:P+IkVU3iWukmiit/Yf9AWvpyRDlUeBaRg6Y+C58QHzg=
go.opentelemetry.io/otel/sdk/metric v1.43.0 h1:S88dyqXjJkuBNLeMcVPRFXpRw2fuwdvfCGLEo89fDkw=
go.opentelemetry.io/otel/sdk/metric v1.43.0/go.mod h1:C/RJtwSEJ5hzTiUz5pXF1kILHStzb9zFlIEe85bhj6A=
go.opentelemetry.io/otel/trace v1.43.0 h1:BkNrHpup+4k4w+ZZ86CZoHHEkohws8AY+WTX09nk+3A=
go.opentelemetry.io/otel/trace v1.43.0/go.mod h1:/QJhyVBUUswCphDVxq+8mld+AvhXZLhe+8WVFxiFff0=
go.opentelemetry.io/otel/metric v1.44.0 h1:1w0gILTcHdr3YI+ixLyjemwrVnsMURbTZFrSYCdDdmc=
go.opentelemetry.io/otel/metric v1.44.0/go.mod h1:8O7hanEPBNgEMmybD3s2VBKcgWOCsA6tzHBPODAiquo=
go.opentelemetry.io/otel/metric/x v0.66.0 h1:YkCrx1zLOChi9ZcZ6euupOcsgzbVlec7D/xoEU1+cTA=
go.opentelemetry.io/otel/metric/x v0.66.0/go.mod h1:d1+BDj9t96do0/1LoU1ayfCv79ZgNE41qbhBvnMOBZk=
go.opentelemetry.io/otel/sdk v1.44.0 h1:nHYwb9lK+fJPU/dnT6s7W7Z8itMWyqrnVfbheVYrZ58=
go.opentelemetry.io/otel/sdk v1.44.0/go.mod h1:Osuydd3Se74nqjAKxid74N5eC+jfEqfTegHRnq58oK0=
go.opentelemetry.io/otel/sdk/metric v1.44.0 h1:3LlKgI+VjbVsjNRFZJZAJ30WjXC5VkNRks6si09iEfI=
go.opentelemetry.io/otel/sdk/metric v1.44.0/go.mod h1:5B5pMARnXxKhltooO4xUuCBorl65a4EpnTalObqOigA=
go.opentelemetry.io/otel/trace v1.44.0 h1:jxF5CsGYCe74MCRx2X4g7WsY/VBKRqqpNvXlX/6gtIk=
go.opentelemetry.io/otel/trace v1.44.0/go.mod h1:oLl1jrMQAVo6v3GAggN+1VH9VIz9iUSvW53sW1Q8PIE=
go.opentelemetry.io/proto/slim/otlp v1.10.0 h1:iR97Vs/ZDR+y9TfuP9b1XBtdPWeC+OMslIBmhcLU7jM=
go.opentelemetry.io/proto/slim/otlp v1.10.0/go.mod h1:lV9250stpjYLPNA5viFabIgP2QlUGRT1GdTgAf8SIUk=
go.opentelemetry.io/proto/slim/otlp/collector/profiles/v1development v0.3.0 h1:RUF5rO0hAlgiJt1fzQVzcVs3vZVNHIcMLgOgG4rWNcQ=
@@ -500,8 +514,8 @@ go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto=
go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE=
go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0=
go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y=
go.uber.org/zap v1.27.1 h1:08RqriUEv8+ArZRYSTXy1LeBScaMpVSTBhCeaZYfMYc=
go.uber.org/zap v1.27.1/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E=
go.uber.org/zap v1.28.0 h1:IZzaP1Fv73/T/pBMLk4VutPl36uNC+OSUh3JLG3FIjo=
go.uber.org/zap v1.28.0/go.mod h1:rDLpOi171uODNm/mxFcuYWxDsqWSAVkFdX4XojSKg/Q=
go.yaml.in/yaml/v2 v2.4.4 h1:tuyd0P+2Ont/d6e2rl3be67goVK4R6deVxCUX5vyPaQ=
go.yaml.in/yaml/v2 v2.4.4/go.mod h1:gMZqIpDtDqOfM0uNfy0SkpRhvUryYH0Z6wdMYcacYXQ=
go.yaml.in/yaml/v3 v3.0.4 h1:tfq32ie2Jv2UxXFdLJdh3jXuOzWiL1fo0bu/FbuKpbc=
@@ -509,65 +523,65 @@ go.yaml.in/yaml/v3 v3.0.4/go.mod h1:DhzuOOF2ATzADvBadXxruRBLzYTpT36CKvDb3+aBEFg=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.52.0 h1:RMs7fP2rXdep0CftQlK8Uf+kibLm7qkCcradZWYz988=
golang.org/x/crypto v0.52.0/go.mod h1:1QgfPxDqh0T2M/elOJtp9RvuR95kVjir0e6/BvEmGbc=
golang.org/x/exp v0.0.0-20260410095643-746e56fc9e2f h1:W3F4c+6OLc6H2lb//N1q4WpJkhzJCK5J6kUi1NTVXfM=
golang.org/x/exp v0.0.0-20260410095643-746e56fc9e2f/go.mod h1:J1xhfL/vlindoeF/aINzNzt2Bket5bjo9sdOYzOsU80=
golang.org/x/crypto v0.53.0 h1:QZ4Muo8THX6CizN2vPPd5fBGHyogrdK9fG4wLPFUsto=
golang.org/x/crypto v0.53.0/go.mod h1:DNLU434OwVakk9PzuwV8w62mAJpRJL3vsgcfp4Qnsio=
golang.org/x/exp v0.0.0-20260611194520-c48552f49976 h1:X8Hz2ImujgbmetVuW+w2YkyZChE3cBpZi2P158rTG9M=
golang.org/x/exp v0.0.0-20260611194520-c48552f49976/go.mod h1:vnf4pv9iKZXY58sQE1L86zmNWJ4159e1RkcWiLCkeEY=
golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.35.0 h1:Ww1D637e6Pg+Zb2KrWfHQUnH2dQRLBQyAtpr/haaJeM=
golang.org/x/mod v0.35.0/go.mod h1:+GwiRhIInF8wPm+4AoT6L0FA1QWAad3OMdTRx4tFYlU=
golang.org/x/mod v0.37.0 h1:vF1DjpVEshcIqoEaauuHebaLk1O1forxjxBaVn884JQ=
golang.org/x/mod v0.37.0/go.mod h1:m8S8VeM9r4dzDwjrKO0a1sZP3YjeMamRRlD+fmR2Q/0=
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.55.0 h1:bcvxaJn3e1U6InsFWt1JUq1aSjnRxLzT2rtD2KfkDF8=
golang.org/x/net v0.55.0/go.mod h1:L5U2KuzuOe1lY7Z+aWVIKK6qEeJXnXV9yzGA+WCHJww=
golang.org/x/net v0.56.0 h1:Rw8j/hFzGvJUZwNBXnAtf5sVDVt+65SK2C7IxCxZt5o=
golang.org/x/net v0.56.0/go.mod h1:D3Ku6r+V6JROoZK144D2XfMHFcMq/0zSfLelVTCFKec=
golang.org/x/oauth2 v0.36.0 h1:peZ/1z27fi9hUOFCAZaHyrpWG5lwe0RJEEEeH0ThlIs=
golang.org/x/oauth2 v0.36.0/go.mod h1:YDBUJMTkDnJS+A4BP4eZBjCqtokkg1hODuPjwiGPO7Q=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.20.0 h1:e0PTpb7pjO8GAtTs2dQ6jYa5BWYlMuX047Dco/pItO4=
golang.org/x/sync v0.20.0/go.mod h1:9xrNwdLfx4jkKbNva9FpL6vEN7evnE43NNNJQ2LF3+0=
golang.org/x/sync v0.21.0 h1:HLII4xRRTtCRkxYp4HNFF0Js/Og6q2i++KXbg0gHCwM=
golang.org/x/sync v0.21.0/go.mod h1:9xrNwdLfx4jkKbNva9FpL6vEN7evnE43NNNJQ2LF3+0=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.45.0 h1:dO4czNzziLiiXplLQgBCEpCvXQ3dnkn0SdaZSYdQ+FY=
golang.org/x/sys v0.45.0/go.mod h1:4GL1E5IUh+htKOUEOaiffhrAeqysfVGipDYzABqnCmw=
golang.org/x/term v0.43.0 h1:S4RLU2sB31O/NCl+zFN9Aru9A/Cq2aqKpTZJ6B+DwT4=
golang.org/x/term v0.43.0/go.mod h1:lrhlHNdQJHO+1qVYiHfFKVuVioJIheAc3fBSMFYEIsk=
golang.org/x/sys v0.46.0 h1:noSf2Fq6F8DBgS+LysIkx7rIExoNHJsxOAtPp4rthXw=
golang.org/x/sys v0.46.0/go.mod h1:4GL1E5IUh+htKOUEOaiffhrAeqysfVGipDYzABqnCmw=
golang.org/x/term v0.44.0 h1:0rLvDRCtNj0gZkyIXhCyOb2OAzEhLVqc4B+hrsBhrmc=
golang.org/x/term v0.44.0/go.mod h1:7ze4MdzUzLXpSAoFP1H0bOI9aXDqveSvatT5vKcFh2Y=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.37.0 h1:Cqjiwd9eSg8e0QAkyCaQTNHFIIzWtidPahFWR83rTrc=
golang.org/x/text v0.37.0/go.mod h1:a5sjxXGs9hsn/AJVwuElvCAo9v8QYLzvavO5z2PiM38=
golang.org/x/text v0.38.0 h1:sXmwo9DwP3OK9EZ7PqAdaooSGozfl/3a6/xJcbzPRhE=
golang.org/x/text v0.38.0/go.mod h1:YXZt3QhHUKYT53r2lLKFIVi6Ao1jdzrTR/KQ09qyxF4=
golang.org/x/time v0.15.0 h1:bbrp8t3bGUeFOx08pvsMYRTCVSMk89u4tKbNOZbp88U=
golang.org/x/time v0.15.0/go.mod h1:Y4YMaQmXwGQZoFaVFk4YpCt4FLQMYKZe9oeV/f4MSno=
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.44.0 h1:UP4ajHPIcuMjT1GqzDWRlalUEoY+uzoZKnhOjbIPD2c=
golang.org/x/tools v0.44.0/go.mod h1:KA0AfVErSdxRZIsOVipbv3rQhVXTnlU6UhKxHd1seDI=
golang.org/x/tools v0.46.0 h1:7jTurBkPZu4moS/Uy4OQT1M+QBlsj3wejyZwsT8Z7rk=
golang.org/x/tools v0.46.0/go.mod h1:FrD85F8l+NWL+9XWBSyVSHO6Ne4jutsfIFba7AWQ5Ys=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
gonum.org/v1/gonum v0.17.0 h1:VbpOemQlsSMrYmn7T2OUvQ4dqxQXU+ouZFQsZOx50z4=
gonum.org/v1/gonum v0.17.0/go.mod h1:El3tOrEuMpv2UdMrbNlKEh9vd86bmQ6vqIcDwxEOc1E=
google.golang.org/api v0.276.0 h1:nVArUtfLEihtW+b0DdcqRGK1xoEm2+ltAihyztq7MKY=
google.golang.org/api v0.276.0/go.mod h1:Fnag/EWUPIcJXuIkP1pjoTgS5vdxlk3eeemL7Do6bvw=
google.golang.org/genproto v0.0.0-20260414002931-afd174a4e478 h1:aLsVTW0lZ8+IY5u/ERjZSCvAmhuR7slKzyha3YikDNA=
google.golang.org/genproto v0.0.0-20260414002931-afd174a4e478/go.mod h1:YJAzKjfHIUHb9T+bfu8L7mthAp7VVXQBUs1PLdBWS7M=
google.golang.org/genproto/googleapis/api v0.0.0-20260414002931-afd174a4e478 h1:yQugLulqltosq0B/f8l4w9VryjV+N/5gcW0jQ3N8Qec=
google.golang.org/genproto/googleapis/api v0.0.0-20260414002931-afd174a4e478/go.mod h1:C6ADNqOxbgdUUeRTU+LCHDPB9ttAMCTff6auwCVa4uc=
google.golang.org/genproto/googleapis/rpc v0.0.0-20260414002931-afd174a4e478 h1:RmoJA1ujG+/lRGNfUnOMfhCy5EipVMyvUE+KNbPbTlw=
google.golang.org/genproto/googleapis/rpc v0.0.0-20260414002931-afd174a4e478/go.mod h1:4Hqkh8ycfw05ld/3BWL7rJOSfebL2Q+DVDeRgYgxUU8=
google.golang.org/grpc v1.80.0 h1:Xr6m2WmWZLETvUNvIUmeD5OAagMw3FiKmMlTdViWsHM=
google.golang.org/grpc v1.80.0/go.mod h1:ho/dLnxwi3EDJA4Zghp7k2Ec1+c2jqup0bFkw07bwF4=
google.golang.org/protobuf v1.36.11 h1:fV6ZwhNocDyBLK0dj+fg8ektcVegBBuEolpbTQyBNVE=
google.golang.org/protobuf v1.36.11/go.mod h1:HTf+CrKn2C3g5S8VImy6tdcUvCska2kB7j23XfzDpco=
google.golang.org/api v0.284.0 h1:i+cKTgeQRcRySkP7QTl5PDO7/pAm8EcMFIUMlNbk4Vc=
google.golang.org/api v0.284.0/go.mod h1:AU44fU+XVZOCcd8uLaBIa/ZgzgPf/0qqY3+m7lQaado=
google.golang.org/genproto v0.0.0-20260615183401-62b3387ff324 h1:r7/+bt4yKglJiN8eUY8enbRjglCvFm1eh8ezYdYoKTM=
google.golang.org/genproto v0.0.0-20260615183401-62b3387ff324/go.mod h1:V5M1lxGXNUICs0aOqAMsK6HtmLnCyuzY031uOQS9rJE=
google.golang.org/genproto/googleapis/api v0.0.0-20260615183401-62b3387ff324 h1:g0RAkxK/smSu/iRwC/KIX1mwUoVJtk2OjbgaeS4DmUM=
google.golang.org/genproto/googleapis/api v0.0.0-20260615183401-62b3387ff324/go.mod h1:Z4WJ5pJOYWFWcHEQUelD5QaZDknIQkpIL/+fyJOT9+A=
google.golang.org/genproto/googleapis/rpc v0.0.0-20260615183401-62b3387ff324 h1:9HZDLIdYBJXAnaFOr9WHrKVycfpY+75s9HGadC0305A=
google.golang.org/genproto/googleapis/rpc v0.0.0-20260615183401-62b3387ff324/go.mod h1:4Hqkh8ycfw05ld/3BWL7rJOSfebL2Q+DVDeRgYgxUU8=
google.golang.org/grpc v1.81.1 h1:VnnIIZ88UzOOKLukQi+ImGz8O1Wdp8nAGGnvOfEIWQQ=
google.golang.org/grpc v1.81.1/go.mod h1:xGH9GfzOyMTGIOXBJmXt+BX/V0kcdQbdcuwQ/zNw42I=
google.golang.org/protobuf v1.36.12-0.20260120151049-f2248ac996af h1:+5/Sw3GsDNlEmu7TfklWKPdQ0Ykja5VEmq2i817+jbI=
google.golang.org/protobuf v1.36.12-0.20260120151049-f2248ac996af/go.mod h1:HTf+CrKn2C3g5S8VImy6tdcUvCska2kB7j23XfzDpco=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
@@ -575,30 +589,30 @@ gopkg.in/evanphx/json-patch.v4 v4.13.0 h1:czT3CmqEaQ1aanPc5SdlgQrrEIb8w/wwCvWWnf
gopkg.in/evanphx/json-patch.v4 v4.13.0/go.mod h1:p8EYWUEYMpynmqDbY58zCKCFZw8pRWMG4EsWvDvM72M=
gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc=
gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
gopkg.in/ini.v1 v1.67.1 h1:tVBILHy0R6e4wkYOn3XmiITt/hEVH4TFMYvAX2Ytz6k=
gopkg.in/ini.v1 v1.67.1/go.mod h1:x/cyOwCgZqOkJoDIJ3c1KNHMo10+nLGAhh+kn3Zizss=
gopkg.in/ini.v1 v1.67.2 h1:JtOSMb9OuaCZKr7h5D/h6iii14sK0hLbplTc6frx4Ss=
gopkg.in/ini.v1 v1.67.2/go.mod h1:x/cyOwCgZqOkJoDIJ3c1KNHMo10+nLGAhh+kn3Zizss=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
k8s.io/api v0.35.3 h1:pA2fiBc6+N9PDf7SAiluKGEBuScsTzd2uYBkA5RzNWQ=
k8s.io/api v0.35.3/go.mod h1:9Y9tkBcFwKNq2sxwZTQh1Njh9qHl81D0As56tu42GA4=
k8s.io/apimachinery v0.35.3 h1:MeaUwQCV3tjKP4bcwWGgZ/cp/vpsRnQzqO6J6tJyoF8=
k8s.io/apimachinery v0.35.3/go.mod h1:jQCgFZFR1F4Ik7hvr2g84RTJSZegBc8yHgFWKn//hns=
k8s.io/client-go v0.35.3 h1:s1lZbpN4uI6IxeTM2cpdtrwHcSOBML1ODNTCCfsP1pg=
k8s.io/client-go v0.35.3/go.mod h1:RzoXkc0mzpWIDvBrRnD+VlfXP+lRzqQjCmKtiwZ8Q9c=
k8s.io/api v0.36.2 h1:TF6YDLIzKfccK7cq9YpTcGX8TJmEkHVRv78DM51fRYY=
k8s.io/api v0.36.2/go.mod h1:F4LbMO4brjZYh7yFkXWhynSvtB7YauxV4c+HHkNRGNg=
k8s.io/apimachinery v0.36.2 h1:0PE/W/WNy1UX61NLbXY5TMbJ6UwLL6E6lAPkYrKFxbQ=
k8s.io/apimachinery v0.36.2/go.mod h1:fvf/HOLXq9RId0rnDIbN1OEBvHXdQbLMM8nu0LcBUf4=
k8s.io/client-go v0.36.2 h1:bfgxmFKc9CgqsgX4xKLAAdmTQlWee7Ob/HlDOrJ5TBI=
k8s.io/client-go v0.36.2/go.mod h1:1vgO4OAlfPnoLcb+Rze2GF5rAr14w8qjrYMoyXJzQj0=
k8s.io/klog/v2 v2.140.0 h1:Tf+J3AH7xnUzZyVVXhTgGhEKnFqye14aadWv7bzXdzc=
k8s.io/klog/v2 v2.140.0/go.mod h1:o+/RWfJ6PwpnFn7OyAG3QnO47BFsymfEfrz6XyYSSp0=
k8s.io/kube-openapi v0.0.0-20260414162039-ec9c827d403f h1:4Qiq0YAoQATdgmHALJWz9rJ4fj20pB3xebpB4CFNhYM=
k8s.io/kube-openapi v0.0.0-20260414162039-ec9c827d403f/go.mod h1:uGBT7iTA6c6MvqUvSXIaYZo9ukscABYi2btjhvgKGZ0=
k8s.io/utils v0.0.0-20260319190234-28399d86e0b5 h1:kBawHLSnx/mYHmRnNUf9d4CpjREbeZuxoSGOX/J+aYM=
k8s.io/utils v0.0.0-20260319190234-28399d86e0b5/go.mod h1:xDxuJ0whA3d0I4mf/C4ppKHxXynQ+fxnkmQH0vTHnuk=
k8s.io/kube-openapi v0.0.0-20260603220949-865597e52e25 h1:mPMaPMpBij2V1Wv/fR+HW124vVGXXvOSS9ver/9yjWs=
k8s.io/kube-openapi v0.0.0-20260603220949-865597e52e25/go.mod h1:V/QaCUYDa+0QpcHhVVc5l99Uz56wEMEXBSj9oCDkNDY=
k8s.io/utils v0.0.0-20260507154919-ff6756f316d2 h1:wU4tMEhLGgIbLvXQb1cfN+EcM0wf7zC6CPF+C79jroc=
k8s.io/utils v0.0.0-20260507154919-ff6756f316d2/go.mod h1:xDxuJ0whA3d0I4mf/C4ppKHxXynQ+fxnkmQH0vTHnuk=
sigs.k8s.io/json v0.0.0-20250730193827-2d320260d730 h1:IpInykpT6ceI+QxKBbEflcR5EXP7sU1kvOlxwZh5txg=
sigs.k8s.io/json v0.0.0-20250730193827-2d320260d730/go.mod h1:mdzfpAEoE6DHQEN0uh9ZbOCuHbLK5wOm7dK4ctXE9Tg=
sigs.k8s.io/randfill v1.0.0 h1:JfjMILfT8A6RbawdsK2JXGBR5AQVfd+9TbzrlneTyrU=
sigs.k8s.io/randfill v1.0.0/go.mod h1:XeLlZ/jmk4i1HRopwe7/aU3H5n1zNUcX6TM94b3QxOY=
sigs.k8s.io/structured-merge-diff/v6 v6.3.2 h1:kwVWMx5yS1CrnFWA/2QHyRVJ8jM6dBA80uLmm0wJkk8=
sigs.k8s.io/structured-merge-diff/v6 v6.3.2/go.mod h1:M3W8sfWvn2HhQDIbGWj3S099YozAsymCo/wrT5ohRUE=
sigs.k8s.io/structured-merge-diff/v6 v6.4.0 h1:qmp2e3ZfFi1/jJbDGpD4mt3wyp6PE1NfKHCYLqgNQJo=
sigs.k8s.io/structured-merge-diff/v6 v6.4.0/go.mod h1:M3W8sfWvn2HhQDIbGWj3S099YozAsymCo/wrT5ohRUE=
sigs.k8s.io/yaml v1.6.0 h1:G8fkbMSAFqgEFgh4b1wmtzDnioxFCUgTZhlbj5P9QYs=
sigs.k8s.io/yaml v1.6.0/go.mod h1:796bPqUfzR/0jLAl6XjHl3Ck7MiyVv8dbTdyT3/pMf4=

View File

@@ -179,6 +179,9 @@ func tryRemoveDir(dirPath string) bool {
// times simultaneously and properly close it, fs caching may still
// confuse NFS client.
if err := os.RemoveAll(dirEntryPath); err != nil {
if os.IsNotExist(err) {
return
}
if !isTemporaryNFSError(err) {
logger.Fatalf("FATAL: cannot remove %q: %s", dirEntryPath, err)
}
@@ -203,8 +206,9 @@ func tryRemoveDir(dirPath string) bool {
deleteFilePath := filepath.Join(dirPath, deleteDirFilename)
// Remove the deleteDirFilename file, since there are no other entries left in the directory.
MustRemovePath(deleteFilePath)
if !tryRemovePath(deleteFilePath) {
return false
}
// Sync the directory after the removing deletDirFilename file in order to make sure
// all the metadata files are removed at some exotic filesystems such as OSSFS2.
// See https://github.com/VictoriaMetrics/VictoriaLogs/issues/649
@@ -212,7 +216,9 @@ func tryRemoveDir(dirPath string) bool {
MustSyncPath(dirPath)
// Remove the dirPath itself
MustRemovePath(dirPath)
if !tryRemovePath(dirPath) {
return false
}
// Do not sync the parent directory for the dirPath - the caller can do this if needed.
// It is OK if the dirPath will remain undeleted after unclean shutdown - it will be deleted
@@ -221,6 +227,23 @@ func tryRemoveDir(dirPath string) bool {
return true
}
// tryRemovePath removes given path and returns true on success
// or false if error is temporary NFS error
func tryRemovePath(path string) bool {
if err := os.Remove(path); err != nil {
if os.IsNotExist(err) {
return true
}
if !isTemporaryNFSError(err) {
logger.Fatalf("FATAL: cannot remove %q: %s", path, err)
}
nfsDirRemoveFailedAttempts.Inc()
return false
}
return true
}
var (
dirRemoverWG sync.WaitGroup
nfsDirRemoveFailedAttempts = metrics.NewCounter(`vm_nfs_dir_remove_failed_attempts_total`)

View File

@@ -811,10 +811,20 @@ func LogError(req *http.Request, errStr string) {
logger.Errorf("uri: %s, remote address: %q: %s", uri, remoteAddr, errStr)
}
// tlsErrorSkipLogger must be passed as the out argument to log.New only.
// It suppresses noisy TCP probe errors on TLS connections to avoid log pollution.
//
// This cannot be implemented in net.Listener because a TLS handshake may take seconds,
// during which no other connections can be accepted. Therefor, the implementation inside net.Listener can lead to DoS.
// Once a connection is passed to the conn serve goroutine, there is no direct access to the handshake logic, so this indirect
// approach is used instead.
type tlsErrorSkipLogger struct{}
// Write filters out TLS handshake errors from health-check probes.
// log.Logger guarantees that each complete message is delivered in a single Write call
// and that calls are serialized, so we can safely inspect p for a "TLS handshake error".
// See https://github.com/golang/go/blob/38e988efb4b8f5e73e887027f386a342c138b649/src/log/log.go#L53-L57
func (*tlsErrorSkipLogger) Write(p []byte) (int, error) {
// skip common health check errors produced by Kubernetes and other tools
if bytes.Contains(p, []byte("TLS handshake error")) &&
(bytes.Contains(p, []byte("EOF")) || bytes.Contains(p, []byte("connection reset by peer"))) {
return len(p), nil

View File

@@ -48,6 +48,12 @@ const maxPartSize = 400e9
// The interval for flushing buffered data to parts, so it becomes visible to search.
const pendingItemsFlushInterval = time.Second
// The default interval for calling flushCallback when there is pending data to flush.
//
// It is set relatively high in order to improve the effectiveness of caches reset by flushCallback.
// It is used when the flushCallbackInterval arg at MustOpenTable is set to zero.
const defaultFlushCallbackInterval = 10 * time.Second
// maxItemsPerCachedPart is the maximum items per created part by the merge,
// which must be cached in the OS page cache.
//
@@ -88,6 +94,7 @@ type Table struct {
flushInterval time.Duration
flushCallback func()
flushCallbackInterval time.Duration
needFlushCallbackCall atomic.Bool
prepareBlock PrepareBlockCallback
@@ -332,11 +339,14 @@ func (pw *partWrapper) decRef() {
// Optional flushCallback is called every time new data batch is flushed
// to the underlying storage and becomes visible to search.
//
// The flushCallbackInterval is how often flushCallback is invoked when there is
// pending data to flush. If it is set to zero, then defaultFlushCallbackInterval is used.
//
// Optional prepareBlock is called during merge before flushing the prepared block
// to persistent storage.
//
// The table is created if it doesn't exist yet.
func MustOpenTable(path string, flushInterval time.Duration, flushCallback func(), prepareBlock PrepareBlockCallback, isReadOnly *atomic.Bool) *Table {
func MustOpenTable(path string, flushInterval time.Duration, flushCallback func(), flushCallbackInterval time.Duration, prepareBlock PrepareBlockCallback, isReadOnly *atomic.Bool) *Table {
path = filepath.Clean(path)
if flushInterval < pendingItemsFlushInterval {
@@ -345,6 +355,10 @@ func MustOpenTable(path string, flushInterval time.Duration, flushCallback func(
flushInterval = pendingItemsFlushInterval
}
if flushCallbackInterval <= 0 {
flushCallbackInterval = defaultFlushCallbackInterval
}
// Create a directory at the path if it doesn't exist yet.
fs.MustMkdirIfNotExist(path)
@@ -355,14 +369,15 @@ func MustOpenTable(path string, flushInterval time.Duration, flushCallback func(
fs.MustSyncPathAndParentDir(path)
tb := &Table{
path: path,
flushInterval: flushInterval,
flushCallback: flushCallback,
prepareBlock: prepareBlock,
isReadOnly: isReadOnly,
fileParts: pws,
inmemoryPartsLimitCh: make(chan struct{}, maxInmemoryParts),
stopCh: make(chan struct{}),
path: path,
flushInterval: flushInterval,
flushCallback: flushCallback,
flushCallbackInterval: flushCallbackInterval,
prepareBlock: prepareBlock,
isReadOnly: isReadOnly,
fileParts: pws,
inmemoryPartsLimitCh: make(chan struct{}, maxInmemoryParts),
stopCh: make(chan struct{}),
}
tb.mergeIdx.Store(uint64(time.Now().UnixNano()))
tb.rawItems.init()
@@ -429,9 +444,9 @@ func (tb *Table) startFlushCallbackWorker() {
}
tb.wg.Go(func() {
// call flushCallback once per 10 seconds in order to improve the effectiveness of caches,
// which are reset by the flushCallback.
d := timeutil.AddJitterToDuration(time.Second * 10)
// call flushCallback at flushCallbackInterval in order to improve the effectiveness
// of caches, which are reset by the flushCallback.
d := timeutil.AddJitterToDuration(tb.flushCallbackInterval)
tc := time.NewTicker(d)
for {
select {

View File

@@ -40,7 +40,7 @@ func TestTableSearchSerial(t *testing.T) {
func() {
// Re-open the table and verify the search works.
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
defer tb.MustClose()
if err := testTableSearchSerial(tb, items); err != nil {
t.Fatalf("unexpected error: %s", err)
@@ -70,7 +70,7 @@ func TestTableSearchConcurrent(t *testing.T) {
// Re-open the table and verify the search works.
func() {
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
defer tb.MustClose()
if err := testTableSearchConcurrent(tb, items); err != nil {
t.Fatalf("unexpected error: %s", err)
@@ -144,7 +144,7 @@ func newTestTable(r *rand.Rand, path string, itemsCount int) (*Table, []string,
flushes.Add(1)
}
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, flushCallback, nil, &isReadOnly)
tb := MustOpenTable(path, 0, flushCallback, 0, nil, &isReadOnly)
items := make([]string, itemsCount)
for i := range itemsCount {
item := fmt.Sprintf("%d:%d", r.Intn(1e9), i)

View File

@@ -33,7 +33,7 @@ func benchmarkTableSearch(b *testing.B, itemsCount int) {
// Force finishing pending merges
tb.MustClose()
var isReadOnly atomic.Bool
tb = MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb = MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
defer tb.MustClose()
keys := make([][]byte, len(items))

View File

@@ -18,14 +18,14 @@ func TestTableOpenClose(t *testing.T) {
// Create a new table
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
// Close it
tb.MustClose()
// Re-open created table multiple times.
for range 4 {
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
tb.MustClose()
}
}
@@ -35,7 +35,7 @@ func TestTableAddItemsTooLongItem(t *testing.T) {
fs.MustRemoveDir(path)
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
tb.AddItems([][]byte{make([]byte, maxInmemoryBlockSize+1)})
tb.MustClose()
fs.MustRemoveDir(path)
@@ -52,7 +52,7 @@ func TestTableAddItemsSerial(t *testing.T) {
flushes.Add(1)
}
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, flushCallback, nil, &isReadOnly)
tb := MustOpenTable(path, 0, flushCallback, 0, nil, &isReadOnly)
const itemsCount = 10e3
testAddItemsSerial(r, tb, itemsCount)
@@ -75,7 +75,7 @@ func TestTableAddItemsSerial(t *testing.T) {
testReopenTable(t, path, itemsCount)
// Add more items in order to verify merge between inmemory parts and file-based parts.
tb = MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb = MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
const moreItemsCount = itemsCount * 3
testAddItemsSerial(r, tb, moreItemsCount)
tb.MustClose()
@@ -99,7 +99,7 @@ func TestTableCreateSnapshotAt(t *testing.T) {
fs.MustRemoveDir(path)
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
// Write a lot of items into the table, so background merges would start.
const itemsCount = 3e5
@@ -111,7 +111,7 @@ func TestTableCreateSnapshotAt(t *testing.T) {
// Close and open the table in order to flush all the data to disk before creating snapshots.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4272#issuecomment-1550221840
tb.MustClose()
tb = MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb = MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
// Create multiple snapshots.
snapshot1 := path + "-test-snapshot1"
@@ -121,8 +121,8 @@ func TestTableCreateSnapshotAt(t *testing.T) {
tb.MustCreateSnapshotAt(snapshot2)
// Verify snapshots contain all the data.
tb1 := MustOpenTable(snapshot1, 0, nil, nil, &isReadOnly)
tb2 := MustOpenTable(snapshot2, 0, nil, nil, &isReadOnly)
tb1 := MustOpenTable(snapshot1, 0, nil, 0, nil, &isReadOnly)
tb2 := MustOpenTable(snapshot2, 0, nil, 0, nil, &isReadOnly)
var ts, ts1, ts2 TableSearch
ts.Init(tb, false)
@@ -197,7 +197,7 @@ func TestTableAddItemsConcurrentStress(t *testing.T) {
}
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, flushCallback, prepareBlock, &isReadOnly)
tb := MustOpenTable(path, 0, flushCallback, 0, prepareBlock, &isReadOnly)
testAddItems(tb)
@@ -232,7 +232,7 @@ func TestTableAddItemsConcurrent(t *testing.T) {
return data, items
}
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, flushCallback, prepareBlock, &isReadOnly)
tb := MustOpenTable(path, 0, flushCallback, 0, prepareBlock, &isReadOnly)
const itemsCount = 10e3
testAddItemsConcurrent(tb, itemsCount)
@@ -255,7 +255,7 @@ func TestTableAddItemsConcurrent(t *testing.T) {
testReopenTable(t, path, itemsCount)
// Add more items in order to verify merge between inmemory parts and file-based parts.
tb = MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb = MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
const moreItemsCount = itemsCount * 3
testAddItemsConcurrent(tb, moreItemsCount)
tb.MustClose()
@@ -292,7 +292,7 @@ func testReopenTable(t *testing.T, path string, itemsCount int) {
for range 10 {
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
var m TableMetrics
tb.UpdateMetrics(&m)
if n := m.TotalItemsCount(); n != uint64(itemsCount) {
@@ -308,7 +308,7 @@ func TestTableMustMergeInmemoryPartsFinal_pwsRefCount(t *testing.T) {
defer fs.MustRemoveDir(path)
var isReadOnly atomic.Bool
tb := MustOpenTable(path, 0, nil, nil, &isReadOnly)
tb := MustOpenTable(path, 0, nil, 0, nil, &isReadOnly)
defer tb.MustClose()
generatePartWrappers := func(n int) []*partWrapper {

View File

@@ -76,9 +76,6 @@ var (
"Every %d occurrence in the template is substituted with -promscrape.cluster.memberNum at urls to vmagent instances responsible for scraping the given target "+
"at /service-discovery page. For example -promscrape.cluster.memberURLTemplate='http://vmagent-%d:8429/targets'. "+
"See https://docs.victoriametrics.com/victoriametrics/vmagent/#scraping-big-number-of-targets for more details")
clusterShardByLabels = flagutil.NewArrayString("promscrape.cluster.shardByLabels", "Optional list of target labels, which will be used for sharding targets among cluster members "+
"if -promscrape.cluster.membersCount is greater than 1. If none of the specified labels are found in a target, then all the target labels will be used for sharding. "+
"See https://docs.victoriametrics.com/victoriametrics/vmagent/#scraping-big-number-of-targets for more info")
clusterReplicationFactor = flag.Int("promscrape.cluster.replicationFactor", 1, "The number of members in the cluster, which scrape the same targets. "+
"If the replication factor is greater than 1, then the deduplication must be enabled at remote storage side. "+
"See https://docs.victoriametrics.com/victoriametrics/vmagent/#scraping-big-number-of-targets for more info")
@@ -89,10 +86,7 @@ var (
"Bigger uncompressed responses are rejected. See also max_scrape_size option at https://docs.victoriametrics.com/victoriametrics/sd_configs/#scrape_configs")
)
var (
clusterMemberID int
clusterShardByLabelsSorted []string
)
var clusterMemberID int
func mustInitClusterMemberID() {
s := *clusterMemberNum
@@ -116,14 +110,6 @@ func mustInitClusterMemberID() {
clusterMemberID = n
}
func initClusterShardByLabels() {
if len(*clusterShardByLabels) == 0 {
return
}
clusterShardByLabelsSorted = slices.Clone(*clusterShardByLabels)
slices.Sort(clusterShardByLabelsSorted)
}
// Config represents essential parts from Prometheus config defined at https://prometheus.io/docs/prometheus/latest/configuration/configuration/
type Config struct {
Global GlobalConfig `yaml:"global,omitempty"`
@@ -1151,29 +1137,13 @@ func (stc *StaticConfig) appendScrapeWork(dst []*ScrapeWork, swc *scrapeWorkConf
return dst
}
func appendScrapeWorkKey(dst []byte, labels *promutil.Labels, shardByLabels []string) []byte {
originalDstLen := len(dst)
for _, targetLabelName := range shardByLabels {
for _, label := range labels.GetLabels() {
if label.Name == targetLabelName {
// Do not use strconv.AppendQuote, since it is slow according to CPU profile.
dst = append(dst, label.Name...)
dst = append(dst, '=')
dst = append(dst, label.Value...)
dst = append(dst, ',')
break
}
}
}
// none of the labels specified in -promscrape.cluster.shardByLabels is present, use all labels for backward compatibility.
if len(dst) == originalDstLen {
for _, label := range labels.GetLabels() {
dst = append(dst, label.Name...)
dst = append(dst, '=')
dst = append(dst, label.Value...)
dst = append(dst, ',')
}
return dst
func appendScrapeWorkKey(dst []byte, labels *promutil.Labels) []byte {
for _, label := range labels.GetLabels() {
// Do not use strconv.AppendQuote, since it is slow according to CPU profile.
dst = append(dst, label.Name...)
dst = append(dst, '=')
dst = append(dst, label.Value...)
dst = append(dst, ',')
}
return dst
}
@@ -1225,7 +1195,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1687#issuecomment-940629495
if *clusterMembersCount > 1 {
bb := scrapeWorkKeyBufPool.Get()
bb.B = appendScrapeWorkKey(bb.B[:0], labels, clusterShardByLabelsSorted)
bb.B = appendScrapeWorkKey(bb.B[:0], labels)
memberNums := getClusterMemberNumsForScrapeWork(bytesutil.ToUnsafeString(bb.B), *clusterMembersCount, *clusterReplicationFactor)
scrapeWorkKeyBufPool.Put(bb)
if !slices.Contains(memberNums, clusterMemberID) {

View File

@@ -148,112 +148,6 @@ func TestGetClusterMemberNumsForScrapeWork(t *testing.T) {
f("foo", 3, 2, []int{2, 0})
}
func TestAppendScrapeWorkKeyShardByLabels(t *testing.T) {
f := func(labelsA, labelsB map[string]string, shardByLabels []string, equal bool) {
t.Helper()
originValue := *clusterShardByLabels
*clusterShardByLabels = shardByLabels
initClusterShardByLabels()
keyA := string(appendScrapeWorkKey(nil, promutil.NewLabelsFromMap(labelsA), clusterShardByLabelsSorted))
keyB := string(appendScrapeWorkKey(nil, promutil.NewLabelsFromMap(labelsB), clusterShardByLabelsSorted))
if equal && keyA != keyB {
t.Fatalf("unexpected different scrape work keys for shardByLabels=%q;\nlabelsA=%v\nlabelsB=%v\nkeyA=%q\nkeyB=%q",
shardByLabels, labelsA, labelsB, keyA, keyB)
} else if !equal && keyA == keyB {
t.Fatalf("unexpected equal scrape work keys for shardByLabels=%q;\nlabelsA=%v\nlabelsB=%v\nkeyA=%q\nkeyB=%q",
shardByLabels, labelsA, labelsB, keyA, keyB)
}
*clusterShardByLabels = originValue
}
// didn't specify -promscrape.cluster.shardByLabels, and all labels are the same
f(
map[string]string{
"a": "aa",
"b": "bb",
"c": "cc",
"d": "dd"},
map[string]string{
"a": "aa",
"b": "bb",
"c": "cc",
"d": "dd"},
[]string{},
true,
)
// match all labels in -promscrape.cluster.shardByLabels, and they're the same
f(
map[string]string{
"a": "aa",
"b": "bb",
"c": "cc",
"d": "dd"},
map[string]string{
"c": "cc",
"a": "aa",
"b": "other",
"d": "other"},
[]string{"c", "a"},
true,
)
// match all labels in -promscrape.cluster.shardByLabels, and they're different
f(
map[string]string{
"a": "aa",
"b": "bb",
"c": "cc",
"d": "dd"},
map[string]string{
"a": "aa",
"b": "other",
"c": "cc-------",
"d": "other"},
[]string{"a", "c"},
false,
)
// match part of labels in -promscrape.cluster.shardByLabels, and they're the same
f(
map[string]string{
"a": "aa",
"c": "cc",
"d": "dd"},
map[string]string{
"a": "aa",
"c": "cc",
"e": "ee"},
[]string{"a", "b", "c"},
true,
)
// match part of labels in -promscrape.cluster.shardByLabels, and they're different
f(
map[string]string{
"a": "aa",
"c": "cc",
"d": "dd"},
map[string]string{
"a": "aa-------",
"c": "cc",
"e": "ee"},
[]string{"a", "b", "c"},
false,
)
// none of labels in -promscrape.cluster.shardByLabels is matched, so all labels will be used to sharding
f(
map[string]string{
"d": "dd",
"e": "ee"},
map[string]string{
"d": "dd",
"e": "ee"},
[]string{"a", "b", "c"},
true,
)
}
func TestLoadStaticConfigs(t *testing.T) {
scs, err := loadStaticConfigs("testdata/file_sd.json")
if err != nil {

View File

@@ -66,7 +66,6 @@ func CheckConfig() error {
// Scraped data is passed to pushData.
func Init(pushData func(at *auth.Token, wr *prompb.WriteRequest)) {
mustInitClusterMemberID()
initClusterShardByLabels()
globalStopChan = make(chan struct{})
scraperWG.Go(func() {
runScraper(*promscrapeConfigFile, pushData, globalStopChan)
@@ -156,9 +155,22 @@ func runScraper(configFile string, pushData func(at *auth.Token, wr *prompb.Writ
tickerCh = ticker.C
defer ticker.Stop()
}
stop := func() {
cfg.mustStop()
logger.Infof("stopping Prometheus scrapers")
startTime := time.Now()
scs.stop()
logger.Infof("stopped Prometheus scrapers in %.3f seconds", time.Since(startTime).Seconds())
}
for {
scs.updateConfig(cfg)
waitForChans:
select {
case <-globalStopCh:
stop()
return
default:
}
select {
case <-sighupCh:
logger.Infof("SIGHUP received; reloading Prometheus configs from %q", configFile)
@@ -197,11 +209,7 @@ func runScraper(configFile string, pushData func(at *auth.Token, wr *prompb.Writ
configReloads.Inc()
configTimestamp.Set(fasttime.UnixTimestamp())
case <-globalStopCh:
cfg.mustStop()
logger.Infof("stopping Prometheus scrapers")
startTime := time.Now()
scs.stop()
logger.Infof("stopped Prometheus scrapers in %.3f seconds", time.Since(startTime).Seconds())
stop()
return
}
}

View File

@@ -172,7 +172,7 @@ func mustOpenIndexDB(id uint64, tr TimeRange, name, path string, s *Storage, isR
}
tfssCache := lrucache.NewCache(getTagFiltersCacheSize)
tb := mergeset.MustOpenTable(path, dataFlushInterval, tfssCache.Reset, mergeTagToMetricIDsRows, isReadOnly)
tb := mergeset.MustOpenTable(path, dataFlushInterval, tfssCache.Reset, 0, mergeTagToMetricIDsRows, isReadOnly)
db := &indexDB{
legacyMinMissingTimestampByKey: make(map[string]int64),
id: id,

View File

@@ -1,6 +1,10 @@
# Changes
## [1.11.0](https://github.com/googleapis/google-cloud-go/releases/tag/iam%2Fv1.11.0) (2026-05-07)
## [1.10.0](https://github.com/googleapis/google-cloud-go/releases/tag/iam%2Fv1.10.0) (2026-04-30)
## [1.9.0](https://github.com/googleapis/google-cloud-go/releases/tag/iam%2Fv1.9.0) (2026-04-13)
## [1.8.0](https://github.com/googleapis/google-cloud-go/releases/tag/iam%2Fv1.8.0) (2026-04-09)

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/iam/v1/iam_policy.proto
package iampb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/iam/v1/iam_policy.proto
package iampb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/iam/v1/options.proto
package iampb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/iam/v1/policy.proto
package iampb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/iam/v1/resource_policy_member.proto
package iampb

View File

@@ -331,7 +331,7 @@ func (c *alertPolicyGRPCClient) ListAlertPolicies(ctx context.Context, req *moni
}
opts = append((*c.CallOptions).ListAlertPolicies[0:len((*c.CallOptions).ListAlertPolicies):len((*c.CallOptions).ListAlertPolicies)], opts...)
it := &AlertPolicyIterator{}
req = proto.Clone(req).(*monitoringpb.ListAlertPoliciesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.AlertPolicy, string, error) {
resp := &monitoringpb.ListAlertPoliciesResponse{}
if pageToken != "" {

View File

@@ -353,7 +353,7 @@ func (c *groupGRPCClient) ListGroups(ctx context.Context, req *monitoringpb.List
}
opts = append((*c.CallOptions).ListGroups[0:len((*c.CallOptions).ListGroups):len((*c.CallOptions).ListGroups)], opts...)
it := &GroupIterator{}
req = proto.Clone(req).(*monitoringpb.ListGroupsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.Group, string, error) {
resp := &monitoringpb.ListGroupsResponse{}
if pageToken != "" {
@@ -494,7 +494,7 @@ func (c *groupGRPCClient) ListGroupMembers(ctx context.Context, req *monitoringp
}
opts = append((*c.CallOptions).ListGroupMembers[0:len((*c.CallOptions).ListGroupMembers):len((*c.CallOptions).ListGroupMembers)], opts...)
it := &MonitoredResourceIterator{}
req = proto.Clone(req).(*monitoringpb.ListGroupMembersRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoredrespb.MonitoredResource, string, error) {
resp := &monitoringpb.ListGroupMembersResponse{}
if pageToken != "" {

View File

@@ -394,7 +394,7 @@ func (c *metricGRPCClient) ListMonitoredResourceDescriptors(ctx context.Context,
}
opts = append((*c.CallOptions).ListMonitoredResourceDescriptors[0:len((*c.CallOptions).ListMonitoredResourceDescriptors):len((*c.CallOptions).ListMonitoredResourceDescriptors)], opts...)
it := &MonitoredResourceDescriptorIterator{}
req = proto.Clone(req).(*monitoringpb.ListMonitoredResourceDescriptorsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoredrespb.MonitoredResourceDescriptor, string, error) {
resp := &monitoringpb.ListMonitoredResourceDescriptorsResponse{}
if pageToken != "" {
@@ -470,7 +470,7 @@ func (c *metricGRPCClient) ListMetricDescriptors(ctx context.Context, req *monit
}
opts = append((*c.CallOptions).ListMetricDescriptors[0:len((*c.CallOptions).ListMetricDescriptors):len((*c.CallOptions).ListMetricDescriptors)], opts...)
it := &MetricDescriptorIterator{}
req = proto.Clone(req).(*monitoringpb.ListMetricDescriptorsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*metricpb.MetricDescriptor, string, error) {
resp := &monitoringpb.ListMetricDescriptorsResponse{}
if pageToken != "" {
@@ -590,7 +590,7 @@ func (c *metricGRPCClient) ListTimeSeries(ctx context.Context, req *monitoringpb
}
opts = append((*c.CallOptions).ListTimeSeries[0:len((*c.CallOptions).ListTimeSeries):len((*c.CallOptions).ListTimeSeries)], opts...)
it := &TimeSeriesIterator{}
req = proto.Clone(req).(*monitoringpb.ListTimeSeriesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.TimeSeries, string, error) {
resp := &monitoringpb.ListTimeSeriesResponse{}
if pageToken != "" {

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/alert.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/alert_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/alert_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/common.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/dropped_labels.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/group.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/group_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/group_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/metric.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/metric_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/metric_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/mutation_record.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/notification.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/notification_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/notification_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/query_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/query_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/service_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/service_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/snooze.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/snooze_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/snooze_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/span_context.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/uptime.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/monitoring/v3/uptime_service.proto
package monitoringpb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/monitoring/v3/uptime_service.proto
package monitoringpb

View File

@@ -441,7 +441,7 @@ func (c *notificationChannelGRPCClient) ListNotificationChannelDescriptors(ctx c
}
opts = append((*c.CallOptions).ListNotificationChannelDescriptors[0:len((*c.CallOptions).ListNotificationChannelDescriptors):len((*c.CallOptions).ListNotificationChannelDescriptors)], opts...)
it := &NotificationChannelDescriptorIterator{}
req = proto.Clone(req).(*monitoringpb.ListNotificationChannelDescriptorsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.NotificationChannelDescriptor, string, error) {
resp := &monitoringpb.ListNotificationChannelDescriptorsResponse{}
if pageToken != "" {
@@ -517,7 +517,7 @@ func (c *notificationChannelGRPCClient) ListNotificationChannels(ctx context.Con
}
opts = append((*c.CallOptions).ListNotificationChannels[0:len((*c.CallOptions).ListNotificationChannels):len((*c.CallOptions).ListNotificationChannels)], opts...)
it := &NotificationChannelIterator{}
req = proto.Clone(req).(*monitoringpb.ListNotificationChannelsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.NotificationChannel, string, error) {
resp := &monitoringpb.ListNotificationChannelsResponse{}
if pageToken != "" {

View File

@@ -231,7 +231,7 @@ func (c *queryGRPCClient) QueryTimeSeries(ctx context.Context, req *monitoringpb
}
opts = append((*c.CallOptions).QueryTimeSeries[0:len((*c.CallOptions).QueryTimeSeries):len((*c.CallOptions).QueryTimeSeries)], opts...)
it := &TimeSeriesDataIterator{}
req = proto.Clone(req).(*monitoringpb.QueryTimeSeriesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.TimeSeriesData, string, error) {
resp := &monitoringpb.QueryTimeSeriesResponse{}
if pageToken != "" {

View File

@@ -436,7 +436,7 @@ func (c *serviceMonitoringGRPCClient) ListServices(ctx context.Context, req *mon
}
opts = append((*c.CallOptions).ListServices[0:len((*c.CallOptions).ListServices):len((*c.CallOptions).ListServices)], opts...)
it := &ServiceIterator{}
req = proto.Clone(req).(*monitoringpb.ListServicesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.Service, string, error) {
resp := &monitoringpb.ListServicesResponse{}
if pageToken != "" {
@@ -577,7 +577,7 @@ func (c *serviceMonitoringGRPCClient) ListServiceLevelObjectives(ctx context.Con
}
opts = append((*c.CallOptions).ListServiceLevelObjectives[0:len((*c.CallOptions).ListServiceLevelObjectives):len((*c.CallOptions).ListServiceLevelObjectives)], opts...)
it := &ServiceLevelObjectiveIterator{}
req = proto.Clone(req).(*monitoringpb.ListServiceLevelObjectivesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.ServiceLevelObjective, string, error) {
resp := &monitoringpb.ListServiceLevelObjectivesResponse{}
if pageToken != "" {

View File

@@ -312,7 +312,7 @@ func (c *snoozeGRPCClient) ListSnoozes(ctx context.Context, req *monitoringpb.Li
}
opts = append((*c.CallOptions).ListSnoozes[0:len((*c.CallOptions).ListSnoozes):len((*c.CallOptions).ListSnoozes)], opts...)
it := &SnoozeIterator{}
req = proto.Clone(req).(*monitoringpb.ListSnoozesRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.Snooze, string, error) {
resp := &monitoringpb.ListSnoozesResponse{}
if pageToken != "" {

View File

@@ -340,7 +340,7 @@ func (c *uptimeCheckGRPCClient) ListUptimeCheckConfigs(ctx context.Context, req
}
opts = append((*c.CallOptions).ListUptimeCheckConfigs[0:len((*c.CallOptions).ListUptimeCheckConfigs):len((*c.CallOptions).ListUptimeCheckConfigs)], opts...)
it := &UptimeCheckConfigIterator{}
req = proto.Clone(req).(*monitoringpb.ListUptimeCheckConfigsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.UptimeCheckConfig, string, error) {
resp := &monitoringpb.ListUptimeCheckConfigsResponse{}
if pageToken != "" {
@@ -475,7 +475,7 @@ func (c *uptimeCheckGRPCClient) ListUptimeCheckIps(ctx context.Context, req *mon
}
opts = append((*c.CallOptions).ListUptimeCheckIps[0:len((*c.CallOptions).ListUptimeCheckIps):len((*c.CallOptions).ListUptimeCheckIps)], opts...)
it := &UptimeCheckIpIterator{}
req = proto.Clone(req).(*monitoringpb.ListUptimeCheckIpsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*monitoringpb.UptimeCheckIp, string, error) {
resp := &monitoringpb.ListUptimeCheckIpsResponse{}
if pageToken != "" {

View File

@@ -17,4 +17,4 @@
package internal
// Version is the current tagged release of the library.
const Version = "1.27.0"
const Version = "1.29.0"

View File

@@ -1,6 +1,25 @@
# Changes
## [1.62.3](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.62.3) (2026-06-03)
### Bug Fixes
* add server closed idle connection to retriable errors (#14594) ([20b37d6](https://github.com/googleapis/google-cloud-go/commit/20b37d65d56d4b5e7b8d43b1f6b2ddefcd8944be))
* fix race condition during retries in gRPC writer (#14649) ([04b6c63](https://github.com/googleapis/google-cloud-go/commit/04b6c635c09de1772fcefd8a7fd5e4ffdd370b79))
## [1.62.2](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.62.2) (2026-05-18)
### Features
* enable open telemetry attrs (#14426) ([74eab64](https://github.com/googleapis/google-cloud-go/commit/74eab64d1b4e22d8c79b0de4e5fc9a36bc4c6c19))
### Bug Fixes
* Set default chunkRetryDeadline to 32s in NewWriterFromAppendableObject (#14458) ([ec7c7d6](https://github.com/googleapis/google-cloud-go/commit/ec7c7d66eb0bf6e52a3ae1f529cb8e5de6f8dc86))
* refactor userProject metadata propagation in ListObjects (#14533) ([fbb543e](https://github.com/googleapis/google-cloud-go/commit/fbb543e3bb0d9b45c8e9aa167b6551c154f23169))
* restore metadata operations timeout in gRPC (#14575) ([275ff56](https://github.com/googleapis/google-cloud-go/commit/275ff562aee8c0201b9e5bf2913bb85bcdbe947a))
## [1.62.1](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.62.1) (2026-04-13)
### Bug Fixes
@@ -23,6 +42,18 @@
* Update `EnableParallelUpload` documentation in `writer.go` (#14328) ([22d0749](http://github.com/googleapis/google-cloud-go/commit/22d0749f8fdbeaa34f2a836b63510bd0c3def990))
## [1.61.5](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.61.5) (2026-06-01)
### Bug Fixes
* fix race condition during retries in gRPC writer (#14649) ([e87213b](https://github.com/googleapis/google-cloud-go/commit/e87213b78affff3aafb6ee0ecd542d65719e5c5c))
## [1.61.4](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.61.4) (2026-05-21)
### Bug Fixes
* add server closed idle connection to retriable errors (#14594) ([37580e7](https://github.com/googleapis/google-cloud-go/commit/37580e7eb530bbcf54951425947060c51cb0b30a))
## [1.61.3](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.61.3) (2026-03-13)
### Documentation
@@ -70,6 +101,12 @@
* Update documentation for `BidiReadObject`, `ReadObjectRequest`, and `ObjectContexts` ([611f239](https://github.com/googleapis/google-cloud-go/commit/611f239219225fb03f6475c7238f497a349961e2))
## [1.59.3](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.59.3) (2026-05-05)
### Bug Fixes
* handle MRD hang corner case (#14509) ([1ca3b6f](https://github.com/googleapis/google-cloud-go/commit/1ca3b6f02d35f87c336e34358e16985557c7fd58))
## [1.59.2](https://github.com/googleapis/google-cloud-go/releases/tag/storage%2Fv1.59.2) (2026-01-28)
### Bug Fixes

View File

@@ -156,7 +156,7 @@ func newGRPCStorageClient(ctx context.Context, opts ...storageOption) (*grpcStor
s := initSettings(opts...)
s.clientOption = append(defaultGRPCOptions(), s.clientOption...)
// Disable all gax-level retries in favor of retry logic in the veneer client.
s.gax = append(s.gax, gax.WithRetry(nil), gax.WithTimeout(0))
s.gax = append(s.gax, gax.WithRetry(nil))
config := newStorageConfig(s.clientOption...)
if config.readAPIWasSet {
@@ -187,10 +187,23 @@ func newGRPCStorageClient(ctx context.Context, opts ...storageOption) (*grpcStor
if err != nil {
return nil, err
}
configureStreamingTimeouts(g)
c.raw = g
return c, nil
}
// configureStreamingTimeouts explicitly overrides default call timeouts to 0 (unbounded)
// for all generated payload streaming RPCs. This guarantees that long-running data reads
// and writes are not prematurely aborted by default transport deadlines, while allowing
// all transactional and metadata/unary operations to retain their safety deadlines.
func configureStreamingTimeouts(g *gapic.Client) {
g.CallOptions.ReadObject = append(g.CallOptions.ReadObject, gax.WithTimeout(0))
g.CallOptions.WriteObject = append(g.CallOptions.WriteObject, gax.WithTimeout(0))
g.CallOptions.BidiReadObject = append(g.CallOptions.BidiReadObject, gax.WithTimeout(0))
g.CallOptions.BidiWriteObject = append(g.CallOptions.BidiWriteObject, gax.WithTimeout(0))
g.CallOptions.CancelResumableWrite = append(g.CallOptions.CancelResumableWrite, gax.WithTimeout(0))
}
func (c *grpcStorageClient) routingInterceptors() (grpc.UnaryClientInterceptor, grpc.StreamClientInterceptor) {
unary := func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
ctx, err := c.prepareDirectPathMetadata(ctx, cc.Target())
@@ -536,6 +549,9 @@ func (c *grpcStorageClient) LockBucketRetentionPolicy(ctx context.Context, bucke
}
func (c *grpcStorageClient) ListObjects(ctx context.Context, bucket string, q *Query, opts ...storageOption) *ObjectIterator {
s := callSettings(c.settings, opts...)
if s.userProject != "" {
ctx = setUserProjectMetadata(ctx, s.userProject)
}
it := &ObjectIterator{
ctx: ctx,
}
@@ -556,9 +572,6 @@ func (c *grpcStorageClient) ListObjects(ctx context.Context, bucket string, q *Q
IncludeFoldersAsPrefixes: it.query.IncludeFoldersAsPrefixes,
Filter: it.query.Filter,
}
if s.userProject != "" {
ctx = setUserProjectMetadata(ctx, s.userProject)
}
fetch := func(pageSize int, pageToken string) (token string, err error) {
// Add trace span around List API call within the fetch.
ctx, _ = startSpan(ctx, "grpcStorageClient.ObjectsListCall")
@@ -567,7 +580,6 @@ func (c *grpcStorageClient) ListObjects(ctx context.Context, bucket string, q *Q
var gitr *gapic.ObjectIterator
err = run(it.ctx, func(ctx context.Context) error {
gitr = c.raw.ListObjects(ctx, req, s.gax...)
it.ctx = ctx
objects, token, err = gitr.InternalFetch(pageSize, pageToken)
return err
}, s.retry, s.idempotent)

View File

@@ -335,8 +335,7 @@ func (c *mrdAddStreamErrorCmd) apply(ctx context.Context, m *multiRangeDownloade
} else {
err = errors.New("no streams available")
}
m.setPermanentError(err)
m.failAllPending(m.getPermanentError())
m.failManager(err)
}
}
@@ -433,6 +432,13 @@ func (m *multiRangeDownloaderManager) add(output io.Writer, offset, length int64
}
func (m *multiRangeDownloaderManager) close(err error) error {
if m.ctx.Err() != nil {
m.wg.Wait()
if pErr := m.getPermanentError(); pErr != nil {
return pErr
}
return m.ctx.Err()
}
cmd := &mrdCloseCmd{err: err}
select {
case m.cmds <- cmd:
@@ -452,6 +458,10 @@ func (m *multiRangeDownloaderManager) close(err error) error {
}
func (m *multiRangeDownloaderManager) wait() {
if err := m.ctx.Err(); err != nil {
m.callbackWg.Wait()
return
}
doneC := make(chan struct{})
cmd := &mrdWaitCmd{doneC: doneC}
select {
@@ -476,7 +486,9 @@ func (m *multiRangeDownloaderManager) getHandle() []byte {
case <-m.ctx.Done():
return nil
}
if err := m.ctx.Err(); err != nil {
return nil
}
respC := make(chan []byte, 1)
cmd := &mrdGetHandleCmd{respC: respC}
select {
@@ -536,6 +548,9 @@ func (m *multiRangeDownloaderManager) eventLoop() {
defer m.cleanup()
for {
if m.ctx.Err() != nil {
return
}
var nextReq *storagepb.BidiReadObjectRequest
var nextRangeReq *rangeRequest
var targetStream *mrdStream
@@ -603,10 +618,18 @@ func (m *multiRangeDownloaderManager) cleanup() {
}
// Drain and free any remaining responses to prevent buffer leaks.
close(m.sessionResps)
for result := range m.sessionResps {
if result.decoder != nil {
result.decoder.databufs.Free()
sessionDrainLoop:
for {
select {
case result, ok := <-m.sessionResps:
if !ok {
break sessionDrainLoop
}
if result.decoder != nil {
result.decoder.databufs.Free()
}
default:
break sessionDrainLoop
}
}
@@ -624,6 +647,39 @@ func (m *multiRangeDownloaderManager) cleanup() {
close(waiter)
}
m.attrsOnce.Do(func() { close(m.attrsReady) })
// Complete any commands leftover in cmds channel.
cmdDrainLoop:
for {
select {
case cmd, ok := <-m.cmds:
if !ok {
break cmdDrainLoop
}
// Parse type of command.
switch cmd := cmd.(type) {
case *mrdCloseCmd:
case *mrdWaitCmd:
close(cmd.doneC)
case *mrdAddCmd:
m.runCallback(cmd.offset, cmd.length, finalErr, cmd.callback)
case *mrdGetHandleCmd:
// Non-blocking send of handle if attributes are ready, otherwise close.
select {
case <-m.attrsReady:
select {
case cmd.respC <- m.lastReadHandle:
default:
close(cmd.respC)
}
default:
close(cmd.respC)
}
}
default:
break cmdDrainLoop
}
}
// Wait for all callbacks (including any initiated by the drained command) to finish.
m.callbackWg.Wait()
}
@@ -779,12 +835,17 @@ func (m *multiRangeDownloaderManager) handleAddCmd(ctx context.Context, cmd *mrd
return
}
}
if m.attrs != nil && req.length == 0 {
req.length = m.attrs.Size - req.offset
}
m.unsentRequests.PushBack(req)
}
func (m *multiRangeDownloaderManager) shouldAddStream() bool {
if m.streamCreating || len(m.streams) >= m.params.maxConnections {
if m.ctx.Err() != nil ||
m.streamCreating ||
len(m.streams) >= m.params.maxConnections {
return false
}
if len(m.streams) < m.params.minConnections {
@@ -843,6 +904,25 @@ func (m *multiRangeDownloaderManager) handleWaitCmd(ctx context.Context, cmd *mr
}
func (m *multiRangeDownloaderManager) handleAddStreamCmd(ctx context.Context, cmd *addStreamCmd) {
// Check for any error in stream before adding this stream.
var streamErr error
if cmd.stream != nil && cmd.stream.session != nil {
streamErr = cmd.stream.session.getError()
}
if cmd.stream == nil ||
cmd.stream.session == nil ||
streamErr != nil {
m.streamCreating = false
if len(m.streams) == 0 {
err := streamErr
if err == nil {
err = errors.New("no streams available: stream creation failed or has error")
}
m.failManager(err)
}
return
}
m.streams[cmd.id] = cmd.stream
if cmd.spec != nil {
m.readSpec = cmd.spec
@@ -860,13 +940,23 @@ func (m *multiRangeDownloaderManager) handleReconnectStreamCmd(ctx context.Conte
return
}
stream.reconnecting = false
if cmd.err != nil {
m.failStream(stream, cmd.err)
var streamErr error
if cmd.session != nil {
streamErr = cmd.session.getError()
}
if cmd.err != nil ||
cmd.session == nil ||
streamErr != nil {
finalErr := cmd.err
if finalErr == nil && cmd.session == nil {
finalErr = errors.New("session nil for reconnected stream")
} else if finalErr == nil {
finalErr = streamErr
}
m.failStream(stream, finalErr)
if len(m.streams) == 0 && !m.streamCreating {
err := fmt.Errorf("no streams available. Last observed error: %w", cmd.err)
m.setPermanentError(err)
m.failAllPending(m.getPermanentError())
err := fmt.Errorf("no streams available. Last observed error: %w", finalErr)
m.failManager(err)
}
return
}
@@ -1034,8 +1124,7 @@ func (m *multiRangeDownloaderManager) handleStreamEnd(result mrdSessionResult, s
m.failStream(stream, err)
if len(m.streams) == 0 && !m.streamCreating {
err := fmt.Errorf("no streams available. Last observed error: %w", err)
m.setPermanentError(err)
m.failAllPending(m.getPermanentError())
m.failManager(err)
}
}
}
@@ -1101,6 +1190,12 @@ func (m *multiRangeDownloaderManager) setPermanentError(err error) {
}
}
func (m *multiRangeDownloaderManager) failManager(err error) {
m.setPermanentError(err)
m.failAllPending(m.getPermanentError())
m.cancel()
}
// --- bidiReadStreamSession ---
// Controls lifespan of an individual bi-directional gRPC stream to the
// object in GCS. Spins up goroutines for the read and write sides of the
@@ -1121,8 +1216,12 @@ type bidiReadStreamSession struct {
respC chan<- mrdSessionResult
wg sync.WaitGroup
errOnce sync.Once
streamErr error
// To make manualShutdown and streamErr fields thread-safe
// as it can be accessed by both event loop and receive loop.
mu sync.RWMutex
errOnce sync.Once
streamErr error
manualShutdown bool
}
func newBidiReadStreamSession(ctx context.Context, id int, respC chan<- mrdSessionResult, client *grpcStorageClient, settings *settings, params *newMultiRangeDownloaderParams, readSpec *storagepb.BidiReadObjectSpec) (*bidiReadStreamSession, error) {
@@ -1177,14 +1276,26 @@ func (s *bidiReadStreamSession) SendRequest(req *storagepb.BidiReadObjectRequest
}
}
func (s *bidiReadStreamSession) Shutdown() {
s.mu.Lock()
s.manualShutdown = true
s.mu.Unlock()
s.cancel()
s.wg.Wait()
s.setError(s.ctx.Err())
}
func (s *bidiReadStreamSession) setError(err error) {
s.errOnce.Do(func() {
s.mu.Lock()
defer s.mu.Unlock()
s.streamErr = err
})
}
func (s *bidiReadStreamSession) getError() error {
s.mu.RLock()
defer s.mu.RUnlock()
return s.streamErr
}
func (s *bidiReadStreamSession) sendLoop() {
defer s.wg.Done()
defer s.stream.CloseSend()
@@ -1224,6 +1335,13 @@ func (s *bidiReadStreamSession) receiveLoop() {
if err != nil {
databufs.Free()
s.mu.RLock()
isManual := s.manualShutdown
s.mu.RUnlock()
if isManual {
return
}
redirectErr, isRedirect := isRedirectError(err)
result := mrdSessionResult{
err: err,
@@ -1257,9 +1375,17 @@ func (s *bidiReadStreamSession) receiveLoop() {
}:
case <-s.ctx.Done():
s.mu.RLock()
isManual := s.manualShutdown
s.mu.RUnlock()
if isManual {
databufs.Free()
return
}
// If context is cancelled unexpectedly, make sure to notify
// eventLoop before returning
err := s.streamErr
err := s.getError()
if err == nil {
err = s.ctx.Err()
}

View File

@@ -443,6 +443,31 @@ func (w *gRPCWriter) writeLoop(ctx context.Context) error {
defer cancel()
w.streamSender.connect(ctx, bscs, w.settings.gax...)
// Drain any initial completions (like QueryWriteStatus results).
Loop:
for {
select {
case c, ok := <-completions:
if !ok {
return w.streamSender.err()
}
w.handleCompletion(c)
default:
break Loop
}
}
if w.bufFlushedIdx > 0 {
copy(w.buf, w.buf[w.bufFlushedIdx:])
w.buf = w.buf[:len(w.buf)-w.bufFlushedIdx]
w.bufBaseOffset += int64(w.bufFlushedIdx)
w.bufUnsentIdx -= w.bufFlushedIdx
if w.bufUnsentIdx < 0 {
w.bufUnsentIdx = 0
}
w.bufFlushedIdx = -1
}
// Send any full quantum in w.buf, possibly including a flush
if err := w.withCommandRetryDeadline(func() error {
sentOffset, ok := w.sendBufferToTarget(chcs, w.buf, w.bufBaseOffset, cap(w.buf),
@@ -591,6 +616,26 @@ func (c *gRPCWriterCommandWrite) handle(w *gRPCWriter, cs gRPCWriterCommandHandl
return nil
}
if !c.hasStarted {
c.initialOffset = w.bufBaseOffset + int64(len(w.buf))
c.hasStarted = true
} else {
// Retrying this command; check if server has persisted some bytes of this command's payload.
bytesPersisted := w.bufBaseOffset - c.initialOffset
if bytesPersisted > 0 {
if int64(len(c.p)) < bytesPersisted {
bytesPersisted = int64(len(c.p))
}
c.p = c.p[bytesPersisted:]
c.initialOffset = w.bufBaseOffset
if len(c.p) == 0 {
c.markDone()
return nil
}
}
}
// Zero-Copy send.
if w.forceOneShot {
err := c.zeroCopyWrite(w, cs)
@@ -650,6 +695,7 @@ func (c *gRPCWriterCommandWrite) handle(w *gRPCWriter, cs gRPCWriterCommandHandl
w.buf = w.buf[:wblen+toNextWriteQuantum]
copied := copy(w.buf[wblen:], c.p)
c.p = c.p[copied:]
c.initialOffset += int64(copied)
firstFullBufFromCmd := cap(w.buf) - len(w.buf)
sending := w.buf[w.bufUnsentIdx:]
@@ -674,6 +720,7 @@ func (c *gRPCWriterCommandWrite) handle(w *gRPCWriter, cs gRPCWriterCommandHandl
trim = len(c.p)
}
c.p = c.p[trim:]
c.initialOffset += int64(trim)
cmdBaseOffset = bufTail
}
offset := cmdBaseOffset

View File

@@ -29,6 +29,7 @@ import (
iampb "cloud.google.com/go/iam/apiv1/iampb"
storagepb "cloud.google.com/go/storage/internal/apiv2/storagepb"
gax "github.com/googleapis/gax-go/v2"
"github.com/googleapis/gax-go/v2/callctx"
"google.golang.org/api/iterator"
"google.golang.org/api/option"
"google.golang.org/api/option/internaloption"
@@ -985,6 +986,16 @@ type gRPCClient struct {
// any other character (no special directory semantics).
func NewClient(ctx context.Context, opts ...option.ClientOption) (*Client, error) {
clientOpts := defaultGRPCClientOptions()
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
clientOpts = append(clientOpts, internaloption.WithTelemetryAttributes(map[string]string{
"gcp.client.service": "storage",
"gcp.client.version": getVersionClient(),
"gcp.client.repo": "googleapis/google-cloud-go",
"gcp.client.artifact": "cloud.google.com/go/storage/internal/apiv2",
"gcp.client.language": "go",
"url.domain": "storage.googleapis.com",
}))
}
if newClientHook != nil {
hookOpts, err := newClientHook(ctx, clientHookParams{})
if err != nil {
@@ -1006,6 +1017,43 @@ func NewClient(ctx context.Context, opts ...option.ClientOption) (*Client, error
logger: internaloption.GetLogger(opts),
}
c.setGoogleClientInfo()
if gax.IsFeatureEnabled("METRICS") {
metrics := gax.NewClientMetrics(
gax.WithTelemetryLogger(c.logger),
gax.WithTelemetryAttributes(map[string]string{
gax.ClientService: "storage",
gax.ClientVersion: getVersionClient(),
gax.ClientArtifact: "cloud.google.com/go/storage/internal/apiv2",
gax.RPCSystem: "grpc",
gax.URLDomain: "storage.googleapis.com",
}),
)
client.CallOptions.DeleteBucket = append(client.CallOptions.DeleteBucket, gax.WithClientMetrics(metrics))
client.CallOptions.GetBucket = append(client.CallOptions.GetBucket, gax.WithClientMetrics(metrics))
client.CallOptions.CreateBucket = append(client.CallOptions.CreateBucket, gax.WithClientMetrics(metrics))
client.CallOptions.ListBuckets = append(client.CallOptions.ListBuckets, gax.WithClientMetrics(metrics))
client.CallOptions.LockBucketRetentionPolicy = append(client.CallOptions.LockBucketRetentionPolicy, gax.WithClientMetrics(metrics))
client.CallOptions.GetIamPolicy = append(client.CallOptions.GetIamPolicy, gax.WithClientMetrics(metrics))
client.CallOptions.SetIamPolicy = append(client.CallOptions.SetIamPolicy, gax.WithClientMetrics(metrics))
client.CallOptions.TestIamPermissions = append(client.CallOptions.TestIamPermissions, gax.WithClientMetrics(metrics))
client.CallOptions.UpdateBucket = append(client.CallOptions.UpdateBucket, gax.WithClientMetrics(metrics))
client.CallOptions.ComposeObject = append(client.CallOptions.ComposeObject, gax.WithClientMetrics(metrics))
client.CallOptions.DeleteObject = append(client.CallOptions.DeleteObject, gax.WithClientMetrics(metrics))
client.CallOptions.RestoreObject = append(client.CallOptions.RestoreObject, gax.WithClientMetrics(metrics))
client.CallOptions.CancelResumableWrite = append(client.CallOptions.CancelResumableWrite, gax.WithClientMetrics(metrics))
client.CallOptions.GetObject = append(client.CallOptions.GetObject, gax.WithClientMetrics(metrics))
client.CallOptions.ReadObject = append(client.CallOptions.ReadObject, gax.WithClientMetrics(metrics))
client.CallOptions.BidiReadObject = append(client.CallOptions.BidiReadObject, gax.WithClientMetrics(metrics))
client.CallOptions.UpdateObject = append(client.CallOptions.UpdateObject, gax.WithClientMetrics(metrics))
client.CallOptions.WriteObject = append(client.CallOptions.WriteObject, gax.WithClientMetrics(metrics))
client.CallOptions.BidiWriteObject = append(client.CallOptions.BidiWriteObject, gax.WithClientMetrics(metrics))
client.CallOptions.ListObjects = append(client.CallOptions.ListObjects, gax.WithClientMetrics(metrics))
client.CallOptions.RewriteObject = append(client.CallOptions.RewriteObject, gax.WithClientMetrics(metrics))
client.CallOptions.StartResumableWrite = append(client.CallOptions.StartResumableWrite, gax.WithClientMetrics(metrics))
client.CallOptions.QueryWriteStatus = append(client.CallOptions.QueryWriteStatus, gax.WithClientMetrics(metrics))
client.CallOptions.MoveObject = append(client.CallOptions.MoveObject, gax.WithClientMetrics(metrics))
}
client.internalClient = c
@@ -1051,6 +1099,12 @@ func (c *gRPCClient) DeleteBucket(ctx context.Context, req *storagepb.DeleteBuck
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetName()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/DeleteBucket")
}
opts = append((*c.CallOptions).DeleteBucket[0:len((*c.CallOptions).DeleteBucket):len((*c.CallOptions).DeleteBucket)], opts...)
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
var err error
@@ -1074,6 +1128,12 @@ func (c *gRPCClient) GetBucket(ctx context.Context, req *storagepb.GetBucketRequ
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetName()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/GetBucket")
}
opts = append((*c.CallOptions).GetBucket[0:len((*c.CallOptions).GetBucket):len((*c.CallOptions).GetBucket)], opts...)
var resp *storagepb.Bucket
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1104,6 +1164,12 @@ func (c *gRPCClient) CreateBucket(ctx context.Context, req *storagepb.CreateBuck
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetParent()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/CreateBucket")
}
opts = append((*c.CallOptions).CreateBucket[0:len((*c.CallOptions).CreateBucket):len((*c.CallOptions).CreateBucket)], opts...)
var resp *storagepb.Bucket
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1131,9 +1197,15 @@ func (c *gRPCClient) ListBuckets(ctx context.Context, req *storagepb.ListBuckets
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetParent()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/ListBuckets")
}
opts = append((*c.CallOptions).ListBuckets[0:len((*c.CallOptions).ListBuckets):len((*c.CallOptions).ListBuckets)], opts...)
it := &BucketIterator{}
req = proto.Clone(req).(*storagepb.ListBucketsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*storagepb.Bucket, string, error) {
resp := &storagepb.ListBucketsResponse{}
if pageToken != "" {
@@ -1186,6 +1258,12 @@ func (c *gRPCClient) LockBucketRetentionPolicy(ctx context.Context, req *storage
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/LockBucketRetentionPolicy")
}
opts = append((*c.CallOptions).LockBucketRetentionPolicy[0:len((*c.CallOptions).LockBucketRetentionPolicy):len((*c.CallOptions).LockBucketRetentionPolicy)], opts...)
var resp *storagepb.Bucket
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1216,6 +1294,12 @@ func (c *gRPCClient) GetIamPolicy(ctx context.Context, req *iampb.GetIamPolicyRe
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetResource()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/GetIamPolicy")
}
opts = append((*c.CallOptions).GetIamPolicy[0:len((*c.CallOptions).GetIamPolicy):len((*c.CallOptions).GetIamPolicy)], opts...)
var resp *iampb.Policy
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1246,6 +1330,12 @@ func (c *gRPCClient) SetIamPolicy(ctx context.Context, req *iampb.SetIamPolicyRe
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetResource()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/SetIamPolicy")
}
opts = append((*c.CallOptions).SetIamPolicy[0:len((*c.CallOptions).SetIamPolicy):len((*c.CallOptions).SetIamPolicy)], opts...)
var resp *iampb.Policy
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1279,6 +1369,12 @@ func (c *gRPCClient) TestIamPermissions(ctx context.Context, req *iampb.TestIamP
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetResource()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/TestIamPermissions")
}
opts = append((*c.CallOptions).TestIamPermissions[0:len((*c.CallOptions).TestIamPermissions):len((*c.CallOptions).TestIamPermissions)], opts...)
var resp *iampb.TestIamPermissionsResponse
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1306,6 +1402,9 @@ func (c *gRPCClient) UpdateBucket(ctx context.Context, req *storagepb.UpdateBuck
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/UpdateBucket")
}
opts = append((*c.CallOptions).UpdateBucket[0:len((*c.CallOptions).UpdateBucket):len((*c.CallOptions).UpdateBucket)], opts...)
var resp *storagepb.Bucket
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1333,6 +1432,12 @@ func (c *gRPCClient) ComposeObject(ctx context.Context, req *storagepb.ComposeOb
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetKmsKey()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/ComposeObject")
}
opts = append((*c.CallOptions).ComposeObject[0:len((*c.CallOptions).ComposeObject):len((*c.CallOptions).ComposeObject)], opts...)
var resp *storagepb.Object
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1360,6 +1465,12 @@ func (c *gRPCClient) DeleteObject(ctx context.Context, req *storagepb.DeleteObje
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/DeleteObject")
}
opts = append((*c.CallOptions).DeleteObject[0:len((*c.CallOptions).DeleteObject):len((*c.CallOptions).DeleteObject)], opts...)
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
var err error
@@ -1383,6 +1494,12 @@ func (c *gRPCClient) RestoreObject(ctx context.Context, req *storagepb.RestoreOb
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/RestoreObject")
}
opts = append((*c.CallOptions).RestoreObject[0:len((*c.CallOptions).RestoreObject):len((*c.CallOptions).RestoreObject)], opts...)
var resp *storagepb.Object
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1410,6 +1527,9 @@ func (c *gRPCClient) CancelResumableWrite(ctx context.Context, req *storagepb.Ca
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/CancelResumableWrite")
}
opts = append((*c.CallOptions).CancelResumableWrite[0:len((*c.CallOptions).CancelResumableWrite):len((*c.CallOptions).CancelResumableWrite)], opts...)
var resp *storagepb.CancelResumableWriteResponse
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1437,6 +1557,12 @@ func (c *gRPCClient) GetObject(ctx context.Context, req *storagepb.GetObjectRequ
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/GetObject")
}
opts = append((*c.CallOptions).GetObject[0:len((*c.CallOptions).GetObject):len((*c.CallOptions).GetObject)], opts...)
var resp *storagepb.Object
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1464,6 +1590,12 @@ func (c *gRPCClient) ReadObject(ctx context.Context, req *storagepb.ReadObjectRe
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/ReadObject")
}
opts = append((*c.CallOptions).ReadObject[0:len((*c.CallOptions).ReadObject):len((*c.CallOptions).ReadObject)], opts...)
var resp storagepb.Storage_ReadObjectClient
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1481,6 +1613,9 @@ func (c *gRPCClient) ReadObject(ctx context.Context, req *storagepb.ReadObjectRe
func (c *gRPCClient) BidiReadObject(ctx context.Context, opts ...gax.CallOption) (storagepb.Storage_BidiReadObjectClient, error) {
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, c.xGoogHeaders...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/BidiReadObject")
}
var resp storagepb.Storage_BidiReadObjectClient
opts = append((*c.CallOptions).BidiReadObject[0:len((*c.CallOptions).BidiReadObject):len((*c.CallOptions).BidiReadObject)], opts...)
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1510,6 +1645,9 @@ func (c *gRPCClient) UpdateObject(ctx context.Context, req *storagepb.UpdateObje
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/UpdateObject")
}
opts = append((*c.CallOptions).UpdateObject[0:len((*c.CallOptions).UpdateObject):len((*c.CallOptions).UpdateObject)], opts...)
var resp *storagepb.Object
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1525,6 +1663,9 @@ func (c *gRPCClient) UpdateObject(ctx context.Context, req *storagepb.UpdateObje
func (c *gRPCClient) WriteObject(ctx context.Context, opts ...gax.CallOption) (storagepb.Storage_WriteObjectClient, error) {
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, c.xGoogHeaders...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/WriteObject")
}
var resp storagepb.Storage_WriteObjectClient
opts = append((*c.CallOptions).WriteObject[0:len((*c.CallOptions).WriteObject):len((*c.CallOptions).WriteObject)], opts...)
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1542,6 +1683,9 @@ func (c *gRPCClient) WriteObject(ctx context.Context, opts ...gax.CallOption) (s
func (c *gRPCClient) BidiWriteObject(ctx context.Context, opts ...gax.CallOption) (storagepb.Storage_BidiWriteObjectClient, error) {
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, c.xGoogHeaders...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/BidiWriteObject")
}
var resp storagepb.Storage_BidiWriteObjectClient
opts = append((*c.CallOptions).BidiWriteObject[0:len((*c.CallOptions).BidiWriteObject):len((*c.CallOptions).BidiWriteObject)], opts...)
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1571,9 +1715,15 @@ func (c *gRPCClient) ListObjects(ctx context.Context, req *storagepb.ListObjects
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetParent()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/ListObjects")
}
opts = append((*c.CallOptions).ListObjects[0:len((*c.CallOptions).ListObjects):len((*c.CallOptions).ListObjects)], opts...)
it := &ObjectIterator{}
req = proto.Clone(req).(*storagepb.ListObjectsRequest)
req = proto.CloneOf(req)
it.InternalFetch = func(pageSize int, pageToken string) ([]*storagepb.Object, string, error) {
resp := &storagepb.ListObjectsResponse{}
if pageToken != "" {
@@ -1629,6 +1779,12 @@ func (c *gRPCClient) RewriteObject(ctx context.Context, req *storagepb.RewriteOb
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetDestinationBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/RewriteObject")
}
opts = append((*c.CallOptions).RewriteObject[0:len((*c.CallOptions).RewriteObject):len((*c.CallOptions).RewriteObject)], opts...)
var resp *storagepb.RewriteResponse
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1656,6 +1812,9 @@ func (c *gRPCClient) StartResumableWrite(ctx context.Context, req *storagepb.Sta
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/StartResumableWrite")
}
opts = append((*c.CallOptions).StartResumableWrite[0:len((*c.CallOptions).StartResumableWrite):len((*c.CallOptions).StartResumableWrite)], opts...)
var resp *storagepb.StartResumableWriteResponse
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1683,6 +1842,9 @@ func (c *gRPCClient) QueryWriteStatus(ctx context.Context, req *storagepb.QueryW
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/QueryWriteStatus")
}
opts = append((*c.CallOptions).QueryWriteStatus[0:len((*c.CallOptions).QueryWriteStatus):len((*c.CallOptions).QueryWriteStatus)], opts...)
var resp *storagepb.QueryWriteStatusResponse
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {
@@ -1710,6 +1872,12 @@ func (c *gRPCClient) MoveObject(ctx context.Context, req *storagepb.MoveObjectRe
hds = append(c.xGoogHeaders, hds...)
ctx = gax.InsertMetadataIntoOutgoingContext(ctx, hds...)
if gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "resource_name", fmt.Sprintf("//storage.googleapis.com/%v", req.GetBucket()))
}
if gax.IsFeatureEnabled("METRICS") || gax.IsFeatureEnabled("TRACING") || gax.IsFeatureEnabled("LOGGING") {
ctx = callctx.WithTelemetryContext(ctx, "rpc_method", "google.storage.v2.Storage/MoveObject")
}
opts = append((*c.CallOptions).MoveObject[0:len((*c.CallOptions).MoveObject):len((*c.CallOptions).MoveObject)], opts...)
var resp *storagepb.Object
err := gax.Invoke(ctx, func(ctx context.Context, settings gax.CallSettings) error {

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.36.11
// protoc v6.31.0
// protoc v6.33.2
// source: google/storage/v2/storage.proto
package storagepb

View File

@@ -15,7 +15,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v6.31.0
// - protoc v6.33.2
// source: google/storage/v2/storage.proto
package storagepb

View File

@@ -17,4 +17,4 @@
package internal
// Version is the current tagged release of the library.
const Version = "1.62.1"
const Version = "1.62.3"

View File

@@ -206,10 +206,11 @@ func ShouldRetry(err error) bool {
// https://cloud.google.com/storage/docs/exponential-backoff.
return e.Code == 408 || e.Code == 429 || (e.Code >= 500 && e.Code < 600)
case *net.OpError, *url.Error:
// Retry socket-level errors ECONNREFUSED and ECONNRESET (from syscall).
// Retry socket-level errors ECONNREFUSED and ECONNRESET (from syscall)
// and transport-level errors like server closed idle connections.
// Unfortunately the error type is unexported, so we resort to string
// matching.
retriable := []string{"connection refused", "connection reset", "broken pipe", "client connection lost"}
retriable := []string{"connection refused", "connection reset", "broken pipe", "client connection lost", "server closed idle connection"}
for _, s := range retriable {
if strings.Contains(e.Error(), s) {
return true

View File

@@ -496,11 +496,14 @@ func (mrd *MultiRangeDownloader) Add(output io.Writer, offset, length int64, cal
// Close the MultiRangeDownloader. It must be called when done reading.
// Adding new ranges after this has been called will cause an error.
//
// This will immediately close the stream and can result in a
// This will immediately close the streams and can result in a
// "stream closed early" error if a response for a range is still not processed.
// Call [MultiRangeDownloader.Wait] to avoid this error.
//
// If the downloader is in a permanent error state, this will return an error.
//
// This must not be called from the callback sent into Add command otherwise
// it could lead to a deadlock.
func (mrd *MultiRangeDownloader) Close() error {
err := mrd.impl.close(nil)
endSpan(mrd.impl.getSpanCtx(), err)

View File

@@ -1305,6 +1305,9 @@ func (o *ObjectHandle) NewWriterFromAppendableObject(ctx context.Context, opts *
if w.ChunkSize == 0 {
w.ChunkSize = googleapi.DefaultUploadChunkSize
}
if w.ChunkRetryDeadline == 0 {
w.ChunkRetryDeadline = defaultWriteChunkRetryDeadline
}
err := w.openWriter()
if err != nil {
return nil, 0, err

View File

@@ -1,5 +1,28 @@
# Release History
## 1.22.0 (2026-06-04)
### Features Added
* Added type `datetime.RFC7231` for date/time values in RFC 1123 format with a fixed GMT timezone.
### Other Changes
* Upgraded dependencies.
## 1.21.1 (2026-04-16)
### Bugs Fixed
* Fixed an issue in `ResponseError.Error()` where the request URL path was being logged unescaped.
* Redact query parameters when logging errors.
* For `runtime.JoinPaths`, don't add a slash between root and paths when `paths` starts with `?` (query string).
### Other Changes
* Upgraded to Go 1.25.0.
* Upgraded dependencies.
## 1.21.0 (2026-01-12)
### Features Added

View File

@@ -131,7 +131,7 @@ func (e *ResponseError) Error() string {
msg := &bytes.Buffer{}
if e.RawResponse != nil {
if e.RawResponse.Request != nil {
fmt.Fprintf(msg, "%s %s://%s%s\n", e.RawResponse.Request.Method, e.RawResponse.Request.URL.Scheme, e.RawResponse.Request.URL.Host, e.RawResponse.Request.URL.Path)
fmt.Fprintf(msg, "%s %s://%s%s\n", e.RawResponse.Request.Method, e.RawResponse.Request.URL.Scheme, e.RawResponse.Request.URL.Host, e.RawResponse.Request.URL.EscapedPath())
} else {
fmt.Fprintln(msg, "Request information not available")
}

View File

@@ -37,5 +37,5 @@ const (
Module = "azcore"
// Version is the semantic version (see http://semver.org) of this module.
Version = "v1.21.0"
Version = "v1.22.0"
)

View File

@@ -182,7 +182,7 @@ func (b *BearerTokenPolicy) handleChallenge(req *policy.Request, res *http.Respo
func checkHTTPSForAuth(req *policy.Request, allowHTTP bool) error {
if strings.ToLower(req.Raw().URL.Scheme) != "https" && !allowHTTP {
return errorinfo.NonRetriableError(errors.New("authenticated requests are not permitted for non TLS protected (https) endpoints"))
return errorinfo.NonRetriableError(errors.New("authorized requests are not permitted for non-TLS protected (https) endpoints"))
}
return nil
}

View File

@@ -5,10 +5,8 @@ package runtime
import (
"context"
"errors"
"fmt"
"net/http"
"net/url"
"strings"
"github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/exported"
@@ -73,14 +71,7 @@ func (h *httpTracePolicy) Do(req *policy.Request) (resp *http.Response, err erro
span.SetAttributes(tracing.Attribute{Key: attrAZServiceReqID, Value: reqID})
}
} else if err != nil {
var urlErr *url.Error
if errors.As(err, &urlErr) {
// calling *url.Error.Error() will include the unsanitized URL
// which we don't want. in addition, we already have the HTTP verb
// and sanitized URL in the trace so we aren't losing any info
err = urlErr.Err
}
span.SetStatus(tracing.SpanStatusError, err.Error())
span.SetStatus(tracing.SpanStatusError, getSanitizedURLString(err.Error(), req.Raw().URL, h.allowedQP))
}
span.End()
}()

View File

@@ -158,6 +158,27 @@ func getSanitizedURL(u url.URL, allowedQueryParams map[string]struct{}) string {
return u.String()
}
// getSanitizedURLString returns s with the query params of u redacted.
// if s doesn't contain u, then s is returned unchanged.
func getSanitizedURLString(s string, u *url.URL, allowedQueryParams map[string]struct{}) string {
if u.RawQuery == "" {
// the URL doesn't have any query params, so nothing to redact
return s
}
urlIndex := strings.Index(strings.ToLower(s), strings.ToLower(u.String()))
if urlIndex < 0 {
// the URL isn't in the string, so nothing to redact
return s
}
// replace the unsanitized URL in the error message with the sanitized version
sanitizedURL := getSanitizedURL(*u, allowedQueryParams)
s = s[:urlIndex] + sanitizedURL + s[urlIndex+len(u.String()):]
return s
}
// writeRequestWithResponse appends a formatted HTTP request into a Buffer. If request and/or err are
// not nil, then these are also written into the Buffer.
func (p *logPolicy) writeRequestWithResponse(b *bytes.Buffer, req *policy.Request, resp *http.Response, err error) {
@@ -171,7 +192,7 @@ func (p *logPolicy) writeRequestWithResponse(b *bytes.Buffer, req *policy.Reques
}
if err != nil {
fmt.Fprintln(b, " --------------------------------------------------------------------------------")
fmt.Fprint(b, " ERROR:\n"+err.Error()+"\n")
fmt.Fprint(b, " ERROR:\n"+getSanitizedURLString(err.Error(), req.Raw().URL, p.allowedQP)+"\n")
}
}

View File

@@ -90,12 +90,16 @@ func JoinPaths(root string, paths ...string) string {
}
if qps != "" {
p = p + "?" + qps
if strings.Contains(p, "?") {
p = p + "&" + qps
} else {
p = p + "?" + qps
}
}
if strings.HasSuffix(root, "/") && strings.HasPrefix(p, "/") {
root = root[:len(root)-1]
} else if !strings.HasSuffix(root, "/") && !strings.HasPrefix(p, "/") {
} else if !strings.HasSuffix(root, "/") && !strings.HasPrefix(p, "/") && !strings.HasPrefix(p, "?") {
p = "/" + p
}
return root + p

View File

@@ -1,11 +1,61 @@
# Release History
## 1.14.0 (2026-06-15)
### Breaking Changes
> These changes affect only code written against a beta version such as v1.14.0-beta.3
- Removed `WorkloadIdentityCredentialOptions.EnableAzureProxy`.
It will return in v1.15.0-beta.1
### Bugs Fixed
- `AzureDeveloperCLICredential` improved reporting of error messages returned from `azd`
### Other Changes
- Returned `azidentity` errors include links to the troubleshooting guide when appropriate
- This module now requires a minimum Go version of 1.25
- Upgraded dependencies
## 1.14.0-beta.3 (2026-02-10)
### Breaking Changes
> These changes affect only code written against a beta version such as v1.14.0-beta.2
- Renamed `WorkloadIdentityCredentialOptions.EnableAzureTokenProxy` to `EnableAzureProxy`
### Other Changes
- Removed extraneous JSON from `AzureDeveloperCLICredential` errors
## 1.14.0-beta.2 (2025-11-10)
### Breaking Changes
> These changes affect only code written against a beta version such as v1.13.0-beta.1
- `WorkloadIdentityCredential` identity binding mode is disabled by default. To enable it, set
`WorkloadIdentityCredentialOptions.EnableAzureTokenProxy` to `true`
- Removed identity binding mode support from `DefaultAzureCredential`. To use this feature, use
`WorkloadIdentityCredential` directly instead and set
`WorkloadIdentityCredentialOptions.EnableAzureTokenProxy` to `true`
### Bugs Fixed
- `AzureCLICredential` quoted arguments incorrectly on Windows
## 1.13.1 (2025-11-10)
### Bugs Fixed
- `AzureCLICredential` quoted arguments incorrectly on Windows
## 1.14.0-beta.1 (2025-10-07)
### Features Added
- Restored the `WorkloadIdentityCredential` identity binding mode support removed in v1.13.0
## 1.13.0 (2025-10-07)
### Features Added

View File

@@ -83,7 +83,7 @@ azlog.SetEvents(azidentity.EventAuthentication)
| Error |Description| Mitigation |
|---|---|---|
|"DefaultAzureCredential failed to acquire a token"|No credential in the `DefaultAzureCredential` chain provided a token|<ul><li>[Enable logging](#enable-and-configure-logging) to get further diagnostic information.</li><li>Consult the troubleshooting guide for underlying credential types for more information.</li><ul><li>[EnvironmentCredential](#troubleshoot-environmentcredential-authentication-issues)</li><li>[ManagedIdentityCredential](#troubleshoot-managedidentitycredential-authentication-issues)</li><li>[AzureCLICredential](#troubleshoot-azureclicredential-authentication-issues)</li></ul>|
|"DefaultAzureCredential failed to acquire a token"|No credential in the `DefaultAzureCredential` chain provided a token|<ul><li>[Enable logging](#enable-and-configure-logging) to get further diagnostic information.</li><li>Consult the troubleshooting guide for underlying credential types for more information.</li></ul><ul><li>[EnvironmentCredential](#troubleshoot-environmentcredential-authentication-issues)</li><li>[ManagedIdentityCredential](#troubleshoot-managedidentitycredential-authentication-issues)</li><li>[AzureCLICredential](#troubleshoot-azureclicredential-authentication-issues)</li></ul>|
|Error from the client with a status code of 401 or 403|Authentication succeeded but the authorizing Azure service responded with a 401 (Unauthorized), or 403 (Forbidden) status code|<ul><li>[Enable logging](#enable-and-configure-logging) to determine which credential in the chain returned the authenticating token.</li><li>If an unexpected credential is returning a token, check application configuration such as environment variables.</li><li>Ensure the correct role is assigned to the authenticated identity. For example, a service specific role rather than the subscription Owner role.</li></ul>|
|"managed identity timed out"|`DefaultAzureCredential` sets a short timeout on its first managed identity authentication attempt to prevent very long timeouts during local development when no managed identity is available. That timeout causes this error in production when an application requests a token before the hosting environment is ready to provide one.|Use [ManagedIdentityCredential](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#ManagedIdentityCredential) directly, at least in production. It doesn't set a timeout on its authentication attempts.|
|invalid AZURE_TOKEN_CREDENTIALS value "..."|AZURE_TOKEN_CREDENTIALS has an unexpected value|Specify a valid value as described in [DefaultAzureCredential documentation](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#DefaultAzureCredential)
@@ -130,7 +130,7 @@ azlog.SetEvents(azidentity.EventAuthentication)
|The requested identity hasnt been assigned to this resource.|The IMDS endpoint responded with a status code of 400, indicating the requested identity isnt assigned to the VM.|If using a user assigned identity, ensure the specified ID is correct.<p/><p/>If using a system assigned identity, make sure it has been enabled as described in [managed identity documentation](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm#enable-system-assigned-managed-identity-on-an-existing-vm).|
|The request failed due to a gateway error.|The request to the IMDS endpoint failed due to a gateway error, 502 or 504 status code.|IMDS doesn't support requests via proxy or gateway. Disable proxies or gateways running on the VM for requests to the IMDS endpoint `http://169.254.169.254`|
|No response received from the managed identity endpoint.|No response was received for the request to IMDS or the request timed out.|<ul><li>Ensure the VM is configured for managed identity as described in [managed identity documentation](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm).</li><li>Verify the IMDS endpoint is reachable on the VM. See [below](#verify-imds-is-available-on-the-vm) for instructions.</li></ul>|
|Multiple attempts failed to obtain a token from the managed identity endpoint.|The credential has exhausted its retries for a token request.|<ul><li>Refer to the error message for more details on specific failures.<li>Ensure the VM is configured for managed identity as described in [managed identity documentation](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm).</li><li>Verify the IMDS endpoint is reachable on the VM. See [below](#verify-imds-is-available-on-the-vm) for instructions.</li></ul>|
|Multiple attempts failed to obtain a token from the managed identity endpoint.|The credential has exhausted its retries for a token request.|<ul><li>Refer to the error message for more details on specific failures.</li><li>Ensure the VM is configured for managed identity as described in [managed identity documentation](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm).</li><li>Verify the IMDS endpoint is reachable on the VM. See [below](#verify-imds-is-available-on-the-vm) for instructions.</li></ul>|
#### Verify IMDS is available on the VM
@@ -193,17 +193,26 @@ az account get-access-token --output json --resource https://management.core.win
#### Verify the Azure Developer CLI can obtain tokens
You can manually verify that the Azure Developer CLI is properly authenticated and can obtain tokens. First, use the `config` command to verify the account that is currently logged in to the Azure Developer CLI.
You can manually verify that the Azure Developer CLI is properly authenticated and can obtain tokens. Execute the command corresponding to your CLI version to verify the account currently logged in.
```sh
azd config list
```
- In Azure Developer CLI versions >= 1.23.0:
```sh
azd auth status
```
- In Azure Developer CLI versions < 1.23.0:
```sh
azd config list
```
Once you've verified the Azure Developer CLI is using correct account, you can validate that it's able to obtain tokens for this account.
```sh
azd auth token --output json --scope https://management.core.windows.net/.default
```
>Note that output of this command will contain a valid access token, and SHOULD NOT BE SHARED to avoid compromising account security.
<a id="azure-pwsh"></a>
@@ -239,7 +248,7 @@ Get-AzAccessToken -ResourceUrl "https://management.core.windows.net"
| Error Message |Description| Mitigation |
|---|---|---|
|no client ID/tenant ID/token file specified|Incomplete configuration|In most cases these values are provided via environment variables set by Azure Workload Identity.<ul><li>If your application runs on Azure Kubernetes Service (AKS) or a cluster that has deployed the Azure Workload Identity admission webhook, check pod labels and service account configuration. See the [AKS documentation](https://learn.microsoft.com/azure/aks/workload-identity-deploy-cluster#disable-workload-identity) and [Azure Workload Identity troubleshooting guide](https://azure.github.io/azure-workload-identity/docs/troubleshooting.html) for more details.<li>If your application isn't running on AKS or your cluster hasn't deployed the Workload Identity admission webhook, set these values in `WorkloadIdentityCredentialOptions`
|no client ID/tenant ID/token file specified|Incomplete configuration|In most cases these values are provided via environment variables set by Azure Workload Identity.<ul><li>If your application runs on Azure Kubernetes Service (AKS) or a cluster that has deployed the Azure Workload Identity admission webhook, check pod labels and service account configuration. See the [AKS documentation](https://learn.microsoft.com/azure/aks/workload-identity-deploy-cluster#disable-workload-identity) and [Azure Workload Identity troubleshooting guide](https://azure.github.io/azure-workload-identity/docs/troubleshooting.html) for more details.</li><li>If your application isn't running on AKS or your cluster hasn't deployed the Workload Identity admission webhook, set these values in `WorkloadIdentityCredentialOptions`</li></ul>
<a id="apc"></a>
## Troubleshoot AzurePipelinesCredential authentication issues

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.
@@ -154,7 +151,7 @@ func validTenantID(tenantID string) bool {
return false
}
for _, r := range tenantID {
if !(alphanumeric(r) || r == '.' || r == '-') {
if !alphanumeric(r) && r != '.' && r != '-' {
return false
}
}

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.
@@ -57,7 +54,7 @@ func NewAzureCLICredential(options *AzureCLICredentialOptions) (*AzureCLICredent
cp = *options
}
for _, r := range cp.Subscription {
if !(alphanumeric(r) || r == '-' || r == '_' || r == ' ' || r == '.') {
if !alphanumeric(r) && r != '-' && r != '_' && r != ' ' && r != '.' {
return nil, fmt.Errorf(
"%s: Subscription %q contains invalid characters. If this is the name of a subscription, use its ID instead",
credNameAzureCLI,

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.
@@ -115,8 +112,6 @@ func (c *AzureDeveloperCLICredential) GetToken(ctx context.Context, opts policy.
mfaRequired+". Run this command then retry the operation: "+commandNoClaims,
nil,
)
case strings.Contains(msg, "azd auth login"):
err = newCredentialUnavailableError(credNameAzureDeveloperCLI, `please run "azd auth login" from a command prompt to authenticate before using this credential`)
}
err = unavailableIfInDAC(err, c.opts.inDefaultChain)
return at, err

View File

@@ -89,7 +89,7 @@ func NewAzurePipelinesCredential(tenantID, clientID, serviceConnectionID, system
options = &AzurePipelinesCredentialOptions{}
}
// these headers are useful to the DevOps team when debugging OIDC error responses
options.ClientOptions.Logging.AllowedHeaders = append(options.ClientOptions.Logging.AllowedHeaders, xMsEdgeRef, xVssE2eId)
options.Logging.AllowedHeaders = append(options.Logging.AllowedHeaders, xMsEdgeRef, xVssE2eId)
caco := ClientAssertionCredentialOptions{
AdditionallyAllowedTenants: options.AdditionallyAllowedTenants,
Cache: options.Cache,

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.

View File

@@ -29,13 +29,13 @@ extends:
SubscriptionConfigurations:
- $(sub-config-identity-test-resources)
EnableRaceDetector: true
Location: westus2
RunLiveTests: true
Location: westus3
ServiceDirectory: azidentity
UsePipelineProxy: false
${{ if endsWith(variables['Build.DefinitionName'], 'weekly') }}:
PersistOidcToken: true
RunLiveTests: true
MatrixConfigs:
- Name: managed_identity_matrix
GenerateVMJobs: true

View File

@@ -1,6 +1,3 @@
//go:build go1.18
// +build go1.18
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT License.

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