diff --git a/.gitignore b/.gitignore index 3bc0d4d..f190874 100644 --- a/.gitignore +++ b/.gitignore @@ -1,2 +1,4 @@ parquet-gateway .cover +.blocks +.data diff --git a/Makefile b/Makefile index b59fc11..a6eb4c3 100644 --- a/Makefile +++ b/Makefile @@ -1,29 +1,39 @@ -.PHONY: all build proto lint test +.PHONY: all ci build proto lint test-norace test deps all: build -build: proto parquet-gateway +ci: test-norace build lint + +build: protos parquet-gateway GO = go -GOIMPORTS = goimports -REVIVE = revive +GOTOOL = $(GO) tool -modfile=go.tools.mod +GOIMPORTS = $(GOTOOL) golang.org/x/tools/cmd/goimports +REVIVE = $(GOTOOL) github.com/mgechev/revive +MODERNIZE = $(GOTOOL) golang.org/x/tools/gopls/internal/analysis/modernize/cmd/modernize PROTOC = protoc lint: $(wildcard **/*.go) @echo ">> running lint..." - @$(REVIVE) -config revive.toml ./... + $(REVIVE) -config revive.toml ./... + $(MODERNIZE) -test ./... find . -name '*.go' ! -path './proto/*' | xargs $(GOIMPORTS) -l -w -local $(head -n 1 go.mod | cut -d ' ' -f 2) +test-norace: $(wildcard **/*.go) + @echo ">> running tests without checking for races..." + @mkdir -p .cover + $(GO) test -v -tags stringlabels -short -count=1 ./... -coverprofile .cover/cover.out + test: $(wildcard **/*.go) @echo ">> running tests..." @mkdir -p .cover - $(GO) test -v -race -count=1 ./... -coverprofile .cover/cover.out + $(GO) test -v -tags stringlabels -race -short -count=1 ./... -coverprofile .cover/cover.out -parquet-gateway: $(wildcard **/*.go) +parquet-gateway: $(shell find . -type f -name '*.go') @echo ">> building binaries..." - @$(GO) build -o parquet-gateway github.com/cloudflare/parquet-tsdb-poc/cmd + @$(GO) build -tags stringlabels -o parquet-gateway github.com/cloudflare/parquet-tsdb-poc/cmd -proto: proto/metapb/meta.pb.go +protos: proto/metapb/meta.pb.go proto/metapb/meta.pb.go: proto/metapb/meta.proto @echo ">> compiling protos..." diff --git a/README.md b/README.md index 7965755..8ad5645 100644 --- a/README.md +++ b/README.md @@ -38,7 +38,7 @@ This will: - load blocks from the `.data/my-prefix` directory - expose internal metrics and readiness handlers on port 6060 - expose a subset of the Prometheus HTTP API on port 9090 -- expose an Thanos Info and Query gRPC service on port 9091 +- expose an Thanos Info, Series and Query gRPC service on port 9091 You can now query it by pointing a Thanos Querier at it or through curl: @@ -73,6 +73,10 @@ To convert TSDB blocks in the `.data/source` directory that overlap `09/2021` an parquet-gateway convert \ --tsdb.storage.prefix source \ --parquet.storage.prefix destination \ - --convert.start=2021-09-01T00:00:00Z \ - --convert.end=2021-10-01T00:00:00Z + --convert.sorting.label=__name__ \ + --convert.sorting.label=namespace ``` + +## Note + +The code has significant overlap with the work in "https://github.com/prometheus-community/parquet-common". We are in the process of upstreaming and eventually plan to use it as a library. diff --git a/api/grpc/metrics.go b/api/grpc/metrics.go new file mode 100644 index 0000000..82e9405 --- /dev/null +++ b/api/grpc/metrics.go @@ -0,0 +1,28 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package grpc + +import ( + "errors" + + grpc_prom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" + "github.com/prometheus/client_golang/prometheus" +) + +var ( + ServerMetrics = grpc_prom.NewServerMetrics( + grpc_prom.WithServerHandlingTimeHistogram( + grpc_prom.WithHistogramOpts(&prometheus.HistogramOpts{ + Buckets: prometheus.ExponentialBucketsRange(0.1, 30, 20), + }), + ), + ) +) + +func RegisterMetrics(reg prometheus.Registerer) error { + return errors.Join( + reg.Register(ServerMetrics), + ) +} diff --git a/api/grpc/thanos.go b/api/grpc/thanos.go index 5f67868..cac9db6 100644 --- a/api/grpc/thanos.go +++ b/api/grpc/thanos.go @@ -9,8 +9,11 @@ import ( "fmt" "time" + "github.com/alecthomas/units" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" "github.com/prometheus/prometheus/util/stats" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -19,29 +22,113 @@ import ( "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/info/infopb" "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/store/storepb/prompb" - cfdb "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" ) // Taken from https://github.com/thanos-community/thanos-promql-connector/blob/main/main.go -type infoServer struct { +type queryGRPCConfig struct { + concurrentQuerySemaphore *limits.Semaphore + selectChunkBytesQuota units.Base2Bytes + selectRowCountQuota int64 + selectChunkPartitionMaxRange units.Base2Bytes + selectChunkPartitionMaxGap units.Base2Bytes + selectChunkPartitionMaxConcurrency int +} + +type QueryGRPCOption func(*queryGRPCConfig) + +func ConcurrentQueryQuota(n int) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.concurrentQuerySemaphore = limits.NewSempahore(n) + } +} + +func SelectChunkBytesQuota(q units.Base2Bytes) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.selectChunkBytesQuota = q + } +} + +func SelectRowCountQuota(n int64) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.selectRowCountQuota = n + } +} + +func SelectChunkPartitionMaxRange(v units.Base2Bytes) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.selectChunkPartitionMaxRange = v + } +} + +func SelectChunkPartitionMaxGap(v units.Base2Bytes) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.selectChunkPartitionMaxGap = v + } +} + +func SelectChunkPartitionMaxConcurrency(n int) QueryGRPCOption { + return func(qapi *queryGRPCConfig) { + qapi.selectChunkPartitionMaxConcurrency = n + } +} + +type QueryServer struct { + querypb.UnimplementedQueryServer infopb.UnimplementedInfoServer + storepb.UnimplementedStoreServer + + db *db.DB + engine promql.QueryEngine + + concurrentQuerySemaphore *limits.Semaphore + selectChunkBytesQuota units.Base2Bytes + selectRowCountQuota int64 + selectChunkPartitionMaxRange units.Base2Bytes + selectChunkPartitionMaxGap units.Base2Bytes + selectChunkPartitionMaxConcurrency int +} - db *cfdb.DB +func (qs *QueryServer) queryable(replicaLabels ...string) storage.Queryable { + return qs.db.Queryable( + db.DropReplicaLabels(replicaLabels...), + db.SelectChunkBytesQuota(qs.selectChunkBytesQuota), + db.SelectRowCountQuota(qs.selectRowCountQuota), + db.SelectChunkPartitionMaxRange(qs.selectChunkPartitionMaxRange), + db.SelectChunkPartitionMaxGap(qs.selectChunkPartitionMaxGap), + db.SelectChunkPartitionMaxConcurrency(qs.selectChunkPartitionMaxConcurrency), + ) } -func NewInfoServer(db *cfdb.DB) infopb.InfoServer { - return &infoServer{db: db} +func NewQueryServer(db *db.DB, engine promql.QueryEngine, opts ...QueryGRPCOption) *QueryServer { + cfg := queryGRPCConfig{concurrentQuerySemaphore: limits.UnlimitedSemaphore()} + for i := range opts { + opts[i](&cfg) + } + return &QueryServer{ + db: db, + engine: engine, + selectChunkBytesQuota: cfg.selectChunkBytesQuota, + selectRowCountQuota: cfg.selectRowCountQuota, + concurrentQuerySemaphore: cfg.concurrentQuerySemaphore, + selectChunkPartitionMaxRange: cfg.selectChunkPartitionMaxRange, + selectChunkPartitionMaxGap: cfg.selectChunkPartitionMaxGap, + selectChunkPartitionMaxConcurrency: cfg.selectChunkPartitionMaxConcurrency, + } } -func (info *infoServer) Info(_ context.Context, _ *infopb.InfoRequest) (*infopb.InfoResponse, error) { - mint, maxt := info.db.Timerange() - extlabels := info.db.Extlabels() +func (qs *QueryServer) Info(_ context.Context, _ *infopb.InfoRequest) (*infopb.InfoResponse, error) { + mint, maxt := qs.db.Timerange() + extlabels := qs.db.Extlabels() return &infopb.InfoResponse{ ComponentType: component.Query.String(), - LabelSets: labelpb.ZLabelSetsFromPromLabels(extlabels), + LabelSets: zLabelSetsFromPromLabels(extlabels), Store: &infopb.StoreInfo{ MinTime: mint, MaxTime: maxt, @@ -49,7 +136,7 @@ func (info *infoServer) Info(_ context.Context, _ *infopb.InfoRequest) (*infopb. { MinTime: mint, MaxTime: maxt, - Labels: labelpb.ZLabelSet{Labels: labelpb.ZLabelsFromPromLabels(extlabels)}, + Labels: labelpb.ZLabelSet{Labels: zLabelsFromMetric(extlabels)}, }, }, }, @@ -57,27 +144,21 @@ func (info *infoServer) Info(_ context.Context, _ *infopb.InfoRequest) (*infopb. }, nil } -type queryServer struct { - querypb.UnimplementedQueryServer - - db *cfdb.DB - engine promql.QueryEngine -} - -func NewQueryServer(db *cfdb.DB, engine promql.QueryEngine) querypb.QueryServer { - return &queryServer{db: db, engine: engine} -} - -func (qs *queryServer) Query(req *querypb.QueryRequest, srv querypb.Query_QueryServer) error { +func (qs *QueryServer) Query(req *querypb.QueryRequest, srv querypb.Query_QueryServer) error { ts := time.Unix(req.TimeSeconds, 0) timeout := time.Duration(req.TimeoutSeconds) * time.Second ctx, cancel := context.WithTimeout(srv.Context(), timeout) defer cancel() + if err := qs.concurrentQuerySemaphore.Reserve(ctx); err != nil { + return status.Error(codes.Aborted, fmt.Sprintf("semaphore blocked: %s", err)) + } + defer qs.concurrentQuerySemaphore.Release() + opts := promql.NewPrometheusQueryOpts(false, time.Duration(req.LookbackDeltaSeconds)) - qryable := qs.db.ReplicaQueryable(req.ReplicaLabels) + qryable := qs.queryable(req.ReplicaLabels...) qry, err := qs.engine.NewInstantQuery(ctx, qryable, opts, req.Query, ts) if err != nil { @@ -87,6 +168,9 @@ func (qs *queryServer) Query(req *querypb.QueryRequest, srv querypb.Query_QueryS res := qry.Exec(ctx) if err := res.Err; err != nil { + if limits.IsResourceExhausted(err) { + return status.Error(codes.ResourceExhausted, err.Error()) + } return status.Error(codes.Internal, fmt.Sprintf("query eval error: %s", err)) } if warnings := res.Warnings.AsErrors(); len(warnings) > 0 { @@ -123,7 +207,7 @@ func (qs *queryServer) Query(req *querypb.QueryRequest, srv querypb.Query_QueryS return nil } -func (qs *queryServer) QueryRange(req *querypb.QueryRangeRequest, srv querypb.Query_QueryRangeServer) error { +func (qs *QueryServer) QueryRange(req *querypb.QueryRangeRequest, srv querypb.Query_QueryRangeServer) error { start := time.Unix(req.StartTimeSeconds, 0) end := time.Unix(req.EndTimeSeconds, 0) step := time.Duration(req.IntervalSeconds) * time.Second @@ -132,7 +216,12 @@ func (qs *queryServer) QueryRange(req *querypb.QueryRangeRequest, srv querypb.Qu ctx, cancel := context.WithTimeout(srv.Context(), timeout) defer cancel() - qryable := qs.db.ReplicaQueryable(req.ReplicaLabels) + if err := qs.concurrentQuerySemaphore.Reserve(ctx); err != nil { + return status.Error(codes.Aborted, fmt.Sprintf("semaphore blocked: %s", err)) + } + defer qs.concurrentQuerySemaphore.Release() + + qryable := qs.queryable(req.ReplicaLabels...) opts := promql.NewPrometheusQueryOpts(false, time.Duration(req.LookbackDeltaSeconds)) qry, err := qs.engine.NewRangeQuery(ctx, qryable, opts, req.Query, start, end, step) @@ -143,6 +232,9 @@ func (qs *queryServer) QueryRange(req *querypb.QueryRangeRequest, srv querypb.Qu res := qry.Exec(ctx) if err := res.Err; err != nil { + if limits.IsResourceExhausted(err) { + return status.Error(codes.ResourceExhausted, err.Error()) + } return status.Error(codes.Internal, fmt.Sprintf("query eval error: %s", err)) } if warnings := res.Warnings.AsErrors(); len(warnings) > 0 { @@ -191,6 +283,109 @@ func (qs *queryServer) QueryRange(req *querypb.QueryRangeRequest, srv querypb.Qu return nil } +func (qs *QueryServer) Series(request *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { + if !request.SkipChunks { + return status.Error(codes.Unimplemented, "'series' called without skipping chunks") + } + + q, err := qs.queryable(request.WithoutReplicaLabels...).Querier(request.MinTime, request.MaxTime) + if err != nil { + return status.Error(codes.Internal, err.Error()) + } + defer q.Close() + + ms, err := storepb.MatchersToPromMatchers(request.Matchers...) + if err != nil { + return status.Error(codes.Internal, err.Error()) + } + + hints := &storage.SelectHints{ + Start: request.MinTime, + End: request.MaxTime, + Limit: int(request.Limit), + Func: "series", + } + + ss := q.Select(srv.Context(), true, hints, ms...) + + i := int64(0) + for ss.Next() { + i++ + + series := ss.At() + + if request.Limit > 0 && i > request.Limit { + if err := srv.Send(storepb.NewWarnSeriesResponse(warnings.ErrorTruncatedResponse)); err != nil { + return status.Error(codes.Aborted, err.Error()) + } + break + } + + storeSeries := storepb.Series{Labels: zLabelsFromMetric(series.Labels())} + if err := srv.Send(storepb.NewSeriesResponse(&storeSeries)); err != nil { + return status.Error(codes.Aborted, err.Error()) + } + } + + if err := ss.Err(); err != nil { + if limits.IsResourceExhausted(err) { + return status.Error(codes.ResourceExhausted, err.Error()) + } + return status.Error(codes.Internal, err.Error()) + } + for _, w := range ss.Warnings() { + if err := srv.Send(storepb.NewWarnSeriesResponse(w)); err != nil { + return status.Error(codes.Aborted, err.Error()) + } + } + + return nil +} + +func (qs *QueryServer) LabelNames(ctx context.Context, request *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { + q, err := qs.queryable(request.WithoutReplicaLabels...).Querier(request.Start, request.End) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + defer q.Close() + + ms, err := storepb.MatchersToPromMatchers(request.Matchers...) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + lns, warnings, err := q.LabelNames(ctx, &storage.LabelHints{Limit: int(request.Limit)}, ms...) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + + return &storepb.LabelNamesResponse{ + Names: lns, + Warnings: warningsAsStrings(warnings), + }, nil +} + +func (qs *QueryServer) LabelValues(ctx context.Context, request *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error) { + q, err := qs.queryable(request.WithoutReplicaLabels...).Querier(request.Start, request.End) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + defer q.Close() + + ms, err := storepb.MatchersToPromMatchers(request.Matchers...) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + lns, warnings, err := q.LabelValues(ctx, request.Label, &storage.LabelHints{Limit: int(request.Limit)}, ms...) + if err != nil { + return nil, status.Error(codes.Internal, err.Error()) + } + + return &storepb.LabelValuesResponse{ + Values: lns, + Warnings: warningsAsStrings(warnings), + }, nil +} + func zLabelsFromMetric(metric labels.Labels) []labelpb.ZLabel { zlabels := make([]labelpb.ZLabel, 0, metric.Len()) metric.Range(func(lbl labels.Label) { @@ -200,6 +395,33 @@ func zLabelsFromMetric(metric labels.Labels) []labelpb.ZLabel { return zlabels } +func warningsAsStrings(warns annotations.Annotations) []string { + errs := warns.AsErrors() + res := make([]string, len(errs)) + for i := range errs { + res[i] = errs[i].Error() + } + return res +} + +func zLabelSetsFromPromLabels(lss ...labels.Labels) []labelpb.ZLabelSet { + sets := make([]labelpb.ZLabelSet, 0, len(lss)) + for _, ls := range lss { + set := labelpb.ZLabelSet{ + Labels: make([]labelpb.ZLabel, 0, ls.Len()), + } + ls.Range(func(lbl labels.Label) { + set.Labels = append(set.Labels, labelpb.ZLabel{ + Name: lbl.Name, + Value: lbl.Value, + }) + }) + sets = append(sets, set) + } + + return sets +} + func samplesFromModel(samples []promql.FPoint) []prompb.Sample { result := make([]prompb.Sample, 0, len(samples)) for _, s := range samples { diff --git a/api/http/api.go b/api/http/api.go index 7a93c31..9db1765 100644 --- a/api/http/api.go +++ b/api/http/api.go @@ -9,7 +9,8 @@ import ( "github.com/prometheus/common/route" "github.com/prometheus/prometheus/promql" - "github.com/prometheus/prometheus/storage" + + "github.com/cloudflare/parquet-tsdb-poc/db" ) type apiConfig struct { @@ -24,7 +25,7 @@ func QueryOptions(opts ...QueryAPIOption) APIOption { } } -func NewAPI(queryable storage.Queryable, engine promql.QueryEngine, opts ...APIOption) http.Handler { +func NewAPI(db *db.DB, engine promql.QueryEngine, opts ...APIOption) http.Handler { cfg := &apiConfig{} for i := range opts { opts[i](cfg) @@ -33,7 +34,7 @@ func NewAPI(queryable storage.Queryable, engine promql.QueryEngine, opts ...APIO r := route.New() api := r.WithPrefix("/api/v1") - RegisterQueryV1(api, queryable, engine, cfg.queryAPIOpts...) + RegisterQueryV1(api, db, engine, cfg.queryAPIOpts...) return r } diff --git a/api/http/metrics.go b/api/http/metrics.go new file mode 100644 index 0000000..8f07ac7 --- /dev/null +++ b/api/http/metrics.go @@ -0,0 +1,30 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package http + +import ( + "errors" + + "github.com/prometheus/client_golang/prometheus" +) + +var ( + httpRequestsTotal = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "requests_total", + Help: "The total amount of http requests we answered", + }, []string{"code", "method", "path"}) + httpRequestsDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "requests_seconds", + Help: "The histogram of time spent answering http requests", + Buckets: prometheus.ExponentialBucketsRange(0.1, 30, 20), + }, []string{"code", "method", "path"}) +) + +func RegisterMetrics(reg prometheus.Registerer) error { + return errors.Join( + reg.Register(httpRequestsTotal), + reg.Register(httpRequestsDuration), + ) +} diff --git a/api/http/query.go b/api/http/query.go index f903d23..c35c7cc 100644 --- a/api/http/query.go +++ b/api/http/query.go @@ -6,14 +6,20 @@ package http import ( "context" - "encoding/json" "errors" "fmt" + "io" "math" "net/http" "strconv" "time" + "github.com/alecthomas/units" + jsoniter "github.com/json-iterator/go" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + _ "github.com/prometheus/prometheus/web/api/v1" // prometheus json codecs + "github.com/prometheus/common/model" "github.com/prometheus/common/route" "github.com/prometheus/prometheus/model/labels" @@ -26,16 +32,28 @@ import ( "go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp" "go.opentelemetry.io/otel/attribute" + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" ) type queryAPI struct { - engine promql.QueryEngine - queryable storage.Queryable + engine promql.QueryEngine + db *db.DB defaultLookback time.Duration defaultStep time.Duration defaultTimeout time.Duration + + concurrentQuerySemaphore *limits.Semaphore + + selectChunkBytesQuota units.Base2Bytes + selectRowCountQuota int64 + selectChunkPartitionMaxRange units.Base2Bytes + selectChunkPartitionMaxGap units.Base2Bytes + selectChunkPartitionMaxConcurrency int } type QueryAPIOption func(*queryAPI) @@ -58,38 +76,79 @@ func DefaultTimeout(s time.Duration) QueryAPIOption { } } -func withTracing(r *route.Router, path string, h http.HandlerFunc) { - tracedHandler := otelhttp.NewMiddleware(path)(h) +func ConcurrentQueryQuota(n int) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.concurrentQuerySemaphore = limits.NewSempahore(n) + } +} + +func SelectChunkBytesQuota(q units.Base2Bytes) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.selectChunkBytesQuota = q + } +} + +func SelectRowCountQuota(n int64) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.selectRowCountQuota = n + } +} + +func SelectChunkPartitionMaxRange(v units.Base2Bytes) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.selectChunkPartitionMaxRange = v + } +} + +func SelectChunkPartitionMaxGap(v units.Base2Bytes) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.selectChunkPartitionMaxGap = v + } +} + +func SelectChunkPartitionMaxConcurrency(n int) QueryAPIOption { + return func(qapi *queryAPI) { + qapi.selectChunkPartitionMaxConcurrency = n + } +} + +func withInstrumentation(r *route.Router, path string, h http.HandlerFunc) { + handler := otelhttp.NewMiddleware(path)(h) + handler = promhttp.InstrumentHandlerCounter(httpRequestsTotal.MustCurryWith(prometheus.Labels{"path": path}), handler) + handler = promhttp.InstrumentHandlerDuration(httpRequestsDuration.MustCurryWith(prometheus.Labels{"path": path}), handler) - r.Get(path, tracedHandler.ServeHTTP) - r.Post(path, tracedHandler.ServeHTTP) + r.Get(path, handler.ServeHTTP) + r.Post(path, handler.ServeHTTP) } -func RegisterQueryV1(r *route.Router, queryable storage.Queryable, engine promql.QueryEngine, opts ...QueryAPIOption) { +func RegisterQueryV1(r *route.Router, db *db.DB, engine promql.QueryEngine, opts ...QueryAPIOption) { qapi := &queryAPI{ engine: engine, - queryable: queryable, + db: db, defaultLookback: 5 * time.Minute, defaultStep: 30 * time.Second, defaultTimeout: 30 * time.Second, + + concurrentQuerySemaphore: limits.UnlimitedSemaphore(), } for i := range opts { opts[i](qapi) } - withTracing(r, "/query", qapi.query) - withTracing(r, "/query_range", qapi.queryRange) - withTracing(r, "/series", qapi.series) - withTracing(r, "/labels", qapi.labelNames) - withTracing(r, "/label/:name/values", qapi.labelValues) + withInstrumentation(r, "/query", qapi.query) + withInstrumentation(r, "/query_range", qapi.queryRange) + withInstrumentation(r, "/series", qapi.series) + withInstrumentation(r, "/labels", qapi.labelNames) + withInstrumentation(r, "/label/:name/values", qapi.labelValues) } const ( - errBadRequest = "bad_request" - errInternal = "internal" - errCanceled = "canceled" - errTimeout = "timeout" - errUnimplemented = "unimplemented" + errBadRequest = "bad_request" + errInternal = "internal" + errCanceled = "canceled" + errTimeout = "timeout" + errUnimplemented = "unimplemented" + errResourceExhausted = "resource_exhausted" statusSuccess = "success" statusError = "error" @@ -109,18 +168,22 @@ type errorResponse struct { Err error } +func encoder(w io.Writer) *jsoniter.Encoder { + return jsoniter.ConfigCompatibleWithStandardLibrary.NewEncoder(w) +} + func writeErrorResponse(w http.ResponseWriter, r errorResponse) { switch r.Typ { case errUnimplemented: w.WriteHeader(http.StatusNotFound) - case errBadRequest: + case errBadRequest, errResourceExhausted: w.WriteHeader(http.StatusBadRequest) case errInternal: w.WriteHeader(http.StatusInternalServerError) case errCanceled, errTimeout: w.WriteHeader(http.StatusRequestTimeout) } - json.NewEncoder(w).Encode(apiResponse{ + encoder(w).Encode(apiResponse{ Status: statusError, ErrorType: r.Typ, Error: r.Err.Error(), @@ -135,7 +198,7 @@ type queryResponse struct { func writeQueryResponse(w http.ResponseWriter, r *promql.Result) { w.WriteHeader(http.StatusOK) warns, infos := r.Warnings.AsStrings("", 0, 0) - json.NewEncoder(w).Encode(apiResponse{ + encoder(w).Encode(apiResponse{ Status: statusSuccess, Data: queryResponse{ ResultType: r.Value.Type(), @@ -149,7 +212,7 @@ func writeQueryResponse(w http.ResponseWriter, r *promql.Result) { func writeSeriesResponse(w http.ResponseWriter, series []labels.Labels, annos annotations.Annotations) { w.WriteHeader(http.StatusOK) warns, infos := annos.AsStrings("", 0, 0) - json.NewEncoder(w).Encode(apiResponse{ + encoder(w).Encode(apiResponse{ Status: statusSuccess, Data: series, Warnings: warns, @@ -160,7 +223,7 @@ func writeSeriesResponse(w http.ResponseWriter, series []labels.Labels, annos an func writeLabelsResponse(w http.ResponseWriter, values []string, annos annotations.Annotations) { w.WriteHeader(http.StatusOK) warns, infos := annos.AsStrings("", 0, 0) - json.NewEncoder(w).Encode(apiResponse{ + encoder(w).Encode(apiResponse{ Status: statusSuccess, Data: values, Warnings: warns, @@ -222,7 +285,7 @@ func parseQueryParam(r *http.Request) string { return r.FormValue("query") } -func parseMatchersParam(r *http.Request) ([][]*labels.Matcher, error) { +func parseMatchersParamForSeries(r *http.Request) ([][]*labels.Matcher, error) { matchers := r.Form["match[]"] if len(matchers) == 0 { @@ -245,6 +308,26 @@ OUTER: return matcherSets, nil } +func parseMatchersParamForLabels(r *http.Request) ([][]*labels.Matcher, error) { + matchers := r.Form["match[]"] + + matcherSets, err := parser.ParseMetricSelectors(matchers) + if err != nil { + return nil, err + } + +OUTER: + for _, ms := range matcherSets { + for _, lm := range ms { + if lm != nil && !lm.Matches("") { + continue OUTER + } + } + return nil, errors.New("match[] must contain at least one non-empty matcher") + } + return matcherSets, nil +} + func parseLimitParam(r *http.Request) (int, error) { s := r.FormValue("limit") if s == "" { @@ -266,6 +349,16 @@ func (qapi *queryAPI) queryOpts() promql.QueryOpts { return promql.NewPrometheusQueryOpts(false, qapi.defaultLookback) } +func (qapi *queryAPI) queryable() storage.Queryable { + return qapi.db.Queryable( + db.SelectChunkBytesQuota(qapi.selectChunkBytesQuota), + db.SelectRowCountQuota(qapi.selectRowCountQuota), + db.SelectChunkPartitionMaxRange(qapi.selectChunkPartitionMaxRange), + db.SelectChunkPartitionMaxGap(qapi.selectChunkPartitionMaxGap), + db.SelectChunkPartitionMaxConcurrency(qapi.selectChunkPartitionMaxConcurrency), + ) +} + func (qapi *queryAPI) query(w http.ResponseWriter, r *http.Request) { ctx := r.Context() span := tracing.SpanFromContext(ctx) @@ -294,7 +387,13 @@ func (qapi *queryAPI) query(w http.ResponseWriter, r *http.Request) { ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() - query, err := qapi.engine.NewInstantQuery(ctx, qapi.queryable, qapi.queryOpts(), q, t) + if err := qapi.concurrentQuerySemaphore.Reserve(ctx); err != nil { + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: fmt.Errorf("semaphore blocked: %s", err)}) + return + } + defer qapi.concurrentQuerySemaphore.Release() + + query, err := qapi.engine.NewInstantQuery(ctx, qapi.queryable(), qapi.queryOpts(), q, t) if err != nil { writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to create query: %s", err)}) return @@ -302,16 +401,30 @@ func (qapi *queryAPI) query(w http.ResponseWriter, r *http.Request) { defer query.Close() res := query.Exec(ctx) - if res.Err != nil { - switch res.Err.(type) { + if err := res.Err; err != nil { + // storage errors + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + if errors.Is(err, context.DeadlineExceeded) { + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: err}) + return + } + if errors.Is(err, context.Canceled) { + writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: err}) + return + } + // promql errors + switch err.(type) { case promql.ErrQueryCanceled: - writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: err}) case promql.ErrQueryTimeout: - writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: err}) case promql.ErrStorage: - writeErrorResponse(w, errorResponse{Typ: errInternal, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: err}) default: - writeErrorResponse(w, errorResponse{Typ: errInternal, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: err}) } return } @@ -359,7 +472,13 @@ func (qapi *queryAPI) queryRange(w http.ResponseWriter, r *http.Request) { ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() - query, err := qapi.engine.NewRangeQuery(ctx, qapi.queryable, qapi.queryOpts(), q, start, end, step) + if err := qapi.concurrentQuerySemaphore.Reserve(ctx); err != nil { + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: fmt.Errorf("semaphore blocked: %s", err)}) + return + } + defer qapi.concurrentQuerySemaphore.Release() + + query, err := qapi.engine.NewRangeQuery(ctx, qapi.queryable(), qapi.queryOpts(), q, start, end, step) if err != nil { writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to create query: %s", err)}) return @@ -367,16 +486,30 @@ func (qapi *queryAPI) queryRange(w http.ResponseWriter, r *http.Request) { defer query.Close() res := query.Exec(ctx) - if res.Err != nil { - switch res.Err.(type) { + if err := res.Err; err != nil { + // storage errors + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + if errors.Is(err, context.DeadlineExceeded) { + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: err}) + return + } + if errors.Is(err, context.Canceled) { + writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: err}) + return + } + // promql errors + switch err.(type) { case promql.ErrQueryCanceled: - writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errCanceled, Err: err}) case promql.ErrQueryTimeout: - writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errTimeout, Err: err}) case promql.ErrStorage: - writeErrorResponse(w, errorResponse{Typ: errInternal, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: err}) default: - writeErrorResponse(w, errorResponse{Typ: errInternal, Err: res.Err}) + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: err}) } return } @@ -407,9 +540,9 @@ func (qapi *queryAPI) series(w http.ResponseWriter, r *http.Request) { writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get limit: %s", err)}) return } - ms, err := parseMatchersParam(r) + ms, err := parseMatchersParamForSeries(r) if err != nil { - writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get labelmatchers: %s", err)}) + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get label matchers: %s", err)}) return } @@ -421,7 +554,7 @@ func (qapi *queryAPI) series(w http.ResponseWriter, r *http.Request) { ctx, cancel := context.WithTimeout(ctx, qapi.defaultTimeout) defer cancel() - q, err := qapi.queryable.Querier(timestamp.FromTime(start), timestamp.FromTime(end)) + q, err := qapi.queryable().Querier(timestamp.FromTime(start), timestamp.FromTime(end)) if err != nil { writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to create querier: %s", err)}) return @@ -429,8 +562,7 @@ func (qapi *queryAPI) series(w http.ResponseWriter, r *http.Request) { defer q.Close() var ( - series []labels.Labels - sets []storage.SeriesSet + sets []storage.SeriesSet ) hints := &storage.SelectHints{ @@ -444,22 +576,28 @@ func (qapi *queryAPI) series(w http.ResponseWriter, r *http.Request) { sets = append(sets, q.Select(ctx, false, hints, mset...)) } - set := storage.NewMergeSeriesSet(sets, storage.ChainedSeriesMerge) - warnings := set.Warnings() + series := make([]labels.Labels, 0) + + set := storage.NewMergeSeriesSet(sets, 0, storage.ChainedSeriesMerge) + annos := set.Warnings() for set.Next() { series = append(series, set.At().Labels()) if limit > 0 && len(series) > limit { series = series[:limit] - warnings.Add(errors.New("results truncated due to limit")) + annos.Add(warnings.ErrorTruncatedResponse) break } } if err := set.Err(); err != nil { + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to merge series: %s", err)}) return } - writeSeriesResponse(w, series, warnings) + writeSeriesResponse(w, series, annos) } func (qapi *queryAPI) labelValues(w http.ResponseWriter, r *http.Request) { @@ -476,11 +614,6 @@ func (qapi *queryAPI) labelValues(w http.ResponseWriter, r *http.Request) { writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("invalid label name: %q", name)}) } - // TODO: support more labels - if name != model.MetricNameLabel { - writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: errors.New("label values is only supported for the __name__ label")}) - } - start, err := parseTimeParam(r, "start", time.Now()) if err != nil { writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get start: %s", err)}) @@ -496,20 +629,22 @@ func (qapi *queryAPI) labelValues(w http.ResponseWriter, r *http.Request) { writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get limit: %s", err)}) return } - if len(r.Form["match[]"]) != 0 { - writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: errors.New("label values with matchers is not supported")}) + ms, err := parseMatchersParamForLabels(r) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get label matchers: %s", err)}) return } span.SetAttributes(attribute.String("label_values.start", start.String())) span.SetAttributes(attribute.String("label_values.end", end.String())) + span.SetAttributes(attribute.String("label_values.name", name)) span.SetAttributes(attribute.StringSlice("label_values.matchers", r.Form["match[]"])) span.SetAttributes(attribute.Int("label_values.limit", limit)) ctx, cancel := context.WithTimeout(ctx, qapi.defaultTimeout) defer cancel() - q, err := qapi.queryable.Querier(timestamp.FromTime(start), timestamp.FromTime(end)) + q, err := qapi.queryable().Querier(timestamp.FromTime(start), timestamp.FromTime(end)) if err != nil { writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to create querier: %s", err)}) return @@ -520,14 +655,131 @@ func (qapi *queryAPI) labelValues(w http.ResponseWriter, r *http.Request) { Limit: limit, } - labelValues, annos, err := q.LabelValues(ctx, name, hints) - if err != nil { - writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to query label values: %w", err)}) + var annos annotations.Annotations + + res := make([]string, 0) + if len(ms) > 0 { + for _, mset := range ms { + labelValues, lvannos, err := q.LabelValues(ctx, name, hints, mset...) + if err != nil { + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to query label values: %w", err)}) + return + } + annos = annos.Merge(lvannos) + res = append(res, labelValues...) + } + } else { + labelValues, lvannos, err := q.LabelValues(ctx, name, hints) + if err != nil { + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to query label values: %w", err)}) + return + } + annos = annos.Merge(lvannos) + res = append(res, labelValues...) + } + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) } - writeLabelsResponse(w, labelValues, annos) + writeLabelsResponse(w, res, annos) } -func (qapi *queryAPI) labelNames(w http.ResponseWriter, _ *http.Request) { - writeErrorResponse(w, errorResponse{Typ: errUnimplemented, Err: errors.New("unimplemented")}) +func (qapi *queryAPI) labelNames(w http.ResponseWriter, r *http.Request) { + ctx := r.Context() + span := tracing.SpanFromContext(ctx) + + if err := r.ParseForm(); err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to parse form data: %s", err)}) + return + } + + start, err := parseTimeParam(r, "start", time.Now()) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get start: %s", err)}) + return + } + end, err := parseTimeParam(r, "end", time.Now()) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get end: %s", err)}) + return + } + limit, err := parseLimitParam(r) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get limit: %s", err)}) + return + } + ms, err := parseMatchersParamForLabels(r) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errBadRequest, Err: fmt.Errorf("unable to get label matchers: %s", err)}) + return + } + + span.SetAttributes(attribute.String("label_names.start", start.String())) + span.SetAttributes(attribute.String("label_names.end", end.String())) + span.SetAttributes(attribute.StringSlice("label_names.matchers", r.Form["match[]"])) + span.SetAttributes(attribute.Int("label_names.limit", limit)) + + ctx, cancel := context.WithTimeout(ctx, qapi.defaultTimeout) + defer cancel() + + q, err := qapi.queryable().Querier(timestamp.FromTime(start), timestamp.FromTime(end)) + if err != nil { + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to create querier: %s", err)}) + return + } + defer q.Close() + + hints := &storage.LabelHints{ + Limit: limit, + } + + var annos annotations.Annotations + + res := make([]string, 0) + if len(ms) > 0 { + for _, mset := range ms { + labelNames, lnannos, err := q.LabelNames(ctx, hints, mset...) + if err != nil { + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to query label names: %w", err)}) + return + } + annos = annos.Merge(lnannos) + res = append(res, labelNames...) + } + } else { + labelNames, lnannos, err := q.LabelNames(ctx, hints) + if err != nil { + if limits.IsResourceExhausted(err) { + writeErrorResponse(w, errorResponse{Typ: errResourceExhausted, Err: err}) + return + } + writeErrorResponse(w, errorResponse{Typ: errInternal, Err: fmt.Errorf("unable to query label names: %w", err)}) + return + } + annos = annos.Merge(lnannos) + res = append(res, labelNames...) + } + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + + writeLabelsResponse(w, res, annos) } diff --git a/cmd/cmd.go b/cmd/cmd.go index 5e71270..362fd11 100644 --- a/cmd/cmd.go +++ b/cmd/cmd.go @@ -6,6 +6,7 @@ package main import ( "context" + "errors" "fmt" "log/slog" "os" @@ -13,11 +14,14 @@ import ( "syscall" "github.com/KimMachineGun/automemlimit/memlimit" - "github.com/hashicorp/go-multierror" "github.com/prometheus/client_golang/prometheus" "gopkg.in/alecthomas/kingpin.v2" + "github.com/cloudflare/parquet-tsdb-poc/api/grpc" + "github.com/cloudflare/parquet-tsdb-poc/api/http" + "github.com/cloudflare/parquet-tsdb-poc/convert" "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/locate" "github.com/cloudflare/parquet-tsdb-poc/search" ) @@ -29,7 +33,7 @@ var logLevelMap = map[string]slog.Level{ } func main() { - app := kingpin.New("parquet-tsdb-poc", "A POC for a TSDB in parquet.") + app := kingpin.New("parquet-gateway", "parquet metrics experiments") memratio := app.Flag("memlimit.ratio", "gomemlimit ratio").Default("0.9").Float() logLevel := app.Flag("logger.level", "log level").Default("INFO").Enum("DEBUG", "INFO", "WARN", "ERROR") @@ -70,7 +74,7 @@ func main() { switch parsed { case tsdbConvert.FullCommand(): log.Info("Running convert") - if err := tsdbConvertF(ctx, log); err != nil { + if err := tsdbConvertF(ctx, log, reg); err != nil { log.Error("Error converting tsdb block", slog.Any("err", err)) os.Exit(1) } @@ -88,11 +92,14 @@ func setupPrometheusRegistry() (*prometheus.Registry, error) { reg := prometheus.NewRegistry() registerer := prometheus.WrapRegistererWithPrefix("cf_metrics_", reg) - if err := multierror.Append( - nil, - db.RegisterMetrics(prometheus.WrapRegistererWithPrefix("db_", registerer)), + if err := errors.Join( + locate.RegisterMetrics(prometheus.WrapRegistererWithPrefix("locate_", registerer)), search.RegisterMetrics(prometheus.WrapRegistererWithPrefix("search_", registerer)), - ); err.ErrorOrNil() != nil { + convert.RegisterMetrics(prometheus.WrapRegistererWithPrefix("convert_", registerer)), + db.RegisterMetrics(prometheus.WrapRegistererWithPrefix("db_", registerer)), + http.RegisterMetrics(prometheus.WrapRegistererWithPrefix("http_", registerer)), + grpc.RegisterMetrics(prometheus.WrapRegistererWithPrefix("grpc_", registerer)), + ); err != nil { return nil, fmt.Errorf("unable to register metrics: %w", err) } return reg, nil diff --git a/cmd/config.go b/cmd/config.go index a4e85fa..9cdf309 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -8,9 +8,11 @@ import ( "context" "fmt" "log/slog" + "net/http" + "net/http/pprof" "strings" + "time" - "gopkg.in/alecthomas/kingpin.v2" "gopkg.in/yaml.v3" "go.opentelemetry.io/otel" @@ -20,10 +22,28 @@ import ( "go.opentelemetry.io/otel/sdk/trace" semconv "go.opentelemetry.io/otel/semconv/v1.21.0" + "github.com/alecthomas/units" + "github.com/oklog/run" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/thanos-io/objstore" "github.com/thanos-io/objstore/client" + "github.com/thanos-io/thanos/pkg/runutil" + + "github.com/cloudflare/parquet-tsdb-poc/locate" ) +func setupInterrupt(ctx context.Context, g *run.Group, log *slog.Logger) { + ctx, cancel := context.WithCancel(ctx) + g.Add(func() error { + <-ctx.Done() + log.Info("Canceling actors") + return nil + }, func(error) { + cancel() + }) +} + type bucketOpts struct { storage string prefix string @@ -41,57 +61,21 @@ type bucketOpts struct { retries int } -func (opts *bucketOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") - cmd.Flag("storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) - cmd.Flag("storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) - cmd.Flag("storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) - cmd.Flag("storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) - cmd.Flag("storage.s3.access_key", "access key for s3").Default("").Envar("STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) - cmd.Flag("storage.s3.secret_key", "secret key for s3").Default("").Envar("STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) - cmd.Flag("storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) - cmd.Flag("storage.retries", "how many retries to perform").Default("2").IntVar(&opts.retries) -} - -func (opts *bucketOpts) registerParquetFlags(cmd *kingpin.CmdClause) { - cmd.Flag("parquet.storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") - cmd.Flag("parquet.storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) - cmd.Flag("parquet.storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) - cmd.Flag("parquet.storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) - cmd.Flag("parquet.storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) - cmd.Flag("parquet.storage.s3.access_key", "access key for s3").Default("").Envar("PARQUET_STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) - cmd.Flag("parquet.storage.s3.secret_key", "secret key for s3").Default("").Envar("PARQUET_STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) - cmd.Flag("parquet.storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) - cmd.Flag("parquet.storage.retries", "how many retries to perform").Default("2").IntVar(&opts.retries) -} - -func (opts *bucketOpts) registerTSDBFlags(cmd *kingpin.CmdClause) { - cmd.Flag("tsdb.storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") - cmd.Flag("tsdb.storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) - cmd.Flag("tsdb.storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) - cmd.Flag("tsdb.storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) - cmd.Flag("tsdb.storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) - cmd.Flag("tsdb.storage.s3.access_key", "access key for s3").Default("").Envar("TSDB_STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) - cmd.Flag("tsdb.storage.s3.secret_key", "secret key for s3").Default("").Envar("TSDB_STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) - cmd.Flag("tsdb.storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) - cmd.Flag("tsdb.storage.retries", "how many retries to perform").Default("2").IntVar(&opts.retries) -} - func setupBucket(log *slog.Logger, opts bucketOpts) (objstore.Bucket, error) { - prov := client.ObjProvider(strings.ToUpper(opts.storage)) + prov := objstore.ObjProvider(strings.ToUpper(opts.storage)) cfg := client.BucketConfig{ Type: prov, Prefix: opts.prefix, } var subCfg any switch prov { - case client.FILESYSTEM: + case objstore.FILESYSTEM: subCfg = struct { Directory string `yaml:"directory"` }{ Directory: opts.filesystemDirectory, } - case client.S3: + case objstore.S3: subCfg = struct { Bucket string `yaml:"bucket"` Endpoint string `yaml:"endpoint"` @@ -129,7 +113,7 @@ type slogAdapter struct { log *slog.Logger } -func (s slogAdapter) Log(args ...interface{}) error { +func (s slogAdapter) Log(args ...any) error { s.log.Debug("", args...) return nil } @@ -144,13 +128,6 @@ type tracingOpts struct { samplingType string } -func (opts *tracingOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("tracing.exporter.type", "type of tracing exporter").Default("STDOUT").EnumVar(&opts.exporterType, "JAEGER", "STDOUT") - cmd.Flag("tracing.jaeger.endpoint", "endpoint to send traces, eg. https://example.com:4318/v1/traces").StringVar(&opts.jaegerEndpoint) - cmd.Flag("tracing.sampling.param", "sample of traces to send").Default("0.1").Float64Var(&opts.samplingParam) - cmd.Flag("tracing.sampling.type", "type of sampling").Default("PROBABILISTIC").EnumVar(&opts.samplingType, "PROBABILISTIC", "ALWAYS", "NEVER") -} - func setupTracing(ctx context.Context, opts tracingOpts) error { var ( exporter trace.SpanExporter @@ -199,3 +176,209 @@ func setupTracing(ctx context.Context, opts tracingOpts) error { otel.SetTracerProvider(tracerProvider) return nil } + +type apiOpts struct { + port int + + shutdownTimeout time.Duration +} + +func setupInternalAPI(g *run.Group, log *slog.Logger, reg *prometheus.Registry, opts apiOpts) { + mux := http.NewServeMux() + mux.HandleFunc("/debug/pprof/", pprof.Index) + mux.HandleFunc("/debug/pprof/profile", pprof.Profile) + mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + mux.Handle("/metrics", promhttp.HandlerFor(reg, promhttp.HandlerOpts{})) + + mux.HandleFunc("/-/healthy", func(w http.ResponseWriter, _ *http.Request) { + w.WriteHeader(http.StatusOK) + fmt.Fprintf(w, "OK") + }) + mux.HandleFunc("/-/ready", func(w http.ResponseWriter, _ *http.Request) { + w.WriteHeader(http.StatusOK) + fmt.Fprintf(w, "OK") + }) + + server := &http.Server{Addr: fmt.Sprintf(":%d", opts.port), Handler: mux} + g.Add(func() error { + log.Info("Serving internal api", slog.Int("port", opts.port)) + if err := server.ListenAndServe(); err != http.ErrServerClosed { + return err + } + return nil + }, func(error) { + log.Info("Shutting down internal api", slog.Int("port", opts.port)) + ctx, cancel := context.WithTimeout(context.Background(), opts.shutdownTimeout) + defer cancel() + + if err := server.Shutdown(ctx); err != nil { + log.Error("Error shutting down internal server", slog.Any("err", err)) + } + }) +} + +type discoveryOpts struct { + discoveryInterval time.Duration + discoveryConcurrency int +} + +func setupDiscovery(ctx context.Context, g *run.Group, log *slog.Logger, bkt objstore.Bucket, opts discoveryOpts) (*locate.Discoverer, error) { + discoverer := locate.NewDiscoverer(bkt, locate.MetaConcurrency(opts.discoveryConcurrency)) + + log.Info("Running initial discovery") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) + defer iterCancel() + if err := discoverer.Discover(iterCtx); err != nil { + return nil, fmt.Errorf("unable to run initial discovery: %w", err) + } + + ctx, cancel := context.WithCancel(context.Background()) + g.Add(func() error { + return runutil.Repeat(opts.discoveryInterval, ctx.Done(), func() error { + log.Debug("Running discovery") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) + defer iterCancel() + if err := discoverer.Discover(iterCtx); err != nil { + log.Warn("Unable to discover new blocks", slog.Any("err", err)) + } + return nil + }) + }, func(error) { + log.Info("Stopping discovery") + cancel() + }) + return discoverer, nil +} + +type tsdbDiscoveryOpts struct { + discoveryInterval time.Duration + discoveryConcurrency int + discoveryMinBlockAge time.Duration + + externalLabelMatchers matcherSlice +} + +func setupTSDBDiscovery(ctx context.Context, g *run.Group, log *slog.Logger, bkt objstore.Bucket, opts tsdbDiscoveryOpts) (*locate.TSDBDiscoverer, error) { + discoverer := locate.NewTSDBDiscoverer( + bkt, + locate.TSDBMetaConcurrency(opts.discoveryConcurrency), + locate.TSDBMinBlockAge(opts.discoveryMinBlockAge), + locate.TSDBMatchExternalLabels(opts.externalLabelMatchers...), + ) + + log.Info("Running initial tsdb discovery") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) + defer iterCancel() + if err := discoverer.Discover(iterCtx); err != nil { + return nil, fmt.Errorf("unable to run initial discovery: %w", err) + } + + ctx, cancel := context.WithCancel(context.Background()) + g.Add(func() error { + return runutil.Repeat(opts.discoveryInterval, ctx.Done(), func() error { + log.Debug("Running tsdb discovery") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) + defer iterCancel() + if err := discoverer.Discover(iterCtx); err != nil { + log.Warn("Unable to discover new tsdb blocks", slog.Any("err", err)) + } + return nil + }) + }, func(error) { + log.Info("Stopping tsdb discovery") + cancel() + }) + return discoverer, nil +} + +type syncerOpts struct { + syncerInterval time.Duration + syncerConcurrency int + syncerReadBufferSize units.Base2Bytes + syncerLabelFilesDir string + + filterType string + filterThanosBackfillEndpoint string + filterThanosBackfillUpdateInterval time.Duration + filterThanosBackfillOverlap time.Duration +} + +func setupMetaFilter(ctx context.Context, g *run.Group, log *slog.Logger, opts syncerOpts) (locate.MetaFilter, error) { + switch opts.filterType { + case "all-metas": + return locate.AllMetasMetaFilter, nil + case "thanos-backfill": + thanosBackfillMetaFilter := locate.NewThanosBackfillMetaFilter(opts.filterThanosBackfillEndpoint, opts.filterThanosBackfillOverlap) + + log.Info("Initializing thanos-backfill meta filter") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.filterThanosBackfillUpdateInterval) + defer iterCancel() + if err := thanosBackfillMetaFilter.Update(iterCtx); err != nil { + return nil, fmt.Errorf("unable to initialize thanos-backfill meta filter: %w", err) + } + + ctx, cancel := context.WithCancel(context.Background()) + g.Add(func() error { + return runutil.Repeat(opts.filterThanosBackfillUpdateInterval, ctx.Done(), func() error { + log.Debug("Updating thanos-backfill meta filter") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.filterThanosBackfillUpdateInterval) + defer iterCancel() + if err := thanosBackfillMetaFilter.Update(iterCtx); err != nil { + log.Warn("Unable to update thanos-backfill meta filter", slog.Any("err", err)) + } + return nil + }) + }, func(error) { + log.Info("Stopping thanos-backfill meta filter updates") + cancel() + }) + return thanosBackfillMetaFilter, nil + default: + return nil, fmt.Errorf("unknown meta filter type: %s", opts.filterType) + } +} + +func setupSyncer(ctx context.Context, g *run.Group, log *slog.Logger, bkt objstore.Bucket, discoverer *locate.Discoverer, metaFilter locate.MetaFilter, opts syncerOpts) (*locate.Syncer, error) { + syncer := locate.NewSyncer( + bkt, + locate.FilterMetas(metaFilter), + locate.BlockConcurrency(opts.syncerConcurrency), + locate.BlockOptions( + locate.ReadBufferSize(opts.syncerReadBufferSize), + locate.LabelFilesDir(opts.syncerLabelFilesDir), + ), + ) + + log.Info("Running initial sync") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.syncerInterval) + defer iterCancel() + if err := syncer.Sync(iterCtx, discoverer.Metas()); err != nil { + return nil, fmt.Errorf("unable to run initial sync: %w", err) + } + + ctx, cancel := context.WithCancel(context.Background()) + g.Add(func() error { + return runutil.Repeat(opts.syncerInterval, ctx.Done(), func() error { + log.Debug("Running sync") + + iterCtx, iterCancel := context.WithTimeout(ctx, opts.syncerInterval) + defer iterCancel() + if err := syncer.Sync(iterCtx, discoverer.Metas()); err != nil { + log.Warn("Unable to sync new blocks", slog.Any("err", err)) + } + return nil + }) + }, func(error) { + log.Info("Stopping syncer") + cancel() + }) + return syncer, nil +} diff --git a/cmd/convert.go b/cmd/convert.go index 0690e6a..b144049 100644 --- a/cmd/convert.go +++ b/cmd/convert.go @@ -6,51 +6,128 @@ package main import ( "context" - "encoding/json" "fmt" "io" "log/slog" + "os" "path/filepath" - "runtime" "sort" "strings" "time" + "github.com/oklog/run" "github.com/parquet-go/parquet-go" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/runutil" + "golang.org/x/sync/errgroup" "gopkg.in/alecthomas/kingpin.v2" "github.com/cloudflare/parquet-tsdb-poc/convert" - "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/locate" ) type convertOpts struct { - parquetBucket bucketOpts - tsdbBucket bucketOpts - conversion conversionOpts + parquetBucket bucketOpts + tsdbBucket bucketOpts + parquetDiscover discoveryOpts + tsdbDiscover tsdbDiscoveryOpts + conversion conversionOpts + internalAPI apiOpts +} + +type conversionOpts struct { + runInterval time.Duration + runTimeout time.Duration + retryInterval time.Duration + + gracePeriod time.Duration + recompress bool + sortLabels []string + rowGroupSize int + rowGroupCount int + downloadConcurrency int + encodingConcurrency int + + tempDir string } func (opts *convertOpts) registerFlags(cmd *kingpin.CmdClause) { opts.conversion.registerFlags(cmd) - opts.parquetBucket.registerParquetFlags(cmd) - opts.tsdbBucket.registerTSDBFlags(cmd) + opts.parquetBucket.registerConvertParquetFlags(cmd) + opts.tsdbBucket.registerConvertTSDBFlags(cmd) + opts.parquetDiscover.registerConvertParquetFlags(cmd) + opts.tsdbDiscover.registerConvertTSDBFlags(cmd) + opts.internalAPI.registerConvertFlags(cmd) +} + +func (opts *conversionOpts) registerFlags(cmd *kingpin.CmdClause) { + cmd.Flag("convert.run-interval", "interval to run conversion on").Default("1h").DurationVar(&opts.runInterval) + cmd.Flag("convert.run-timeout", "timeout for a single conversion step").Default("24h").DurationVar(&opts.runTimeout) + cmd.Flag("convert.retry-interval", "interval to retry a single conversion after an error").Default("1m").DurationVar(&opts.retryInterval) + cmd.Flag("convert.tempdir", "directory for temporary state").Default(os.TempDir()).StringVar(&opts.tempDir) + cmd.Flag("convert.recompress", "recompress chunks").Default("true").BoolVar(&opts.recompress) + cmd.Flag("convert.grace-period", "dont convert for dates younger than this").Default("48h").DurationVar(&opts.gracePeriod) + cmd.Flag("convert.rowgroup.size", "size of rowgroups").Default("1_000_000").IntVar(&opts.rowGroupSize) + cmd.Flag("convert.rowgroup.count", "rowgroups per shard").Default("6").IntVar(&opts.rowGroupCount) + cmd.Flag("convert.sorting.label", "label to sort by").Default("__name__").StringsVar(&opts.sortLabels) + cmd.Flag("convert.download.concurrency", "concurrency for downloading tsdb blocks").Default("4").IntVar(&opts.downloadConcurrency) + cmd.Flag("convert.encoding.concurrency", "concurrency for encoding chunks").Default("4").IntVar(&opts.encodingConcurrency) +} + +func (opts *bucketOpts) registerConvertParquetFlags(cmd *kingpin.CmdClause) { + cmd.Flag("parquet.storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") + cmd.Flag("parquet.storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) + cmd.Flag("parquet.storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) + cmd.Flag("parquet.storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) + cmd.Flag("parquet.storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) + cmd.Flag("parquet.storage.s3.access_key", "access key for s3").Default("").Envar("PARQUET_STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) + cmd.Flag("parquet.storage.s3.secret_key", "secret key for s3").Default("").Envar("PARQUET_STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) + cmd.Flag("parquet.storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) } -func registerConvertApp(app *kingpin.Application) (*kingpin.CmdClause, func(ctx context.Context, log *slog.Logger) error) { +func (opts *bucketOpts) registerConvertTSDBFlags(cmd *kingpin.CmdClause) { + cmd.Flag("tsdb.storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") + cmd.Flag("tsdb.storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) + cmd.Flag("tsdb.storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) + cmd.Flag("tsdb.storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) + cmd.Flag("tsdb.storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) + cmd.Flag("tsdb.storage.s3.access_key", "access key for s3").Default("").Envar("TSDB_STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) + cmd.Flag("tsdb.storage.s3.secret_key", "secret key for s3").Default("").Envar("TSDB_STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) + cmd.Flag("tsdb.storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) +} + +func (opts *discoveryOpts) registerConvertParquetFlags(cmd *kingpin.CmdClause) { + cmd.Flag("parquet.discovery.interval", "interval to discover blocks").Default("30m").DurationVar(&opts.discoveryInterval) + cmd.Flag("parquet.discovery.concurrency", "concurrency for loading metadata").Default("1").IntVar(&opts.discoveryConcurrency) +} + +func (opts *tsdbDiscoveryOpts) registerConvertTSDBFlags(cmd *kingpin.CmdClause) { + cmd.Flag("tsdb.discovery.interval", "interval to discover blocks").Default("30m").DurationVar(&opts.discoveryInterval) + cmd.Flag("tsdb.discovery.concurrency", "concurrency for loading metadata").Default("1").IntVar(&opts.discoveryConcurrency) + cmd.Flag("tsdb.discovery.min-block-age", "blocks that have metrics that are youner then this wont be loaded").Default("0s").DurationVar(&opts.discoveryMinBlockAge) + MatchersVar(cmd.Flag("tsdb.discovery.select-external-labels", "only external labels matching this selector will be discovered").PlaceHolder("SELECTOR"), &opts.externalLabelMatchers) +} + +func (opts *apiOpts) registerConvertFlags(cmd *kingpin.CmdClause) { + cmd.Flag("http.internal.port", "port to host query api").Default("6060").IntVar(&opts.port) + cmd.Flag("http.internal.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) +} + +func registerConvertApp(app *kingpin.Application) (*kingpin.CmdClause, func(context.Context, *slog.Logger, *prometheus.Registry) error) { cmd := app.Command("convert", "convert TSDB Block to parquet file") var opts convertOpts opts.registerFlags(cmd) - return cmd, func(ctx context.Context, log *slog.Logger) error { - blkDir := filepath.Join(opts.conversion.tempDir, "blocks") + return cmd, func(ctx context.Context, log *slog.Logger, reg *prometheus.Registry) error { + var g run.Group - start, end, err := getStartEnd(opts.conversion) - if err != nil { - return fmt.Errorf("unable to get start, end: %s", err) - } + setupInterrupt(ctx, &g, log) + setupInternalAPI(&g, log, reg, opts.internalAPI) tsdbBkt, err := setupBucket(log, opts.tsdbBucket) if err != nil { @@ -61,146 +138,233 @@ func registerConvertApp(app *kingpin.Application) (*kingpin.CmdClause, func(ctx return fmt.Errorf("unable to setup parquet bucket: %s", err) } - log.Info("Sorting by", "labels", opts.conversion.sortLabels) - - // TODO: this is kinda horrible logic here that is not reentrant or robust against errors - // But for sake of getting started we can use it to convert the first blocks and then iterate - - log.Info("Fetching metas", "start", start, "end", end) - metas, err := fetchTSDBMetas(ctx, tsdbBkt, start, end) + tsdbDiscoverer, err := setupTSDBDiscovery(ctx, &g, log, tsdbBkt, opts.tsdbDiscover) if err != nil { - return fmt.Errorf("unable to fetch tsdb metas: %s", err) + return fmt.Errorf("unable to setup tsdb discovery: %s", err) } - - log.Info("Downloading blocks", "metas", metas) - blocks, err := downloadBlocks(ctx, tsdbBkt, metas, blkDir) + parquetDiscoverer, err := setupDiscovery(ctx, &g, log, parquetBkt, opts.parquetDiscover) if err != nil { - return fmt.Errorf("unable to download blocks: %s", err) + return fmt.Errorf("unable to setup parquet discovery: %s", err) } - for next := start; next != end; next = next.AddDate(0, 0, 1) { - log.Info("Converting next parquet file", "day", next) - - candidates := overlappingBlocks(blocks, next) - if len(candidates) == 0 { - continue - } - convOpts := []convert.ConvertOption{ - convert.SortBufSize(opts.conversion.sortBufSize), - convert.SortBy(opts.conversion.sortLabels), - convert.BufferPool(parquet.NewFileBufferPool(opts.conversion.tempDir, "convert-*")), - } - if err := convert.ConvertTSDBBlock(ctx, parquetBkt, next, candidates, convOpts...); err != nil { - return fmt.Errorf("unable to convert blocks for date %q: %s", next, err) - } - } - return nil + ctx, cancel := context.WithCancel(context.Background()) + g.Add(func() error { + log.Info("Starting conversion", "sort_by", opts.conversion.sortLabels) + return runutil.Repeat(opts.conversion.runInterval, ctx.Done(), func() error { + iterCtx, iterCancel := context.WithTimeout(ctx, opts.conversion.runTimeout) + defer iterCancel() + + if err := runutil.Retry(opts.conversion.retryInterval, iterCtx.Done(), func() error { + // Sync parquet files once here so we have the latest view + log.Info("Discovering parquet blocks before conversion") + if err := parquetDiscoverer.Discover(iterCtx); err != nil { + log.Error("Unable to discover parquet blocks", "error", err) + return err + } + log.Info("Converting next blocks", "sort_by", opts.conversion.sortLabels) + if err := advanceConversion(iterCtx, log, tsdbBkt, parquetBkt, tsdbDiscoverer, parquetDiscoverer, opts.conversion); err != nil { + log.Error("Unable to convert blocks", "error", err) + return err + } + return nil + }); err != nil { + log.Warn("Error during conversion", slog.Any("err", err)) + return nil + } + return nil + }) + }, func(error) { + log.Info("Stopping conversion") + cancel() + }) + return g.Run() } } -type conversionOpts struct { - sortBufSize int - sortLabels []string - - tempDir string - - start, end string -} +func advanceConversion( + ctx context.Context, + log *slog.Logger, + tsdbBkt objstore.Bucket, + parquetBkt objstore.Bucket, + tsdbDiscoverer *locate.TSDBDiscoverer, + parquetDiscoverer *locate.Discoverer, + opts conversionOpts, +) error { + blkDir := filepath.Join(opts.tempDir, ".blocks") + bufferDir := filepath.Join(opts.tempDir, ".buffers") + + log.Info("Cleaning up previous state", "block_directory", blkDir, "buffer_directory", bufferDir) + if err := cleanupDirectory(blkDir); err != nil { + return fmt.Errorf("unable to clean up block directory: %w", err) + } + if err := cleanupDirectory(bufferDir); err != nil { + return fmt.Errorf("unable to clean up buffer directory: %w", err) + } -func (opts *conversionOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("convert.start", "timestamp of the first parquet block to convert (rounded to start of day)").StringVar(&opts.start) - cmd.Flag("convert.end", "timestamp of the last parquet block to convert(rounded to start of day)").StringVar(&opts.end) - cmd.Flag("convert.tempdir", "directory for temporary state").StringVar(&opts.tempDir) - cmd.Flag("convert.sortbuf", "size of sorting buffer").Default("64_000").IntVar(&opts.sortBufSize) - cmd.Flag("convert.sorting.label", "label to sort by").Default("__name__").StringsVar(&opts.sortLabels) -} + parquetMetas := parquetDiscoverer.Metas() + tsdbMetas := tsdbDiscoverer.Metas() -func getStartEnd(opts conversionOpts) (time.Time, time.Time, error) { - from, err := time.Parse(time.RFC3339, opts.start) - if err != nil { - return time.Time{}, time.Time{}, fmt.Errorf("unable to parse start: %w", err) + plan, ok := convert.NewPlanner(time.Now().Add(-opts.gracePeriod)).Plan(tsdbMetas, parquetMetas) + if !ok { + log.Info("Nothing to do") + return nil } - to, err := time.Parse(time.RFC3339, opts.end) + log.Info("Converting dates", "dates", plan.ConvertForDates) + + log.Info("Opening blocks", "ulids", ulidsFromMetas(plan.Download)) + tsdbBlocks, err := downloadedBlocks(ctx, tsdbBkt, plan.Download, blkDir, opts) + defer func() { + for _, blk := range tsdbBlocks { + if cErr := blk.(io.Closer).Close(); cErr != nil { + log.Warn("Unable to close block", "block", blk.Meta().ULID, "err", cErr) + } + } + }() if err != nil { - return time.Time{}, time.Time{}, fmt.Errorf("unable to parse end: %w", err) + return fmt.Errorf("unable to download tsdb blocks: %w", err) } - return util.BeginOfDay(from), util.BeginOfDay(to), nil -} -func fetchTSDBMetas(ctx context.Context, bkt objstore.BucketReader, from, to time.Time) ([]tsdb.BlockMeta, error) { - metas := make([]tsdb.BlockMeta, 0) - err := bkt.Iter(ctx, "", func(name string) error { - split := strings.Split(name, "/") - f := split[1] - if f != "meta.json" { - return nil + for _, next := range plan.ConvertForDates { + log.Info("Converting next parquet file", "day", next) + + candidates := overlappingBlocks(tsdbBlocks, next) + if len(candidates) == 0 { + continue } - content, err := bkt.Get(ctx, name) - if err != nil { - return err + convOpts := []convert.ConvertOption{ + convert.SortBy(opts.sortLabels...), + convert.RowGroupSize(opts.rowGroupSize), + convert.RowGroupCount(opts.rowGroupCount), + convert.EncodingConcurrency(opts.encodingConcurrency), + convert.ChunkBufferPool(parquet.NewFileBufferPool(bufferDir, "chunkbuf-*")), } - defer content.Close() - - var m tsdb.BlockMeta - if err := json.NewDecoder(content).Decode(&m); err != nil { - return err + if err := convert.ConvertTSDBBlock(ctx, parquetBkt, next, candidates, convOpts...); err != nil { + return fmt.Errorf("unable to convert blocks for date %q: %s", next, err) } - - metas = append(metas, m) - - return nil - }, objstore.WithRecursiveIter()) - if err != nil { - return nil, fmt.Errorf("unable to fetch metas: %w", err) } + return nil +} - startMillis := util.BeginOfDay(from).UnixMilli() - endMillis := util.EndOfDay(to).UnixMilli() +func cleanupDirectory(dir string) error { + if err := os.RemoveAll(dir); err != nil { + return fmt.Errorf("unable to delete directory: %w", err) + } + if err := os.MkdirAll(dir, os.ModePerm); err != nil { + return fmt.Errorf("unable to recreate block directory: %w", err) + } + if _, err := os.Stat(dir); err != nil { + return fmt.Errorf("unable to stat block directory: %w", err) + } + return nil +} - res := make([]tsdb.BlockMeta, 0) - for _, m := range metas { - if endMillis >= m.MinTime && startMillis <= m.MaxTime { +func overlappingBlocks(blocks []convert.Convertable, date time.Time) []convert.Convertable { + res := make([]convert.Convertable, 0) + for _, m := range blocks { + if date.AddDate(0, 0, 1).UnixMilli() >= m.Meta().MinTime && date.UnixMilli() <= m.Meta().MaxTime { res = append(res, m) } } sort.Slice(res, func(i, j int) bool { - return res[i].MinTime < res[j].MinTime + return res[i].Meta().MaxTime <= res[j].Meta().MaxTime }) - - return res, nil + return res } -func overlappingBlocks(blocks []*tsdb.Block, date time.Time) []convert.Convertible { - dateStartMillis := date.UnixMilli() - dateEndMillis := date.AddDate(0, 0, 1).UnixMilli() - - res := make([]convert.Convertible, 0) - for _, blk := range blocks { - m := blk.Meta() - if dateEndMillis >= m.MinTime && dateStartMillis <= m.MaxTime { - res = append(res, blk) - } +func ulidsFromMetas(metas []metadata.Meta) []string { + res := make([]string, len(metas)) + for i := range metas { + res[i] = metas[i].ULID.String() } return res } -func downloadBlocks(ctx context.Context, bkt objstore.BucketReader, metas []tsdb.BlockMeta, blkDir string) ([]*tsdb.Block, error) { - logAdapter := slogAdapter{log: slog.New(slog.NewJSONHandler(io.Discard, nil))} +func downloadedBlocks(ctx context.Context, bkt objstore.BucketReader, metas []metadata.Meta, blkDir string, opts conversionOpts) ([]convert.Convertable, error) { + slogger := slog.New(slog.NewJSONHandler(io.Discard, nil)) - opts := []objstore.DownloadOption{objstore.WithFetchConcurrency(runtime.GOMAXPROCS(0))} - res := make([]*tsdb.Block, 0) + res := make([]convert.Convertable, 0) for _, m := range metas { src := m.ULID.String() dst := filepath.Join(blkDir, src) - if err := objstore.DownloadDir(ctx, logAdapter, bkt, src, src, dst, opts...); err != nil { - return nil, fmt.Errorf("unable to download %q: %s", src, err) + if err := runutil.Retry(5*time.Second, ctx.Done(), func() error { + return downloadBlock(ctx, bkt, m, blkDir, opts) + }); err != nil { + return res, fmt.Errorf("unable to download block %q: %w", src, err) } - blk, err := tsdb.OpenBlock(logAdapter, dst, chunkenc.NewPool()) + blk, err := tsdb.OpenBlock(slogger, dst, chunkenc.NewPool(), tsdb.DefaultPostingsDecoderFactory) if err != nil { - return nil, fmt.Errorf("unable to open block %q: %s", src, err) + return res, fmt.Errorf("unable to open block %q: %w", m.ULID, err) } res = append(res, blk) } return res, nil } + +func downloadBlock(ctx context.Context, bkt objstore.BucketReader, meta metadata.Meta, blkDir string, opts conversionOpts) error { + src := meta.ULID.String() + dst := filepath.Join(blkDir, src) + + fmap := make(map[string]metadata.File, len(meta.Thanos.Files)) + for _, fl := range meta.Thanos.Files { + if fl.SizeBytes == 0 || fl.RelPath == "" { + continue + } + fmap[fl.RelPath] = fl + } + + // order is not guaranteed in "Iter" so we need to create directory structure beforehand + if err := os.MkdirAll(dst, 0750); err != nil { + return fmt.Errorf("unable to create block directory: %w", err) + } + if err := os.MkdirAll(filepath.Join(dst, "chunks"), 0750); err != nil { + return fmt.Errorf("unable to create chunks directory: %w", err) + } + + // we reimplement download dir from objstore to skip the cleanup part on partial downloads + g, ctx := errgroup.WithContext(ctx) + g.SetLimit(opts.downloadConcurrency) + + err := bkt.Iter(ctx, src, func(name string) error { + g.Go(func() error { + dst := filepath.Join(dst, strings.TrimPrefix(name, src)) + if strings.HasSuffix(name, objstore.DirDelim) { + return nil + } + // In case the previous upload failed we dont download the files that have the correct size. + // Size is not the best indicator, but its good enough. ideally we would want a hash but we + // dont write those currently. + // If the file was corrupted, then opening the block will very likely fail anyway. + if stat, err := os.Stat(dst); err == nil { + if known, ok := fmap[strings.TrimPrefix(name, src+objstore.DirDelim)]; ok { + if stat.Size() == known.SizeBytes && stat.Size() != 0 { + return nil + } + } + } + + rc, err := bkt.Get(ctx, name) + if err != nil { + return fmt.Errorf("unable to get file %q: %w", name, err) + } + defer rc.Close() + + f, err := os.Create(dst) + if err != nil { + return fmt.Errorf("unable to create file %q: %w", dst, err) + } + if _, err := io.Copy(f, rc); err != nil { + return fmt.Errorf("unable to copy file %q: %w", dst, err) + } + return nil + }) + return nil + }, objstore.WithRecursiveIter()) + if err != nil { + return fmt.Errorf("unable to iter bucket: %w", err) + } + if err := g.Wait(); err != nil { + return fmt.Errorf("unable to download directory: %w", err) + } + return nil +} diff --git a/cmd/flags.go b/cmd/flags.go new file mode 100644 index 0000000..45b6062 --- /dev/null +++ b/cmd/flags.go @@ -0,0 +1,41 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package main + +import ( + "strings" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" + "gopkg.in/alecthomas/kingpin.v2" +) + +type matcherSlice []*labels.Matcher + +func (a *matcherSlice) Set(value string) error { + matchers, err := parser.ParseMetricSelector(value) + if err != nil { + return err + } + *a = append(*a, matchers...) + + return nil +} + +func (a *matcherSlice) String() string { + var b strings.Builder + for i, m := range *a { + b.WriteString(m.String()) + if i != len(*a) { + b.WriteString(",") + } + } + return b.String() +} + +func MatchersVar(flags *kingpin.FlagClause, target *matcherSlice) { + flags.SetValue((*matcherSlice)(target)) + return +} diff --git a/cmd/serve.go b/cmd/serve.go index 554d3c5..012a781 100644 --- a/cmd/serve.go +++ b/cmd/serve.go @@ -11,26 +11,27 @@ import ( "math" "net" "net/http" - "net/http/pprof" "time" "github.com/alecthomas/units" "github.com/oklog/run" + "gopkg.in/alecthomas/kingpin.v2" + "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" - "google.golang.org/grpc" - "google.golang.org/grpc/reflection" - "gopkg.in/alecthomas/kingpin.v2" - "github.com/thanos-io/objstore" "github.com/thanos-io/promql-engine/engine" "github.com/thanos-io/thanos/pkg/api/query/querypb" "github.com/thanos-io/thanos/pkg/info/infopb" - "github.com/thanos-io/thanos/pkg/runutil" + "github.com/thanos-io/thanos/pkg/store/storepb" _ "github.com/mostynb/go-grpc-compression/snappy" + "google.golang.org/grpc" + "google.golang.org/grpc/reflection" + + "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc" + "go.opentelemetry.io/contrib/propagators/autoprop" cfgrpc "github.com/cloudflare/parquet-tsdb-poc/api/grpc" cfhttp "github.com/cloudflare/parquet-tsdb-poc/api/http" @@ -38,24 +39,91 @@ import ( ) type serveOpts struct { - block blockOpts bucket bucketOpts tracing tracingOpts + discovery discoveryOpts + syncer syncerOpts + query queryOpts - promAPI promAPIOpts - thanosAPI thanosAPIOpts - internalAPI internalAPIOpts + promAPI apiOpts + thanosAPI apiOpts + internalAPI apiOpts } func (opts *serveOpts) registerFlags(cmd *kingpin.CmdClause) { - opts.block.registerFlags(cmd) - opts.bucket.registerFlags(cmd) - opts.tracing.registerFlags(cmd) - opts.query.registerFlags(cmd) - opts.promAPI.registerFlags(cmd) - opts.thanosAPI.registerFlags(cmd) - opts.internalAPI.registerFlags(cmd) + opts.bucket.registerServeFlags(cmd) + opts.tracing.registerServeFlags(cmd) + opts.discovery.registerServeFlags(cmd) + opts.syncer.registerServeFlags(cmd) + opts.query.registerServeFlags(cmd) + opts.promAPI.registerServePrometheusAPIFlags(cmd) + opts.thanosAPI.registerServeThanosAPIFlags(cmd) + opts.internalAPI.registerServeInternalAPIFlags(cmd) +} + +func (opts *bucketOpts) registerServeFlags(cmd *kingpin.CmdClause) { + cmd.Flag("storage.type", "type of storage").Default("filesystem").EnumVar(&opts.storage, "filesystem", "s3") + cmd.Flag("storage.prefix", "prefix for the storage").Default("").StringVar(&opts.prefix) + cmd.Flag("storage.filesystem.directory", "directory for filesystem").Default(".data").StringVar(&opts.filesystemDirectory) + cmd.Flag("storage.s3.bucket", "bucket for s3").Default("").StringVar(&opts.s3Bucket) + cmd.Flag("storage.s3.endpoint", "endpoint for s3").Default("").StringVar(&opts.s3Endpoint) + cmd.Flag("storage.s3.access_key", "access key for s3").Default("").Envar("STORAGE_S3_ACCESS_KEY").StringVar(&opts.s3AccessKey) + cmd.Flag("storage.s3.secret_key", "secret key for s3").Default("").Envar("STORAGE_S3_SECRET_KEY").StringVar(&opts.s3SecretKey) + cmd.Flag("storage.s3.insecure", "use http").Default("false").BoolVar(&opts.s3Insecure) +} + +func (opts *tracingOpts) registerServeFlags(cmd *kingpin.CmdClause) { + cmd.Flag("tracing.exporter.type", "type of tracing exporter").Default("STDOUT").EnumVar(&opts.exporterType, "JAEGER", "STDOUT") + cmd.Flag("tracing.jaeger.endpoint", "endpoint to send traces, eg. https://example.com:4318/v1/traces").StringVar(&opts.jaegerEndpoint) + cmd.Flag("tracing.sampling.param", "sample of traces to send").Default("0.1").Float64Var(&opts.samplingParam) + cmd.Flag("tracing.sampling.type", "type of sampling").Default("PROBABILISTIC").EnumVar(&opts.samplingType, "PROBABILISTIC", "ALWAYS", "NEVER") +} + +func (opts *discoveryOpts) registerServeFlags(cmd *kingpin.CmdClause) { + cmd.Flag("block.discovery.interval", "interval to discover blocks").Default("1m").DurationVar(&opts.discoveryInterval) + cmd.Flag("block.discovery.concurrency", "concurrency for loading metadata").Default("1").IntVar(&opts.discoveryConcurrency) +} + +func (opts *syncerOpts) registerServeFlags(cmd *kingpin.CmdClause) { + cmd.Flag("block.syncer.interval", "interval to sync blocks").Default("1m").DurationVar(&opts.syncerInterval) + cmd.Flag("block.syncer.concurrency", "concurrency for loading blocks").Default("1").IntVar(&opts.syncerConcurrency) + cmd.Flag("block.syncer.read-buffer-size", "read buffer size for blocks").Default("2MiB").BytesVar(&opts.syncerReadBufferSize) + cmd.Flag("block.filter.type", "").Default("all-metas").EnumVar(&opts.filterType, "thanos-backfill", "all-metas") + cmd.Flag("block.filter.thanos-backfill.endpoint", "endpoint to ignore for backfill").StringVar(&opts.filterThanosBackfillEndpoint) + cmd.Flag("block.filter.thanos-backfill.interval", "interval to update thanos-backfill timerange").Default("1m").DurationVar(&opts.filterThanosBackfillUpdateInterval) + cmd.Flag("block.filter.thanos-backfill.overlap", "overlap interval to leave for backfill").Default("24h").DurationVar(&opts.filterThanosBackfillOverlap) +} + +func (opts *queryOpts) registerServeFlags(cmd *kingpin.CmdClause) { + // We need to initialize the externalLabels map here + opts.externalLabels = make(map[string]string) + + cmd.Flag("query.step", "default step for range queries").Default("30s").DurationVar(&opts.defaultStep) + cmd.Flag("query.lookback", "default lookback for queries").Default("5m").DurationVar(&opts.defaultLookback) + cmd.Flag("query.timeout", "default timeout for queries").Default("30s").DurationVar(&opts.defaultTimeout) + cmd.Flag("query.external-label", "external label to add to results").StringMapVar(&opts.externalLabels) + cmd.Flag("query.limits.select.max-chunk-bytes", "the amount of chunk bytes a query can fetch in 'Select' operations. (0B is unlimited)").Default("0B").BytesVar(&opts.selectChunkBytesQuota) + cmd.Flag("query.limits.select.max-row-count", "the amount of rows a query can fetch in 'Select' operations. (0 is unlimited)").Default("0").Int64Var(&opts.selectRowCountQuota) + cmd.Flag("query.limits.queries.max-concurrent", "the amount of concurrent queries we can execute").Default("100").IntVar(&opts.concurrentQueryQuota) + cmd.Flag("query.storage.select.chunk-partition.max-range-bytes", "coalesce chunk reads into ranges of this length to be scheduled concurrently.").Default("64MiB").BytesVar(&opts.selectChunkPartitionMaxRange) + cmd.Flag("query.storage.select.chunk-partition.max-gap-bytes", "the maximum acceptable gap when coalescing chunk ranges.").Default("64MiB").BytesVar(&opts.selectChunkPartitionMaxGap) + cmd.Flag("query.storage.select.chunk-partition.max-concurrency", "the maximum amount of concurrent fetches to object storage per partition.").Default("2").IntVar(&opts.selectChunkPartitionMaxConcurrency) +} + +func (opts *apiOpts) registerServeThanosAPIFlags(cmd *kingpin.CmdClause) { + cmd.Flag("http.thanos.port", "port to host thanos gRPC api").Default("9001").IntVar(&opts.port) + cmd.Flag("http.thanos.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) +} + +func (opts *apiOpts) registerServePrometheusAPIFlags(cmd *kingpin.CmdClause) { + cmd.Flag("http.prometheus.port", "port to host prometheus query HTTP api").Default("9000").IntVar(&opts.port) + cmd.Flag("http.prometheus.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) +} + +func (opts *apiOpts) registerServeInternalAPIFlags(cmd *kingpin.CmdClause) { + cmd.Flag("http.internal.port", "port to host query api").Default("6060").IntVar(&opts.port) + cmd.Flag("http.internal.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) } func registerServeApp(app *kingpin.Application) (*kingpin.CmdClause, func(context.Context, *slog.Logger, *prometheus.Registry) error) { @@ -78,17 +146,17 @@ func registerServeApp(app *kingpin.Application) (*kingpin.CmdClause, func(contex return fmt.Errorf("unable to setup bucket: %w", err) } - discoverer, err := setupDiscovery(ctx, &g, log, bkt, opts.block) + discoverer, err := setupDiscovery(ctx, &g, log, bkt, opts.discovery) if err != nil { return fmt.Errorf("unable to setup discovery: %w", err) } - metaFilter, err := setupMetaFilter(ctx, &g, log, opts.block) + metaFilter, err := setupMetaFilter(ctx, &g, log, opts.syncer) if err != nil { return fmt.Errorf("unable to set up meta filter: %w", err) } - syncer, err := setupSyncer(ctx, &g, log, bkt, discoverer, metaFilter, opts.block) + syncer, err := setupSyncer(ctx, &g, log, bkt, discoverer, metaFilter, opts.syncer) if err != nil { return fmt.Errorf("unable to setup syncer: %w", err) } @@ -106,37 +174,25 @@ func registerServeApp(app *kingpin.Application) (*kingpin.CmdClause, func(contex } } -func setupInterrupt(ctx context.Context, g *run.Group, log *slog.Logger) { - ctx, cancel := context.WithCancel(ctx) - g.Add(func() error { - <-ctx.Done() - log.Info("Canceling actors") - return nil - }, func(error) { - cancel() - }) -} - type queryOpts struct { defaultStep time.Duration defaultLookback time.Duration defaultTimeout time.Duration externalLabels map[string]string -} -func (opts *queryOpts) registerFlags(cmd *kingpin.CmdClause) { - // We need to initialize the externalLabels map here - opts.externalLabels = make(map[string]string) + // Limits + selectChunkBytesQuota units.Base2Bytes + selectRowCountQuota int64 + concurrentQueryQuota int - cmd.Flag("query.step", "default step for range queries").Default("30s").DurationVar(&opts.defaultStep) - cmd.Flag("query.lookback", "default lookback for queries").Default("5m").DurationVar(&opts.defaultLookback) - cmd.Flag("query.timeout", "default timeout for queries").Default("30s").DurationVar(&opts.defaultTimeout) - cmd.Flag("query.external-label", "external label to add to results").StringMapVar(&opts.externalLabels) + // Storage + selectChunkPartitionMaxRange units.Base2Bytes + selectChunkPartitionMaxGap units.Base2Bytes + selectChunkPartitionMaxConcurrency int } func engineFromQueryOpts(opts queryOpts) promql.QueryEngine { return engine.New(engine.Opts{ - DisableFallback: false, DisableDuplicateLabelChecks: true, EngineOpts: promql.EngineOpts{ Logger: nil, @@ -154,153 +210,29 @@ func engineFromQueryOpts(opts queryOpts) promql.QueryEngine { } -type blockOpts struct { - discoveryInterval time.Duration - discoveryConcurrency int - - syncerInterval time.Duration - syncerConcurrency int - syncerReadBufferSize units.Base2Bytes - - filterType string - filterThanosBackfillEndpoint string - filterThanosBackfillUpdateInterval time.Duration -} - -func (opts *blockOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("block.discovery.interval", "interval to discover blocks").Default("1m").DurationVar(&opts.discoveryInterval) - cmd.Flag("block.discovery.concurrency", "concurrency for loading metadata").Default("1").IntVar(&opts.discoveryConcurrency) - cmd.Flag("block.syncer.interval", "interval to sync blocks").Default("1m").DurationVar(&opts.syncerInterval) - cmd.Flag("block.syncer.concurrency", "concurrency for loading blocks").Default("1").IntVar(&opts.syncerConcurrency) - cmd.Flag("block.syncer.read-buffer-size", "read buffer size for blocks").Default("2MiB").BytesVar(&opts.syncerReadBufferSize) - cmd.Flag("block.filter.type", "").Default("all-metas").EnumVar(&opts.filterType, "thanos-backfill", "all-metas") - cmd.Flag("block.filter.thanos-backfill.endpoint", "endpoint to ignore for backfill").StringVar(&opts.filterThanosBackfillEndpoint) - cmd.Flag("block.filter.thanos-backfill.interval", "interval to update thanos-backfill timerange").Default("1m").DurationVar(&opts.filterThanosBackfillUpdateInterval) -} - -func setupDiscovery(ctx context.Context, g *run.Group, log *slog.Logger, bkt objstore.Bucket, opts blockOpts) (*cfdb.Discoverer, error) { - discoverer := cfdb.NewDiscoverer(bkt, cfdb.MetaConcurrency(opts.discoveryConcurrency)) - - log.Info("Running initial discovery") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) - defer iterCancel() - if err := discoverer.Discover(iterCtx); err != nil { - return nil, fmt.Errorf("unable to run initial discovery: %w", err) - } - - ctx, cancel := context.WithCancel(context.Background()) - g.Add(func() error { - return runutil.Repeat(opts.discoveryInterval, ctx.Done(), func() error { - log.Debug("Running discovery") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.discoveryInterval) - defer iterCancel() - if err := discoverer.Discover(iterCtx); err != nil { - log.Warn("Unable to discover new blocks", slog.Any("err", err)) - } - return nil - }) - }, func(error) { - log.Info("Stopping discovery") - cancel() - }) - return discoverer, nil -} - -func setupMetaFilter(ctx context.Context, g *run.Group, log *slog.Logger, opts blockOpts) (cfdb.MetaFilter, error) { - switch opts.filterType { - case "all-metas": - return cfdb.AllMetasMetaFilter, nil - case "thanos-backfill": - thanosBackfillMetaFilter := cfdb.NewThanosBackfillMetaFilter(opts.filterThanosBackfillEndpoint) - - log.Info("Initializing thanos-backfill meta filter") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.filterThanosBackfillUpdateInterval) - defer iterCancel() - if err := thanosBackfillMetaFilter.Update(iterCtx); err != nil { - return nil, fmt.Errorf("unable to initialize thanos-backfill meta filter: %w", err) - } - - ctx, cancel := context.WithCancel(context.Background()) - g.Add(func() error { - return runutil.Repeat(opts.filterThanosBackfillUpdateInterval, ctx.Done(), func() error { - log.Debug("Updating thanos-backfill meta filter") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.filterThanosBackfillUpdateInterval) - defer iterCancel() - if err := thanosBackfillMetaFilter.Update(iterCtx); err != nil { - log.Warn("Unable to update thanos-backfill meta filter", slog.Any("err", err)) - } - return nil - }) - }, func(error) { - log.Info("Stopping thanos-backfill meta filter updates") - cancel() - }) - return thanosBackfillMetaFilter, nil - default: - return nil, fmt.Errorf("unknown meta filter type: %s", opts.filterType) - } -} - -func setupSyncer(ctx context.Context, g *run.Group, log *slog.Logger, bkt objstore.Bucket, discoverer *cfdb.Discoverer, metaFilter cfdb.MetaFilter, opts blockOpts) (*cfdb.Syncer, error) { - syncer := cfdb.NewSyncer( - bkt, - cfdb.FilterMetas(metaFilter), - cfdb.BlockConcurrency(opts.syncerConcurrency), - cfdb.BlockOptions( - cfdb.ReadBufferSize(opts.syncerReadBufferSize), - ), - ) - - log.Info("Running initial sync") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.syncerInterval) - defer iterCancel() - if err := syncer.Sync(iterCtx, discoverer.Metas()); err != nil { - return nil, fmt.Errorf("unable to run initial discovery: %w", err) - } - - ctx, cancel := context.WithCancel(context.Background()) - g.Add(func() error { - return runutil.Repeat(opts.syncerInterval, ctx.Done(), func() error { - log.Debug("Running sync") - - iterCtx, iterCancel := context.WithTimeout(ctx, opts.syncerInterval) - defer iterCancel() - if err := syncer.Sync(iterCtx, discoverer.Metas()); err != nil { - log.Warn("Unable to sync new blocks", slog.Any("err", err)) - } - return nil - }) - }, func(error) { - log.Info("Stopping syncer") - cancel() - }) - return syncer, nil -} - -type thanosAPIOpts struct { - port int - - shutdownTimeout time.Duration -} - -func (opts *thanosAPIOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("http.thanos.port", "port to host query api").Default("9001").IntVar(&opts.port) - cmd.Flag("http.thanos.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) -} - -func setupThanosAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts thanosAPIOpts, qOpts queryOpts) { +func setupThanosAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts apiOpts, qOpts queryOpts) { server := grpc.NewServer( grpc.MaxSendMsgSize(math.MaxInt32), grpc.MaxRecvMsgSize(math.MaxInt32), + grpc.StatsHandler(otelgrpc.NewServerHandler(otelgrpc.WithPropagators(autoprop.NewTextMapPropagator()))), + grpc.UnaryInterceptor(cfgrpc.ServerMetrics.UnaryServerInterceptor()), + grpc.StreamInterceptor(cfgrpc.ServerMetrics.StreamServerInterceptor()), ) - infopb.RegisterInfoServer(server, cfgrpc.NewInfoServer(db)) - querypb.RegisterQueryServer(server, cfgrpc.NewQueryServer(db, engineFromQueryOpts(qOpts))) + queryServer := cfgrpc.NewQueryServer( + db, + engineFromQueryOpts(qOpts), + cfgrpc.ConcurrentQueryQuota(qOpts.concurrentQueryQuota), + cfgrpc.SelectChunkBytesQuota(qOpts.selectChunkBytesQuota), + cfgrpc.SelectRowCountQuota(qOpts.selectRowCountQuota), + cfgrpc.SelectChunkPartitionMaxRange(qOpts.selectChunkPartitionMaxRange), + cfgrpc.SelectChunkPartitionMaxGap(qOpts.selectChunkPartitionMaxGap), + cfgrpc.SelectChunkPartitionMaxConcurrency(qOpts.selectChunkPartitionMaxConcurrency), + ) + + infopb.RegisterInfoServer(server, queryServer) + storepb.RegisterStoreServer(server, queryServer) + querypb.RegisterQueryServer(server, queryServer) reflection.Register(server) @@ -333,23 +265,18 @@ func setupThanosAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts thanosAPIO }) } -type promAPIOpts struct { - port int - - shutdownTimeout time.Duration -} - -func (opts *promAPIOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("http.prometheus.port", "port to host query api").Default("9000").IntVar(&opts.port) - cmd.Flag("http.prometheus.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) -} - -func setupPromAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts promAPIOpts, qOpts queryOpts) { - handler := cfhttp.NewAPI(db.Queryable(), engineFromQueryOpts(qOpts), +func setupPromAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts apiOpts, qOpts queryOpts) { + handler := cfhttp.NewAPI(db, engineFromQueryOpts(qOpts), cfhttp.QueryOptions( cfhttp.DefaultStep(qOpts.defaultStep), cfhttp.DefaultLookback(qOpts.defaultLookback), cfhttp.DefaultTimeout(qOpts.defaultTimeout), + cfhttp.ConcurrentQueryQuota(qOpts.concurrentQueryQuota), + cfhttp.SelectChunkBytesQuota(qOpts.selectChunkBytesQuota), + cfhttp.SelectRowCountQuota(qOpts.selectRowCountQuota), + cfhttp.SelectChunkPartitionMaxRange(qOpts.selectChunkPartitionMaxRange), + cfhttp.SelectChunkPartitionMaxGap(qOpts.selectChunkPartitionMaxGap), + cfhttp.SelectChunkPartitionMaxConcurrency(qOpts.selectChunkPartitionMaxConcurrency), )) server := &http.Server{Addr: fmt.Sprintf(":%d", opts.port), Handler: handler} @@ -369,49 +296,3 @@ func setupPromAPI(g *run.Group, log *slog.Logger, db *cfdb.DB, opts promAPIOpts, } }) } - -type internalAPIOpts struct { - port int - - shutdownTimeout time.Duration -} - -func (opts *internalAPIOpts) registerFlags(cmd *kingpin.CmdClause) { - cmd.Flag("http.internal.port", "port to host query api").Default("6060").IntVar(&opts.port) - cmd.Flag("http.internal.shutdown-timeout", "timeout on shutdown").Default("10s").DurationVar(&opts.shutdownTimeout) -} - -func setupInternalAPI(g *run.Group, log *slog.Logger, reg *prometheus.Registry, opts internalAPIOpts) { - mux := http.NewServeMux() - mux.HandleFunc("/debug/pprof/", pprof.Index) - mux.HandleFunc("/debug/pprof/profile", pprof.Profile) - mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - mux.Handle("/metrics", promhttp.HandlerFor(reg, promhttp.HandlerOpts{})) - - mux.HandleFunc("/-/healthy", func(w http.ResponseWriter, _ *http.Request) { - w.WriteHeader(http.StatusOK) - fmt.Fprintf(w, "OK") - }) - mux.HandleFunc("/-/ready", func(w http.ResponseWriter, _ *http.Request) { - w.WriteHeader(http.StatusOK) - fmt.Fprintf(w, "OK") - }) - - server := &http.Server{Addr: fmt.Sprintf(":%d", opts.port), Handler: mux} - g.Add(func() error { - log.Info("Serving internal api", slog.Int("port", opts.port)) - if err := server.ListenAndServe(); err != http.ErrServerClosed { - return err - } - return nil - }, func(error) { - log.Info("Shutting down internal api", slog.Int("port", opts.port)) - ctx, cancel := context.WithTimeout(context.Background(), opts.shutdownTimeout) - defer cancel() - - if err := server.Shutdown(ctx); err != nil { - log.Error("Error shutting down internal server", slog.Any("err", err)) - } - }) -} diff --git a/convert/chunks.go b/convert/chunks.go index abc37b2..1e3aeeb 100644 --- a/convert/chunks.go +++ b/convert/chunks.go @@ -16,15 +16,6 @@ import ( "github.com/cloudflare/parquet-tsdb-poc/schema" ) -func allChunksEmpty(chkBytes [schema.ChunkColumnsPerDay][]byte) bool { - for _, chk := range chkBytes { - if len(chk) != 0 { - return false - } - } - return true -} - func collectChunks(it chunks.Iterator) ([schema.ChunkColumnsPerDay][]byte, error) { var ( res [schema.ChunkColumnsPerDay][]byte @@ -44,8 +35,10 @@ func collectChunks(it chunks.Iterator) ([schema.ChunkColumnsPerDay][]byte, error return chunks[i].MinTime < chunks[j].MinTime }) for _, chk := range chunks { + if chk.Chunk.NumSamples() == 0 { + continue + } enc, bs := chk.Chunk.Encoding(), chk.Chunk.Bytes() - hour := time.UnixMilli(chk.MinTime).UTC().Hour() chkIdx := (hour / int(schema.ChunkColumnLength.Hours())) % schema.ChunkColumnsPerDay chkBytes := res[chkIdx] diff --git a/convert/convert.go b/convert/convert.go index 5f17309..4375617 100644 --- a/convert/convert.go +++ b/convert/convert.go @@ -5,25 +5,32 @@ package convert import ( + "bufio" "bytes" "context" + "errors" "fmt" "io" "time" + "golang.org/x/sync/errgroup" + + "github.com/alecthomas/units" "github.com/parquet-go/parquet-go" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/tombstones" + "github.com/prometheus/prometheus/util/zeropool" "github.com/thanos-io/objstore" "google.golang.org/protobuf/proto" + "github.com/cloudflare/parquet-tsdb-poc/internal/encoding" "github.com/cloudflare/parquet-tsdb-poc/internal/util" "github.com/cloudflare/parquet-tsdb-poc/proto/metapb" "github.com/cloudflare/parquet-tsdb-poc/schema" ) -type Convertible interface { +type Convertable interface { Index() (tsdb.IndexReader, error) Chunks() (tsdb.ChunkReader, error) Tombstones() (tombstones.Reader, error) @@ -31,14 +38,18 @@ type Convertible interface { } type convertOpts struct { - numRowGroups int - rowGroupSize int - sortBufSize int - - bufferPool parquet.BufferPool + numRowGroups int + rowGroupSize int + encodingConcurrency int + sortLabels []string sortingColumns [][]string bloomfilterColumns [][]string + labelBufferPool parquet.BufferPool + chunkbufferPool parquet.BufferPool + + labelPageBufferSize int + chunkPageBufferSize int } func (cfg convertOpts) buildBloomfilterColumns() []parquet.BloomFilterColumn { @@ -52,7 +63,7 @@ func (cfg convertOpts) buildBloomfilterColumns() []parquet.BloomFilterColumn { } func (cfg convertOpts) buildSortingColumns() []parquet.SortingColumn { - cols := make([]parquet.SortingColumn, 0, len(cfg.sortingColumns)) + cols := make([]parquet.SortingColumn, 0, len(cfg.bloomfilterColumns)) for i := range cfg.sortingColumns { cols = append(cols, @@ -69,25 +80,50 @@ func RowGroupSize(rbs int) ConvertOption { } } -func SortBufSize(sbs int) ConvertOption { +func RowGroupCount(rc int) ConvertOption { return func(opts *convertOpts) { - opts.sortBufSize = sbs + opts.numRowGroups = rc } } -func SortBy(labels []string) ConvertOption { +func SortBy(labels ...string) ConvertOption { return func(opts *convertOpts) { sortingColumns := make([][]string, len(labels)) for i := range labels { sortingColumns[i] = []string{schema.LabelNameToColumn(labels[i])} } opts.sortingColumns = sortingColumns + opts.sortLabels = labels + } +} + +func LabelPageBufferSize(pb units.Base2Bytes) ConvertOption { + return func(opts *convertOpts) { + opts.labelPageBufferSize = int(pb) + } +} + +func ChunkPageBufferSize(pb units.Base2Bytes) ConvertOption { + return func(opts *convertOpts) { + opts.chunkPageBufferSize = int(pb) + } +} + +func LabelBufferPool(p parquet.BufferPool) ConvertOption { + return func(opts *convertOpts) { + opts.labelBufferPool = p + } +} + +func ChunkBufferPool(p parquet.BufferPool) ConvertOption { + return func(opts *convertOpts) { + opts.chunkbufferPool = p } } -func BufferPool(p parquet.BufferPool) ConvertOption { +func EncodingConcurrency(c int) ConvertOption { return func(opts *convertOpts) { - opts.bufferPool = p + opts.encodingConcurrency = c } } @@ -95,16 +131,19 @@ func ConvertTSDBBlock( ctx context.Context, bkt objstore.Bucket, day time.Time, - blks []Convertible, + blks []Convertable, opts ...ConvertOption, ) error { cfg := &convertOpts{ - rowGroupSize: 1_000_000, - numRowGroups: 6, - sortBufSize: 128_000, - bufferPool: parquet.NewBufferPool(), - sortingColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}}, - bloomfilterColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}}, + rowGroupSize: 1_000_000, + numRowGroups: 6, + sortingColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}}, + bloomfilterColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}}, + labelBufferPool: parquet.NewBufferPool(), + chunkbufferPool: parquet.NewBufferPool(), + encodingConcurrency: 1, + labelPageBufferSize: int(256 * units.KiB), + chunkPageBufferSize: int(2 * units.MiB), } for i := range opts { opts[i](cfg) @@ -112,11 +151,14 @@ func ConvertTSDBBlock( start, end := util.BeginOfDay(day), util.EndOfDay(day) name, err := schema.BlockNameForDay(start) if err != nil { - return fmt.Errorf("unable to get block name: %s", err) + return fmt.Errorf("unable to get block name: %w", err) } - rr, err := newIndexRowReader(ctx, start.UnixMilli(), end.UnixMilli(), blks) + rr, err := newIndexRowReader(ctx, start.UnixMilli(), end.UnixMilli(), blks, indexReaderOpts{ + sortLabels: cfg.sortLabels, + concurrency: cfg.encodingConcurrency, + }) if err != nil { - return fmt.Errorf("unable to create index rowreader: %s", err) + return fmt.Errorf("unable to create index rowreader: %w", err) } defer rr.Close() @@ -126,17 +168,22 @@ func ConvertTSDBBlock( end.UnixMilli(), rr, bkt, - cfg.bufferPool, - cfg.sortBufSize, cfg.rowGroupSize, cfg.numRowGroups, cfg.buildSortingColumns(), cfg.buildBloomfilterColumns(), + cfg.labelBufferPool, + cfg.chunkbufferPool, + cfg.labelPageBufferSize, + cfg.chunkPageBufferSize, ) if err := converter.convert(ctx); err != nil { - return fmt.Errorf("unable to convert block: %s", err) + return fmt.Errorf("unable to convert block: %w", err) } + + lastSuccessfulConvertTime.SetToCurrentTime() + return nil } @@ -146,17 +193,20 @@ type converter struct { currentShard int seriesPerShard int - sortBufSize int rowGroupSize int numRowGroups int bkt objstore.Bucket rr *indexRowReader - p parquet.BufferPool sortingColumns []parquet.SortingColumn bloomfilterColumns []parquet.BloomFilterColumn + labelBufferPool parquet.BufferPool + chunkBufferPool parquet.BufferPool + + labelPageBufferSize int + chunkPageBufferSize int } func newConverter( @@ -165,12 +215,14 @@ func newConverter( maxt int64, rr *indexRowReader, bkt objstore.Bucket, - p parquet.BufferPool, - sortBufSize int, rowGroupSize int, numRowGroups int, sortingColumns []parquet.SortingColumn, bloomfilterColumns []parquet.BloomFilterColumn, + labelBufferPool parquet.BufferPool, + chunkBufferPool parquet.BufferPool, + labelPageBufferSize int, + chunkPageBufferSize int, ) *converter { return &converter{ @@ -180,19 +232,49 @@ func newConverter( bkt: bkt, rr: rr, - p: p, rowGroupSize: rowGroupSize, numRowGroups: numRowGroups, - sortBufSize: sortBufSize, sortingColumns: sortingColumns, bloomfilterColumns: bloomfilterColumns, + labelBufferPool: labelBufferPool, + chunkBufferPool: chunkBufferPool, + + labelPageBufferSize: labelPageBufferSize, + chunkPageBufferSize: chunkPageBufferSize, + } +} + +func (c *converter) labelWriterOptions() []parquet.WriterOption { + return []parquet.WriterOption{ + parquet.MaxRowsPerRowGroup(int64(c.rowGroupSize)), + parquet.SortingWriterConfig(parquet.SortingColumns(c.sortingColumns...)), + parquet.BloomFilters(c.bloomfilterColumns...), + parquet.SkipPageBounds(schema.LabelIndexColumn), + parquet.ColumnPageBuffers(c.labelBufferPool), + parquet.PageBufferSize(c.labelPageBufferSize), + } +} + +func (c *converter) chunkWriterOptions() []parquet.WriterOption { + return []parquet.WriterOption{ + parquet.MaxRowsPerRowGroup(int64(c.rowGroupSize)), + parquet.SkipPageBounds(schema.LabelHashColumn), + parquet.SkipPageBounds(schema.ChunksColumn0), + parquet.SkipPageBounds(schema.ChunksColumn1), + parquet.SkipPageBounds(schema.ChunksColumn2), + parquet.ColumnPageBuffers(c.chunkBufferPool), + parquet.PageBufferSize(c.chunkPageBufferSize), } } + func (c *converter) convert(ctx context.Context) error { if err := c.convertShards(ctx); err != nil { return fmt.Errorf("unable to convert shards: %w", err) } + if err := c.optimizeShards(ctx); err != nil { + return fmt.Errorf("unable to optimize shards: %w", err) + } if err := c.writeMetaFile(ctx); err != nil { return fmt.Errorf("unable to write meta file: %w", err) } @@ -201,25 +283,18 @@ func (c *converter) convert(ctx context.Context) error { func (c *converter) writeMetaFile(ctx context.Context) error { meta := &metapb.Metadata{ - ColumnsForName: make(map[string]*metapb.Columns), - Mint: c.mint, - Maxt: c.maxt, - Shards: int64(c.currentShard) + 1, - } - for k, v := range c.rr.NameLabelMapping() { - cols := &metapb.Columns{Columns: make([]string, 0, len(v))} - for lbl := range v { - cols.Columns = append(cols.Columns, lbl) - } - meta.ColumnsForName[k] = cols + Version: schema.V2, + Mint: c.mint, + Maxt: c.maxt, + Shards: int64(c.currentShard) + 1, } metaBytes, err := proto.Marshal(meta) if err != nil { - return fmt.Errorf("unable to marshal meta bytes: %s", err) + return fmt.Errorf("unable to marshal meta bytes: %w", err) } if err := c.bkt.Upload(ctx, schema.MetaFileNameForBlock(c.name), bytes.NewReader(metaBytes)); err != nil { - return fmt.Errorf("unable to upload meta file: %s", err) + return fmt.Errorf("unable to upload meta file: %w", err) } return nil @@ -228,7 +303,7 @@ func (c *converter) writeMetaFile(ctx context.Context) error { func (c *converter) convertShards(ctx context.Context) error { for { if ok, err := c.convertShard(ctx); err != nil { - return fmt.Errorf("unable to convert shard: %s", err) + return fmt.Errorf("unable to convert shard: %w", err) } else if !ok { break } @@ -236,28 +311,120 @@ func (c *converter) convertShards(ctx context.Context) error { return nil } +func (c *converter) optimizeShards(ctx context.Context) error { + g, ctx := errgroup.WithContext(ctx) + for i := range c.currentShard + 1 { + g.Go(func() error { + return c.optimizeShard(ctx, i) + }) + } + return g.Wait() +} + +// Since we have to compute the schema from the whole TSDB Block it is highly likely that the +// labels parquet file we wrote just now contains many empty columns - we project them away again +func (c *converter) optimizeShard(ctx context.Context, i int) error { + rc, err := c.bkt.Get(ctx, schema.LabelsPfileNameForShard(c.name, i)) + if err != nil { + return fmt.Errorf("unable to fetch labels parquet file: %w", err) + } + defer rc.Close() + + rbuf := bytes.NewBuffer(nil) + if _, err := io.Copy(rbuf, rc); err != nil { + return fmt.Errorf("unable to copy labels parquet file: %w", err) + } + + pf, err := parquet.OpenFile(bytes.NewReader(rbuf.Bytes()), int64(rbuf.Len())) + if err != nil { + return fmt.Errorf("unable to open labels parquet file: %w", err) + } + + s := pf.Schema() + ns := schema.WithCompression(schema.RemoveNullColumns(pf)) + + buf := bytes.NewBuffer(nil) + w := parquet.NewGenericWriter[any](buf, append(c.labelWriterOptions(), ns)...) + + rb := parquet.NewRowBuilder(ns) + rowBuf := make([]parquet.Row, 128) + colIdxSlice := make([]int, 0) + labelIndexColumn := columnIDForKnownColumn(ns, schema.LabelIndexColumn) + + for _, rg := range pf.RowGroups() { + rows := rg.Rows() + defer rows.Close() + + for { + n, err := rows.ReadRows(rowBuf) + if err != nil && !errors.Is(err, io.EOF) { + return fmt.Errorf("unable to read rows: %w", err) + } + + for i, row := range rowBuf[:n] { + rb.Reset() + colIdxSlice = colIdxSlice[:0] + for j, v := range row { + if !v.IsNull() { + if lc, ok := ns.Lookup(s.Columns()[j]...); ok && lc.ColumnIndex != labelIndexColumn { + colIdxSlice = append(colIdxSlice, lc.ColumnIndex) + rb.Add(lc.ColumnIndex, v) + } + } + } + rb.Add(labelIndexColumn, parquet.ValueOf(encoding.EncodeLabelColumnIndex(colIdxSlice))) + rowBuf[i] = rb.AppendRow(rowBuf[i][:0]) + } + + if m, err := w.WriteRows(rowBuf[:n]); err != nil { + return fmt.Errorf("unable to write transformed rows: %w", err) + } else if m != n { + return fmt.Errorf("unable to write rows: %d != %d", n, m) + } + + if errors.Is(err, io.EOF) { + break + } + } + } + if err := w.Close(); err != nil { + return fmt.Errorf("unable to close writer: %w", err) + } + + if err := c.bkt.Upload(ctx, schema.LabelsPfileNameForShard(c.name, i), buf); err != nil { + return fmt.Errorf("unable to override optimized labels parquet file: %w", err) + } + + return nil +} + func (c *converter) convertShard(ctx context.Context) (bool, error) { s := c.rr.Schema() rowsToWrite := c.numRowGroups * c.rowGroupSize - in := c.p.GetBuffer() - defer c.p.PutBuffer(in) - - sw := newSortingWriter(in, c.p, s, c.sortBufSize, c.sortingColumns...) - n, err := parquet.CopyRows(sw, newLimitReader(c.rr, rowsToWrite)) + w, err := newSplitFileWriter(ctx, c.bkt, s, map[string]writerConfig{ + schema.LabelsPfileNameForShard(c.name, c.currentShard): { + s: schema.WithCompression(schema.LabelsProjection(s)), + opts: c.labelWriterOptions(), + }, + schema.ChunksPfileNameForShard(c.name, c.currentShard): { + s: schema.WithCompression(schema.ChunkProjection(s)), + opts: c.chunkWriterOptions(), + }, + }, + ) if err != nil { - return false, fmt.Errorf("unable to copy rows to sorting writer: %s", err) - } - if err := sw.Flush(); err != nil { - return false, fmt.Errorf("unable to flush sorting writer: %s", err) + return false, fmt.Errorf("unable to build multifile writer: %w", err) } - if err := c.writeShardLabelsPfile(ctx, sw, c.currentShard); err != nil { - return false, fmt.Errorf("unable to write label parquetfile %d: %s", c.currentShard, err) + n, err := parquet.CopyRows(w, newBufferedReader(ctx, newLimitReader(c.rr, rowsToWrite))) + if err != nil { + return false, fmt.Errorf("unable to copy rows to writer: %w", err) } - if err := c.writeShardChunksPfile(ctx, sw, c.currentShard); err != nil { - return false, fmt.Errorf("unable to write chunks parquetfile %d: %s", c.currentShard, err) + if err := w.Close(); err != nil { + return false, fmt.Errorf("unable to close multifile writer: %w", err) } + if n < int64(rowsToWrite) { return false, nil } @@ -265,136 +432,194 @@ func (c *converter) convertShard(ctx context.Context) (bool, error) { return true, nil } -func (c *converter) writeShardLabelsPfile( - ctx context.Context, - sw *sortingWriter, - shard int, -) error { - out := c.p.GetBuffer() - defer c.p.PutBuffer(out) +type limitReader struct { + parquet.RowReader + limit int + cur int +} - inSchema := c.rr.Schema() - outSchema := schema.LabelsProjection(schema.WithCompression(inSchema)) +func newLimitReader(r parquet.RowReader, limit int) parquet.RowReader { + return &limitReader{RowReader: r, limit: limit} +} - conv, err := parquet.Convert(outSchema, inSchema) +func (lr *limitReader) ReadRows(buf []parquet.Row) (int, error) { + n, err := lr.RowReader.ReadRows(buf) if err != nil { - return fmt.Errorf("unable to convert schemas") + return n, err } + lr.cur += n - sr, err := sw.RowReader() - if err != nil { - return fmt.Errorf("unable to get sorted row reader: %s", err) + if lr.cur > lr.limit { + return n, io.EOF } + return n, nil +} - cr := parquet.ConvertRowReader(sr, conv) - - writer := parquet.NewGenericWriter[any](out, outSchema, parquet.BloomFilters(c.bloomfilterColumns...)) - if _, err := parquet.CopyRows(newFlushingWriter(writer, c.rowGroupSize), cr); err != nil { - return fmt.Errorf("unable to copy rows: %s", err) - } - if err := writer.Close(); err != nil { - return fmt.Errorf("unable to close writer: %s", err) - } - if _, err := out.Seek(0, io.SeekStart); err != nil { - return fmt.Errorf("unable to rewind temporary buffer: %s", err) - } - if err := c.bkt.Upload(ctx, schema.LabelsPfileNameForShard(c.name, shard), out); err != nil { - return fmt.Errorf("unable to upload parquet file: %s", err) - } +type fileWriter struct { + pw *parquet.GenericWriter[any] + conv parquet.Conversion + w io.WriteCloser + r io.ReadCloser +} - return nil +type splitPipeFileWriter struct { + fileWriters map[string]*fileWriter + g *errgroup.Group } -func (c *converter) writeShardChunksPfile( - ctx context.Context, - sw *sortingWriter, - shard int, -) error { - out := c.p.GetBuffer() - defer c.p.PutBuffer(out) +type writerConfig struct { + s *parquet.Schema + opts []parquet.WriterOption +} - inSchema := c.rr.Schema() - outSchema := schema.ChunkProjection(schema.WithCompression(inSchema)) +func newSplitFileWriter(ctx context.Context, bkt objstore.Bucket, inSchema *parquet.Schema, files map[string]writerConfig) (*splitPipeFileWriter, error) { + fileWriters := make(map[string]*fileWriter) + g, ctx := errgroup.WithContext(ctx) + for file, cfg := range files { + conv, err := parquet.Convert(cfg.s, inSchema) + if err != nil { + return nil, fmt.Errorf("unable to convert schemas") + } - conv, err := parquet.Convert(outSchema, inSchema) - if err != nil { - return fmt.Errorf("unable to convert schemas") + r, w := io.Pipe() + bw := bufio.NewWriterSize(w, 32_000_000) + br := bufio.NewReaderSize(r, 32_000_000) + fileWriters[file] = &fileWriter{ + pw: parquet.NewGenericWriter[any](bw, append(cfg.opts, cfg.s)...), + w: w, + conv: conv, + } + g.Go(func() error { + defer func() { _ = r.Close() }() + return bkt.Upload(ctx, file, br) + }) } + return &splitPipeFileWriter{ + fileWriters: fileWriters, + g: g, + }, nil - sr, err := sw.RowReader() - if err != nil { - return fmt.Errorf("unable to get sorted row reader: %s", err) - } +} - cr := parquet.ConvertRowReader(sr, conv) +func (s *splitPipeFileWriter) WriteRows(rows []parquet.Row) (int, error) { + var g errgroup.Group + for _, writer := range s.fileWriters { + g.Go(func() error { + rr := make([]parquet.Row, len(rows)) + for i, row := range rows { + rr[i] = row.Clone() + } + _, err := writer.conv.Convert(rr) + if err != nil { + return fmt.Errorf("unable to convert rows: %w", err) + } + n, err := writer.pw.WriteRows(rr) + if err != nil { + return fmt.Errorf("unable to write rows: %w", err) + } + if n != len(rows) { + return fmt.Errorf("unable to write rows: %d != %d", n, len(rows)) + } + return nil + }) + } + return len(rows), g.Wait() +} - writer := parquet.NewGenericWriter[any](out, outSchema, parquet.BloomFilters(c.bloomfilterColumns...)) - if _, err := parquet.CopyRows(newFlushingWriter(writer, c.rowGroupSize), cr); err != nil { - return fmt.Errorf("unable to copy rows: %s", err) - } - if err := writer.Close(); err != nil { - return fmt.Errorf("unable to close writer: %s", err) - } - if _, err := out.Seek(0, io.SeekStart); err != nil { - return fmt.Errorf("unable to rewind temporary buffer: %s", err) +func (s *splitPipeFileWriter) Close() error { + errs := make([]error, 0) + for _, fw := range s.fileWriters { + if err := fw.pw.Close(); err != nil { + errs = append(errs, fmt.Errorf("unable to close pipewriter: %w", err)) + } + if err := fw.w.Close(); err != nil { + errs = append(errs, fmt.Errorf("unable to close writer: %w", err)) + } } - if err := c.bkt.Upload(ctx, schema.ChunksPfileNameForShard(c.name, shard), out); err != nil { - return fmt.Errorf("unable to upload parquet file: %s", err) + if err := s.g.Wait(); err != nil { + errs = append(errs, fmt.Errorf("unable to wait for group: %w", err)) } - - return nil + return errors.Join(errs...) } -type rowWriterFlusher interface { - parquet.RowWriter - Flush() error -} +type bufferedReader struct { + rr parquet.RowReader -type flushingWriter struct { - rowWriterFlusher - flush int - cur int -} + ctx context.Context + c chan []parquet.Row + errCh chan error + rowPool zeropool.Pool[[]parquet.Row] -func newFlushingWriter(w rowWriterFlusher, flush int) parquet.RowWriter { - return &flushingWriter{rowWriterFlusher: w, flush: flush} + current []parquet.Row + currentIndex int } -func (fw *flushingWriter) WriteRows(buf []parquet.Row) (int, error) { - n, err := fw.rowWriterFlusher.WriteRows(buf) - if err != nil { - return n, err +func newBufferedReader(ctx context.Context, rr parquet.RowReader) *bufferedReader { + br := &bufferedReader{ + rr: rr, + ctx: ctx, + c: make(chan []parquet.Row, 128), + errCh: make(chan error, 1), + rowPool: zeropool.New(func() []parquet.Row { + return make([]parquet.Row, 128) + }), } - fw.cur += n - if fw.cur > fw.flush { - if err := fw.rowWriterFlusher.Flush(); err != nil { - return n, err + go br.readRows() + + return br +} + +func (b *bufferedReader) ReadRows(rows []parquet.Row) (int, error) { + if b.current == nil { + select { + case next, ok := <-b.c: + if !ok { + return 0, io.EOF + } + b.current = next + b.currentIndex = 0 + case err := <-b.errCh: + return 0, err } - fw.cur = 0 } - return n, err -} -type limitReader struct { - parquet.RowReader - limit int - cur int + current := b.current[b.currentIndex:] + i := min(len(current), len(rows)) + copy(rows[:i], current[:i]) + b.currentIndex += i + if b.currentIndex >= len(b.current) { + b.rowPool.Put(b.current[0:cap(b.current)]) + b.current = nil + } + return i, nil } -func newLimitReader(r parquet.RowReader, limit int) parquet.RowReader { - return &limitReader{RowReader: r, limit: limit} +func (b *bufferedReader) Close() { + close(b.c) + close(b.errCh) } -func (lr *limitReader) ReadRows(buf []parquet.Row) (int, error) { - n, err := lr.RowReader.ReadRows(buf) - if err != nil { - return n, err - } - lr.cur += n - - if lr.cur > lr.limit { - return n, io.EOF +func (b *bufferedReader) readRows() { + for { + select { + case <-b.ctx.Done(): + b.errCh <- b.ctx.Err() + return + default: + rows := b.rowPool.Get() + n, err := b.rr.ReadRows(rows) + if n > 0 { + b.c <- rows[:n] + } + if err != nil { + if err == io.EOF { + close(b.c) + return + } + b.errCh <- err + return + } + } } - return n, nil } diff --git a/convert/convert_test.go b/convert/convert_test.go new file mode 100644 index 0000000..847b215 --- /dev/null +++ b/convert/convert_test.go @@ -0,0 +1,235 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package convert + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "maps" + "slices" + "testing" + "time" + + "github.com/alecthomas/units" + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/util/teststorage" + "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" + + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/locate" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +func TestConverter(t *testing.T) { + st := teststorage.New(t) + t.Cleanup(func() { _ = st.Close() }) + + bkt, err := filesystem.NewBucket(t.TempDir()) + if err != nil { + t.Fatalf("unable to create bucket: %s", err) + } + t.Cleanup(func() { _ = bkt.Close() }) + + app := st.Appender(t.Context()) + for i := range 1_000 { + for range 120 { + lbls := labels.FromStrings( + "__name__", fmt.Sprintf("foo_%d", i/10), + fmt.Sprintf("col_%d", i/100), fmt.Sprintf("%d", 2*i), + ) + _, err := app.Append(0, lbls, time.Second.Milliseconds(), float64(i)) + if err != nil { + t.Fatalf("unable to append sample: %s", err) + } + } + } + if err := app.Commit(); err != nil { + t.Fatalf("unable to commit samples: %s", err) + } + + h := st.Head() + d := util.BeginOfDay(time.UnixMilli(h.MinTime())).UTC() + + opts := []ConvertOption{ + SortBy(labels.MetricName), + RowGroupSize(250), + RowGroupCount(2), + LabelPageBufferSize(units.KiB), // results in 2 pages + } + if err := ConvertTSDBBlock(t.Context(), bkt, d, []Convertable{h}, opts...); err != nil { + t.Fatalf("unable to convert tsdb block: %s", err) + } + + discoverer := locate.NewDiscoverer(bkt) + if err := discoverer.Discover(t.Context()); err != nil { + t.Fatalf("unable to convert parquet block: %s", err) + } + metas := discoverer.Metas() + + if n := len(metas); n != 1 { + t.Fatalf("unexpected number of metas: %d", n) + } + meta := metas[slices.Collect(maps.Keys(metas))[0]] + + if n := meta.Shards; n != 2 { + t.Fatalf("unexpected number of shards: %d", n) + } + + totalRows := int64(0) + for i := range int(meta.Shards) { + lf, err := loadParquetFile(t.Context(), bkt, schema.LabelsPfileNameForShard(meta.Name, i)) + if err != nil { + t.Fatalf("unable to load label parquet file for shard %d: %s", i, err) + } + cf, err := loadParquetFile(t.Context(), bkt, schema.ChunksPfileNameForShard(meta.Name, i)) + if err != nil { + t.Fatalf("unable to load chunk parquet file for shard %d: %s", i, err) + } + if cf.NumRows() != lf.NumRows() { + t.Fatalf("labels and chunk file have different numbers of rows for shard %d", i) + } + totalRows += lf.NumRows() + + if err := hasNoNullColumns(lf); err != nil { + t.Fatalf("unable to check for null columns: %s", err) + } + if err := hasExpectedIndexes(lf); err != nil { + t.Fatalf("unable to check for null columns: %s", err) + } + if err := nameColumnPageBoundsAreAscending(lf); err != nil { + t.Fatalf("unable to check that __name__ column page bounds are ascending: %s", err) + } + if err := nameColumnValuesAreIncreasing(lf); err != nil { + t.Fatalf("unable to check that __name__ column values are increasing: %s", err) + } + } + if totalRows != int64(st.DB.Head().NumSeries()) { + t.Fatalf("too few rows: %d", totalRows) + } +} + +func loadParquetFile(ctx context.Context, bkt objstore.BucketReader, name string) (*parquet.File, error) { + rdr, err := bkt.Get(ctx, name) + if err != nil { + return nil, fmt.Errorf("unable to get object: %w", err) + } + defer rdr.Close() + + buf := bytes.NewBuffer(nil) + if _, err := io.Copy(buf, rdr); err != nil { + return nil, fmt.Errorf("unable to read object: %w", err) + } + return parquet.OpenFile(bytes.NewReader(buf.Bytes()), int64(buf.Len())) +} + +func hasNoNullColumns(pf *parquet.File) error { + cidxs := pf.ColumnIndexes() + ncols := len(pf.Schema().Columns()) + + nullPages := make([][]bool, ncols) + for i := range ncols { + for j := range cidxs { + if j%ncols == i { + nullPages[i] = append(nullPages[i], cidxs[j].NullPages...) + } + } + } + + for i := range nullPages { + if !slices.ContainsFunc(nullPages[i], func(np bool) bool { return np == false }) { + return fmt.Errorf("column %d has only null pages", i) + } + } + return nil +} + +func hasExpectedIndexes(pf *parquet.File) error { + cidxs := pf.ColumnIndexes() + ncols := len(pf.Schema().Columns()) + + if _, ok := pf.Schema().Lookup(schema.LabelIndexColumn); !ok { + return fmt.Errorf("file is missing column: %s", schema.LabelIndexColumn) + } + for j := range cidxs { + lminv := len(cidxs[j].MinValues) + lmaxv := len(cidxs[j].MaxValues) + + if lminv == 0 { + return fmt.Errorf("column is missing min values: %d", j%ncols) + } + if lmaxv == 0 { + return fmt.Errorf("column is missing max values: %d", j%ncols) + } + } + return nil +} + +func nameColumnPageBoundsAreAscending(pf *parquet.File) error { + lc, ok := pf.Schema().Lookup(schema.LabelNameToColumn(labels.MetricName)) + if !ok { + return fmt.Errorf("file is missing column for label key: %s", labels.MetricName) + } + for _, rg := range pf.RowGroups() { + cc := rg.ColumnChunks()[lc.ColumnIndex] + cidx, err := cc.ColumnIndex() + if err != nil { + return fmt.Errorf("unable to get column index for column: %s", labels.MetricName) + } + // columns with 0 or 1 page are never indexed as ascending + if !cidx.IsAscending() && cidx.NumPages() > 1 { + return fmt.Errorf("column %q was not ascending", labels.MetricName) + } + } + return nil +} + +func nameColumnValuesAreIncreasing(pf *parquet.File) error { + lc, ok := pf.Schema().Lookup(schema.LabelNameToColumn(labels.MetricName)) + if !ok { + return fmt.Errorf("file is missing column for label key: %s", labels.MetricName) + } + comp := parquet.ByteArrayType.Compare + + for _, rg := range pf.RowGroups() { + cc := rg.ColumnChunks()[lc.ColumnIndex] + + pgs := cc.Pages() + defer pgs.Close() + + vwf := parquet.ValueWriterFunc(func(vs []parquet.Value) (int, error) { + if len(vs) == 0 || len(vs) == 1 { + return 0, nil + } + for i := range vs[:len(vs)-1] { + if comp(vs[i], vs[i+1]) > 0 { + return 0, fmt.Errorf("expected %q to be larger or equal to %q", vs[i+1], vs[i]) + } + } + return len(vs), nil + }) + + for { + p, err := pgs.ReadPage() + if err != nil && !errors.Is(err, io.EOF) { + return fmt.Errorf("unable to read page:%w", err) + } + if p == nil { + if errors.Is(err, io.EOF) { + break + } + return errors.New("unexpected nil page") + } + if _, err := parquet.CopyValues(vwf, p.Values()); err != nil { + return fmt.Errorf("unable to copy values :%w", err) + } + } + } + return nil +} diff --git a/convert/merge.go b/convert/merge.go new file mode 100644 index 0000000..3c3c566 --- /dev/null +++ b/convert/merge.go @@ -0,0 +1,120 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package convert + +import ( + "container/heap" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" +) + +func newMergeChunkSeriesSet(sets []storage.ChunkSeriesSet, compare func(a, b labels.Labels) int, mergeFunc storage.VerticalChunkSeriesMergeFunc) storage.ChunkSeriesSet { + h := heapChunkSeries{ + heap: make([]storage.ChunkSeriesSet, 0, len(sets)), + compare: compare, + } + for _, set := range sets { + if set == nil { + continue + } + if set.Next() { + heap.Push(&h, set) + } + if err := set.Err(); err != nil { + return &mergeChunkSeriesSet{err: err} + } + } + + return &mergeChunkSeriesSet{ + h: h, + mergeFunc: mergeFunc, + } +} + +type mergeChunkSeriesSet struct { + h heapChunkSeries + mergeFunc storage.VerticalChunkSeriesMergeFunc + currentSets []storage.ChunkSeriesSet + + err error +} + +func (m *mergeChunkSeriesSet) Next() bool { + if m.err != nil { + return false + } + for _, set := range m.currentSets { + if set == nil { + continue + } + if set.Next() { + heap.Push(&m.h, set) + } + if err := set.Err(); err != nil { + m.err = err + return false + } + } + + if len(m.h.heap) == 0 { + return false + } + + m.currentSets = m.currentSets[:0] + currentLabels := m.h.heap[0].At().Labels() + + for len(m.h.heap) > 0 && labels.Equal(m.h.heap[0].At().Labels(), currentLabels) { + m.currentSets = append(m.currentSets, heap.Pop(&m.h).(storage.ChunkSeriesSet)) + } + + return len(m.currentSets) > 0 +} + +func (m *mergeChunkSeriesSet) At() storage.ChunkSeries { + if len(m.currentSets) == 1 { + return m.currentSets[0].At() + } + series := make([]storage.ChunkSeries, 0, len(m.currentSets)) + for _, seriesSet := range m.currentSets { + series = append(series, seriesSet.At()) + } + return m.mergeFunc(series...) +} + +func (m *mergeChunkSeriesSet) Err() error { + return m.err +} + +func (m *mergeChunkSeriesSet) Warnings() annotations.Annotations { + return nil +} + +type heapChunkSeries struct { + heap []storage.ChunkSeriesSet + compare func(a, b labels.Labels) int +} + +func (h *heapChunkSeries) Len() int { return len(h.heap) } + +func (h *heapChunkSeries) Less(i, j int) bool { + a, b := h.heap[i].At().Labels(), h.heap[j].At().Labels() + return h.compare(a, b) < 0 +} + +func (h *heapChunkSeries) Swap(i, j int) { h.heap[i], h.heap[j] = h.heap[j], h.heap[i] } + +func (h *heapChunkSeries) Push(x any) { + h.heap = append(h.heap, x.(storage.ChunkSeriesSet)) +} + +func (h *heapChunkSeries) Pop() any { + old := h.heap + n := len(old) + x := old[n-1] + h.heap = old[0 : n-1] + return x +} diff --git a/convert/metrics.go b/convert/metrics.go new file mode 100644 index 0000000..ac57f49 --- /dev/null +++ b/convert/metrics.go @@ -0,0 +1,26 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package convert + +import ( + "errors" + + "github.com/prometheus/client_golang/prometheus" +) + +var ( + lastSuccessfulConvertTime = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "last_successful_convert_time_unix_seconds", + Help: "The timestamp the last conversion ran successfully.", + }) +) + +func RegisterMetrics(reg prometheus.Registerer) error { + lastSuccessfulConvertTime.Set(0) + + return errors.Join( + reg.Register(lastSuccessfulConvertTime), + ) +} diff --git a/convert/plan.go b/convert/plan.go new file mode 100644 index 0000000..0d8c0c1 --- /dev/null +++ b/convert/plan.go @@ -0,0 +1,107 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package convert + +import ( + "sort" + "time" + + "github.com/thanos-io/thanos/pkg/block/metadata" + + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +type Plan struct { + Download []metadata.Meta + ConvertForDates []time.Time +} + +type Planner struct { + // do not create parquet blocks that are younger then this + notAfter time.Time +} + +func NewPlanner(notAfter time.Time) Planner { + return Planner{notAfter: notAfter} +} + +func (p Planner) Plan(tsdbMetas map[string]metadata.Meta, parquetMetas map[string]schema.Meta) (Plan, bool) { + var start, end time.Time + + for _, v := range tsdbMetas { + bStart, bEnd := time.UnixMilli(v.MinTime).UTC(), time.UnixMilli(v.MaxTime).UTC() + if start.IsZero() || bStart.Before(start) { + start = bStart + } + if end.IsZero() || bEnd.After(end) { + end = bEnd + } + } + + start, end = util.BeginOfDay(start), util.BeginOfDay(end) + + var first time.Time + + // find first block not covered by parquetMetas +L: + for next := start.UTC(); !next.After(end); next = next.AddDate(0, 0, 1).UTC() { + for _, m := range parquetMetas { + if next.Equal(time.UnixMilli(m.Mint)) { + // we already cover this day + continue L + } + } + first = next + break + } + + if first.IsZero() || first.After(p.notAfter) { + return Plan{}, false + } + + overlappingMetas := overlappingBlockMetas(tsdbMetas, first) + if len(overlappingMetas) == 0 { + return Plan{}, false + } + last := util.BeginOfDay(time.UnixMilli(overlappingMetas[len(overlappingMetas)-1].MaxTime).UTC()) + + convertForDays := make([]time.Time, 0) + convertForDays = append(convertForDays, first) + + if !first.Equal(last) { + for next := first.AddDate(0, 0, 1).UTC(); !next.Equal(last); next = next.AddDate(0, 0, 1).UTC() { + convertForDays = append(convertForDays, next) + } + } else if last.Equal(end) { + // if we only convert one date and if that date is the end of the range that our TSDB + // blocks cover fully - we just wait for more data, it could be that we get more data later + // for this day. + return Plan{}, false + } + + // NOTE: if the metas cover more time, then we could amortize downloads by just converting all + // dates the cover here. Think for example blocks in object storage that were compacted to two + // weeks. We should only do this for full days that are covered by the downloaded blocks though. + // We should also only consider dates that are not covered by parquet files already. + + return Plan{ + Download: overlappingMetas, + ConvertForDates: convertForDays, + }, true +} + +func overlappingBlockMetas(metas map[string]metadata.Meta, date time.Time) []metadata.Meta { + res := make([]metadata.Meta, 0) + for _, m := range metas { + if date.AddDate(0, 0, 1).UnixMilli() >= m.MinTime && date.UnixMilli() <= m.MaxTime { + res = append(res, m) + } + } + sort.Slice(res, func(i, j int) bool { + return res[i].MaxTime <= res[j].MaxTime + }) + return res +} diff --git a/convert/plan_test.go b/convert/plan_test.go new file mode 100644 index 0000000..9c78fb5 --- /dev/null +++ b/convert/plan_test.go @@ -0,0 +1,248 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package convert + +import ( + "math" + "slices" + "testing" + "time" + + "github.com/oklog/ulid/v2" + "github.com/prometheus/prometheus/tsdb" + "github.com/thanos-io/thanos/pkg/block/metadata" + + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +func TestPlanner(t *testing.T) { + for _, tc := range []struct { + name string + + notAfter time.Time + tsdbMetas map[string]metadata.Meta + parquetMetas map[string]schema.Meta + + expectOk bool + expectULIDS []ulid.ULID + expectDates []time.Time + }{ + { + name: "last day only partially covered", + notAfter: time.UnixMilli(math.MaxInt64), + tsdbMetas: map[string]metadata.Meta{ + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2025, time.March, 2, 11, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2025, time.March, 3, 5, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{ + "2025/03/02": { + Name: "2025/03/02", + Mint: time.Date(2025, time.March, 2, 0, 0, 0, 0, time.UTC).UnixMilli(), + Maxt: time.Date(2025, time.March, 3, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + expectOk: false, + }, + { + name: "three blocks cover a full day, previous parquet file for intial overlap", + notAfter: time.UnixMilli(math.MaxInt64), + tsdbMetas: map[string]metadata.Meta{ + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2025, time.March, 2, 11, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2025, time.March, 3, 5, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + "01JT0DWY7YB7TE9TNHY5NTAYWT": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DWY7YB7TE9TNHY5NTAYWT"), + MinTime: time.Date(2025, time.March, 3, 5, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2025, time.March, 3, 18, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + "01JT0DYXS7CJ7VCFH63WD1S006": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DYXS7CJ7VCFH63WD1S006"), + MinTime: time.Date(2025, time.March, 3, 18, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2025, time.March, 4, 4, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{ + "2025/03/02": { + Name: "2025/03/02", + Mint: time.Date(2025, time.March, 2, 0, 0, 0, 0, time.UTC).UnixMilli(), + Maxt: time.Date(2025, time.March, 3, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + expectOk: true, + expectULIDS: []ulid.ULID{ + ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + ulid.MustParse("01JT0DWY7YB7TE9TNHY5NTAYWT"), + ulid.MustParse("01JT0DYXS7CJ7VCFH63WD1S006"), + }, + expectDates: []time.Time{ + time.Date(2025, time.March, 3, 0, 0, 0, 0, time.UTC), + }, + }, + { + name: "we dont convert blocks that are too young still", + notAfter: time.Date(2025, time.March, 1, 0, 0, 0, 0, time.UTC), + tsdbMetas: map[string]metadata.Meta{ + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2025, time.March, 2, 0, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2025, time.March, 3, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{}, + expectOk: false, + }, + { + name: "we have all blocks already", + notAfter: time.UnixMilli(math.MaxInt64), + tsdbMetas: map[string]metadata.Meta{ + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 4, 12, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 24, 18, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + "01JU0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JU0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 24, 18, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 25, 12, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{ + "2020/01/04": { + Name: "2020/01/04", + Mint: time.Date(2020, time.January, 4, 0, 0, 0, 0, time.UTC).UnixMilli(), + Maxt: time.Date(2020, time.January, 5, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + expectOk: true, + expectULIDS: []ulid.ULID{ + ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + }, + expectDates: []time.Time{ + time.Date(2020, time.January, 5, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 6, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 7, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 8, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 9, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 10, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 11, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 12, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 13, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 14, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 15, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 16, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 17, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 18, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 19, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 20, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 21, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 22, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 23, 0, 0, 0, 0, time.UTC), + }, + }, + { + name: "we can amoritize downloads by converting as many blocks as possible", + notAfter: time.UnixMilli(math.MaxInt64), + tsdbMetas: map[string]metadata.Meta{ + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 4, 12, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 7, 18, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{}, + expectOk: true, + expectULIDS: []ulid.ULID{ + ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + }, + expectDates: []time.Time{ + time.Date(2020, time.January, 4, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 5, 0, 0, 0, 0, time.UTC), + time.Date(2020, time.January, 6, 0, 0, 0, 0, time.UTC), + }, + }, + { + name: "upload gap does not stall converter", + notAfter: time.UnixMilli(math.MaxInt64), + tsdbMetas: map[string]metadata.Meta{ + "01JS0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JS0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 4, 12, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 5, 18, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + "01JT0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 5, 18, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 6, 12, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + "01JU0DPYGA1HPW5RBZ1KBXCNXK": { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JU0DPYGA1HPW5RBZ1KBXCNXK"), + MinTime: time.Date(2020, time.January, 7, 18, 0, 0, 0, time.UTC).UnixMilli(), + MaxTime: time.Date(2020, time.January, 8, 6, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + }, + parquetMetas: map[string]schema.Meta{ + "2020/01/04": { + Name: "2020/01/04", + Mint: time.Date(2020, time.January, 4, 0, 0, 0, 0, time.UTC).UnixMilli(), + Maxt: time.Date(2020, time.January, 5, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + "2020/01/05": { + Name: "2020/01/05", + Mint: time.Date(2020, time.January, 5, 0, 0, 0, 0, time.UTC).UnixMilli(), + Maxt: time.Date(2020, time.January, 6, 0, 0, 0, 0, time.UTC).UnixMilli(), + }, + }, + expectOk: true, + expectULIDS: []ulid.ULID{ + ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + }, + expectDates: []time.Time{ + time.Date(2020, time.January, 6, 0, 0, 0, 0, time.UTC), + }, + }, + } { + t.Run(tc.name, func(tt *testing.T) { + plan, ok := NewPlanner(tc.notAfter).Plan(tc.tsdbMetas, tc.parquetMetas) + if tc.expectOk != ok { + tt.Fatalf("expected %t to equal %t", ok, tc.expectOk) + } + if !slices.Equal(tc.expectDates, plan.ConvertForDates) { + tt.Fatalf("expected %q to equal %q", plan.ConvertForDates, tc.expectDates) + } + if !slices.EqualFunc(tc.expectULIDS, plan.Download, func(l ulid.ULID, r metadata.Meta) bool { + return l == r.ULID + }) { + tt.Fatalf("expected %+v, to equal %q", plan.Download, tc.expectULIDS) + } + }) + } +} diff --git a/convert/sort.go b/convert/sort.go deleted file mode 100644 index 07f719d..0000000 --- a/convert/sort.go +++ /dev/null @@ -1,300 +0,0 @@ -// Copyright (c) 2025 Cloudflare, Inc. -// Licensed under the Apache 2.0 license found in the LICENSE file or at: -// https://opensource.org/licenses/Apache-2.0 - -package convert - -import ( - "container/heap" - "fmt" - "io" - "slices" - "sort" - - "github.com/parquet-go/parquet-go" -) - -type sortingWriter struct { - in io.ReadWriteSeeker - - schema *parquet.Schema - buffer *parquet.RowBuffer[any] - writer *parquet.GenericWriter[any] - cols []parquet.SortingColumn - - n int - flushThreshold int -} - -func newSortingWriter(in io.ReadWriteSeeker, p parquet.BufferPool, schema *parquet.Schema, flushThreshold int, cols ...parquet.SortingColumn) *sortingWriter { - return &sortingWriter{ - in: in, - schema: schema, - cols: cols, - flushThreshold: flushThreshold, - buffer: parquet.NewRowBuffer[any](schema, parquet.SortingRowGroupConfig( - parquet.SortingColumns(cols...), - parquet.SortingBuffers(p), - ), - ), - writer: parquet.NewGenericWriter[any](in, schema, parquet.SortingWriterConfig( - parquet.SortingColumns(cols...), - parquet.SortingBuffers(p), - ), - ), - } -} - -var _ parquet.RowWriter = &sortingWriter{} - -func (w *sortingWriter) WriteRows(buf []parquet.Row) (int, error) { - n, err := w.buffer.WriteRows(buf) - if err != nil { - return 0, err - } - w.n += n - if w.n > w.flushThreshold { - sort.Sort(w.buffer) - rows := w.buffer.Rows() - defer rows.Close() - if _, err := parquet.CopyRows(w.writer, rows); err != nil { - return 0, err - } - if err := w.writer.Flush(); err != nil { - return 0, err - } - w.buffer.Reset() - w.n = 0 - } - return n, nil -} - -func (w *sortingWriter) Flush() error { - sort.Sort(w.buffer) - rows := w.buffer.Rows() - defer rows.Close() - if _, err := parquet.CopyRows(w.writer, rows); err != nil { - return err - } - return w.writer.Close() -} - -func (w *sortingWriter) RowReader() (parquet.RowReader, error) { - if _, err := w.in.Seek(0, io.SeekStart); err != nil { - return nil, err - } - sz, err := sizeFromSeeker(w.in) - if err != nil { - return nil, err - } - pf, err := parquet.OpenFile(newReaderAt(w.in), sz) - if err != nil { - return nil, err - } - rrs := make([]parquet.RowReader, 0) - for _, rg := range pf.RowGroups() { - rrs = append(rrs, rg.Rows()) - } - return mergeRowReaders(rrs, w.schema.Comparator(w.cols...)), nil -} - -// Taken from https://github.com/parquet-go/parquet-go/blob/main/merge.go -// This was necessary to fix corruption that happened because the head was not cloned, though maybe we have been using the library wrong here and this is not actually necessary. -type mergedRowReader struct { - compare func(parquet.Row, parquet.Row) int - readers []*bufferedRowReader - initialized bool -} - -func mergeRowReaders(readers []parquet.RowReader, compare func(parquet.Row, parquet.Row) int) *mergedRowReader { - return &mergedRowReader{ - compare: compare, - readers: makeBufferedRowReaders(len(readers), func(i int) parquet.RowReader { return readers[i] }), - } -} - -func makeBufferedRowReaders(numReaders int, readerAt func(int) parquet.RowReader) []*bufferedRowReader { - buffers := make([]bufferedRowReader, numReaders) - readers := make([]*bufferedRowReader, numReaders) - - for i := range readers { - buffers[i].rows = readerAt(i) - readers[i] = &buffers[i] - } - return readers -} - -func (m *mergedRowReader) initialize() error { - for i, r := range m.readers { - switch err := r.read(); err { - case nil: - case io.EOF: - m.readers[i] = nil - default: - m.readers = nil - return err - } - } - - n := 0 - for _, r := range m.readers { - if r != nil { - m.readers[n] = r - n++ - } - } - - toclear := m.readers[n:] - for i := range toclear { - toclear[i] = nil - } - - m.readers = m.readers[:n] - heap.Init(m) - return nil -} - -func (m *mergedRowReader) Close() { - for _, r := range m.readers { - r.close() - } - m.readers = nil -} - -func (m *mergedRowReader) ReadRows(rows []parquet.Row) (n int, err error) { - if !m.initialized { - m.initialized = true - - if err := m.initialize(); err != nil { - return 0, err - } - } - for n < len(rows) && len(m.readers) != 0 { - r := m.readers[0] - h := r.head().Clone() - - rows[n] = slices.Grow(rows[n], len(h))[:len(h)] - copy(rows[n], h) - n++ - - if err := r.next(); err != nil { - if err != io.EOF { - return n, err - } - heap.Pop(m) - } else { - heap.Fix(m, 0) - } - } - - if len(m.readers) == 0 { - err = io.EOF - } - return n, err -} - -func (m *mergedRowReader) Less(i, j int) bool { - return m.compare(m.readers[i].head(), m.readers[j].head()) < 0 -} - -func (m *mergedRowReader) Len() int { - return len(m.readers) -} - -func (m *mergedRowReader) Swap(i, j int) { - m.readers[i], m.readers[j] = m.readers[j], m.readers[i] -} - -func (m *mergedRowReader) Push(_ interface{}) { - panic("NOT IMPLEMENTED") -} - -func (m *mergedRowReader) Pop() interface{} { - i := len(m.readers) - 1 - r := m.readers[i] - m.readers = m.readers[:i] - return r -} - -type bufferedRowReader struct { - rows parquet.RowReader - off int32 - end int32 - buf [64]parquet.Row -} - -func (r *bufferedRowReader) head() parquet.Row { - return r.buf[r.off] -} - -func (r *bufferedRowReader) next() error { - if r.off++; r.off == r.end { - r.off = 0 - r.end = 0 - return r.read() - } - return nil -} - -func (r *bufferedRowReader) read() error { - if r.rows == nil { - return io.EOF - } - n, err := r.rows.ReadRows(r.buf[r.end:]) - if err != nil && n == 0 { - return err - } - r.end += int32(n) - return nil -} - -func (r *bufferedRowReader) close() { - r.rows = nil - r.off = 0 - r.end = 0 -} - -func sizeFromSeeker(seek io.Seeker) (int64, error) { - pos, err := seek.Seek(0, io.SeekCurrent) - if err != nil { - return 0, err - } - end, err := seek.Seek(0, io.SeekEnd) - if err != nil { - return 0, err - } - size := end - pos - pos1, err := seek.Seek(pos, io.SeekStart) - if err != nil { - return 0, err - } - if pos1 != pos { - return 0, fmt.Errorf("unable to restore seek position: %d != %d", pos1, pos) - } - return size, nil -} - -type readerAt struct { - reader io.ReadSeeker - offset int64 -} - -func (r *readerAt) ReadAt(b []byte, off int64) (int, error) { - if r.offset < 0 || off != r.offset { - off, err := r.reader.Seek(off, io.SeekStart) - if err != nil { - return 0, err - } - r.offset = off - } - n, err := r.reader.Read(b) - r.offset += int64(n) - return n, err -} - -func newReaderAt(r io.ReadSeeker) io.ReaderAt { - if rr, ok := r.(io.ReaderAt); ok { - return rr - } - return &readerAt{reader: r, offset: -1} -} diff --git a/convert/tsdb.go b/convert/tsdb.go index 4a4d013..458dca2 100644 --- a/convert/tsdb.go +++ b/convert/tsdb.go @@ -6,17 +6,19 @@ package convert import ( "context" + "errors" "fmt" "io" "slices" + "strings" - "github.com/hashicorp/go-multierror" "github.com/parquet-go/parquet-go" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/cloudflare/parquet-tsdb-poc/internal/encoding" "github.com/cloudflare/parquet-tsdb-poc/schema" ) @@ -27,24 +29,41 @@ type indexRowReader struct { seriesSet storage.ChunkSeriesSet - rowBuilder *parquet.RowBuilder schema *parquet.Schema + rowBuilder *parquet.RowBuilder + + concurrency int - chunksColumn0 int - chunksColumn1 int - chunksColumn2 int + chunksColumn0 int + chunksColumn1 int + chunksColumn2 int + labelIndexColumn int + labelHashColumn int +} - m map[string]map[string]struct{} +type indexReaderOpts struct { + sortLabels []string + concurrency int } var _ parquet.RowReader = &indexRowReader{} -func newIndexRowReader(ctx context.Context, mint, maxt int64, blks []Convertible) (*indexRowReader, error) { +func newIndexRowReader(ctx context.Context, mint, maxt int64, blks []Convertable, opts indexReaderOpts) (*indexRowReader, error) { var ( lbls = make([]string, 0) seriesSets = make([]storage.ChunkSeriesSet, 0, len(blks)) closers = make([]io.Closer, 0, len(blks)) ) + + compareFunc := func(a, b labels.Labels) int { + for _, lb := range opts.sortLabels { + if c := strings.Compare(a.Get(lb), b.Get(lb)); c != 0 { + return c + } + } + return labels.Compare(a, b) + } + for _, blk := range blks { indexr, err := blk.Index() if err != nil { @@ -70,13 +89,13 @@ func newIndexRowReader(ctx context.Context, mint, maxt int64, blks []Convertible } lbls = append(lbls, lblns...) - postings := tsdb.AllSortedPostings(ctx, indexr) + postings := sortedPostings(ctx, indexr, compareFunc) seriesSet := tsdb.NewBlockChunkSeriesSet(blk.Meta().ULID, indexr, chunkr, tombsr, postings, mint, maxt, false) seriesSets = append(seriesSets, seriesSet) } slices.Sort(lbls) - cseriesSet := storage.NewMergeChunkSeriesSet(seriesSets, storage.NewConcatenatingChunkSeriesMerger()) + cseriesSet := newMergeChunkSeriesSet(seriesSets, compareFunc, storage.NewConcatenatingChunkSeriesMerger()) s := schema.BuildSchemaFromLabels(slices.Compact(lbls)) return &indexRowReader{ @@ -84,14 +103,16 @@ func newIndexRowReader(ctx context.Context, mint, maxt int64, blks []Convertible seriesSet: cseriesSet, closers: closers, - rowBuilder: parquet.NewRowBuilder(s), schema: s, + rowBuilder: parquet.NewRowBuilder(s), - chunksColumn0: columnIDForKnownColumn(s, schema.ChunksColumn0), - chunksColumn1: columnIDForKnownColumn(s, schema.ChunksColumn1), - chunksColumn2: columnIDForKnownColumn(s, schema.ChunksColumn2), + concurrency: opts.concurrency, - m: make(map[string]map[string]struct{}), + chunksColumn0: columnIDForKnownColumn(s, schema.ChunksColumn0), + chunksColumn1: columnIDForKnownColumn(s, schema.ChunksColumn1), + chunksColumn2: columnIDForKnownColumn(s, schema.ChunksColumn2), + labelIndexColumn: columnIDForKnownColumn(s, schema.LabelIndexColumn), + labelHashColumn: columnIDForKnownColumn(s, schema.LabelHashColumn), }, nil } @@ -101,21 +122,19 @@ func columnIDForKnownColumn(schema *parquet.Schema, columnName string) int { } func (rr *indexRowReader) Close() error { - err := &multierror.Error{} + errs := make([]error, 0) for i := range rr.closers { - err = multierror.Append(err, rr.closers[i].Close()) + if err := rr.closers[i].Close(); err != nil { + errs = append(errs, fmt.Errorf("unable to close %q-th closer: %w", i, err)) + } } - return err.ErrorOrNil() + return errors.Join(errs...) } func (rr *indexRowReader) Schema() *parquet.Schema { return rr.schema } -func (rr *indexRowReader) NameLabelMapping() map[string]map[string]struct{} { - return rr.m -} - func (rr *indexRowReader) ReadRows(buf []parquet.Row) (int, error) { select { case <-rr.ctx.Done(): @@ -123,39 +142,66 @@ func (rr *indexRowReader) ReadRows(buf []parquet.Row) (int, error) { default: } - var it chunks.Iterator + type chkBytesOrError struct { + chkBytes [schema.ChunkColumnsPerDay][]byte + err error + } + type chunkSeriesPromise struct { + s storage.ChunkSeries + c chan chkBytesOrError + } - i := 0 - for i < len(buf) && rr.seriesSet.Next() { - rr.rowBuilder.Reset() - s := rr.seriesSet.At() - it = s.Iterator(it) + chunkSeriesC := make(chan chunkSeriesPromise, rr.concurrency) - chkBytes, err := collectChunks(it) - if err != nil { - return i, fmt.Errorf("unable to collect chunks: %s", err) - } + go func() { + defer close(chunkSeriesC) + for j := 0; j < len(buf) && rr.seriesSet.Next(); j++ { + s := rr.seriesSet.At() + it := s.Iterator(nil) - // skip series that have no chunks in the requested time - if allChunksEmpty(chkBytes) { - continue + promise := chunkSeriesPromise{ + s: s, + c: make(chan chkBytesOrError, 1), + } + + chunkSeriesC <- promise + + go func() { + chkBytes, err := collectChunks(it) + promise.c <- chkBytesOrError{chkBytes: chkBytes, err: err} + close(promise.c) + }() } + }() - metricName := s.Labels().Get(labels.MetricName) - nameMap, ok := rr.m[metricName] - if !ok { - nameMap = make(map[string]struct{}) + colIdxSlice := make([]int, 0) + i, j := 0, 0 + for promise := range chunkSeriesC { + j++ + + rr.rowBuilder.Reset() + colIdxSlice = colIdxSlice[:0] + + chkBytesOrError := <-promise.c + if err := chkBytesOrError.err; err != nil { + return i, err } - rr.m[metricName] = nameMap - s.Labels().Range(func(l labels.Label) { + chkBytes := chkBytesOrError.chkBytes + chkLbls := promise.s.Labels() + + chkLbls.Range(func(l labels.Label) { colName := schema.LabelNameToColumn(l.Name) lc, _ := rr.schema.Lookup(colName) rr.rowBuilder.Add(lc.ColumnIndex, parquet.ValueOf(l.Value)) - if l.Name != labels.MetricName { - nameMap[colName] = struct{}{} - } + // we need to address for projecting chunk columns away later so we need to correct for the offset here + colIdxSlice = append(colIdxSlice, lc.ColumnIndex-schema.ChunkColumnsPerDay) }) + rr.rowBuilder.Add(rr.labelIndexColumn, parquet.ValueOf(encoding.EncodeLabelColumnIndex(colIdxSlice))) + rr.rowBuilder.Add(rr.labelHashColumn, parquet.ValueOf(chkLbls.Hash())) + if allChunksEmpty(chkBytes) { + continue + } for idx, chk := range chkBytes { if len(chk) == 0 { continue @@ -169,11 +215,59 @@ func (rr *indexRowReader) ReadRows(buf []parquet.Row) (int, error) { rr.rowBuilder.Add(rr.chunksColumn2, parquet.ValueOf(chk)) } } + buf[i] = rr.rowBuilder.AppendRow(buf[i][:0]) i++ } - if i < len(buf) { + if j < len(buf) { return i, io.EOF } return i, rr.seriesSet.Err() } + +func sortedPostings(ctx context.Context, indexr tsdb.IndexReader, compare func(a, b labels.Labels) int) index.Postings { + p := tsdb.AllSortedPostings(ctx, indexr) + + type s struct { + ref storage.SeriesRef + labels labels.Labels + } + series := make([]s, 0, 128) + + lb := labels.NewScratchBuilder(10) + for p.Next() { + select { + case <-ctx.Done(): + return index.ErrPostings(ctx.Err()) + default: + } + lb.Reset() + if err := indexr.Series(p.At(), &lb, nil); err != nil { + return index.ErrPostings(fmt.Errorf("unable to expand series: %w", err)) + } + series = append(series, s{labels: lb.Labels(), ref: p.At()}) + } + if err := p.Err(); err != nil { + return index.ErrPostings(fmt.Errorf("unable to expand postings: %w", err)) + } + + slices.SortFunc(series, func(a, b s) int { + return compare(a.labels, b.labels) + }) + + // Convert back to list. + ep := make([]storage.SeriesRef, 0, len(series)) + for _, p := range series { + ep = append(ep, p.ref) + } + return index.NewListPostings(ep) +} + +func allChunksEmpty(chkBytes [schema.ChunkColumnsPerDay][]byte) bool { + for _, chk := range chkBytes { + if len(chk) != 0 { + return false + } + } + return true +} diff --git a/db/block.go b/db/block.go index cd5cae1..17f68c1 100644 --- a/db/block.go +++ b/db/block.go @@ -6,35 +6,61 @@ package db import ( "context" + "errors" "fmt" - "slices" + "sync" + "time" - "github.com/hashicorp/go-multierror" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" + "go.opentelemetry.io/otel/attribute" + "golang.org/x/sync/errgroup" + + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" + "github.com/cloudflare/parquet-tsdb-poc/schema" ) type Block struct { - meta Meta + meta schema.Meta shards []*Shard } -type Meta struct { - Name string - Mint, Maxt int64 - Shards int64 - ColumnsForName map[string][]string +func NewBlock(meta schema.Meta, shards ...*Shard) *Block { + return &Block{meta: meta, shards: shards} +} + +func (blk *Block) Meta() schema.Meta { + return blk.meta } func (blk *Block) Timerange() (int64, int64) { return blk.meta.Mint, blk.meta.Maxt } -func (blk *Block) Queryable(extlabels labels.Labels, replicaLabelNames []string) storage.Queryable { +func (blk *Block) Queryable( + extlabels labels.Labels, + replicaLabelNames []string, + selectChunkBytesQuota *limits.Quota, + selectRowCountQuota *limits.Quota, + selectChunkPartitionMaxRange uint64, + selectChunkPartitionMaxGap uint64, + selectChunkPartitionMaxConcurrency int, +) storage.Queryable { qs := make([]storage.Queryable, 0, len(blk.shards)) for _, shard := range blk.shards { - qs = append(qs, shard.Queryable(extlabels, replicaLabelNames)) + qs = append(qs, shard.Queryable( + extlabels, + replicaLabelNames, + selectChunkBytesQuota, + selectRowCountQuota, + selectChunkPartitionMaxRange, + selectChunkPartitionMaxGap, + selectChunkPartitionMaxConcurrency, + )) } return &BlockQueryable{extlabels: extlabels, shards: qs} } @@ -64,39 +90,107 @@ type BlockQuerier struct { } func (q BlockQuerier) Close() error { - var err *multierror.Error - for _, q := range q.shards { - err = multierror.Append(err, q.Close()) + errs := make([]error, 0) + for i, q := range q.shards { + if err := q.Close(); err != nil { + errs = append(errs, fmt.Errorf("unable to close shard %q: %w", i, err)) + } } - return err.ErrorOrNil() + return errors.Join(errs...) } -func (q BlockQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, ms ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q BlockQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { var annos annotations.Annotations + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + res := make([]string, 0) - for _, shrd := range q.shards { - lvals, lannos, err := shrd.LabelValues(ctx, name, hints, ms...) - if err != nil { - return nil, nil, fmt.Errorf("unable to query label values for shard: %w", err) - } - annos = annos.Merge(lannos) - res = append(res, lvals...) + for _, s := range q.shards { + g.Go(func() error { + lvals, lannos, err := s.LabelValues(ctx, name, hints, matchers...) + if err != nil { + return fmt.Errorf("unable to query label values for shard: %w", err) + } + annos = annos.Merge(lannos) + mu.Lock() + res = append(res, lvals...) + mu.Unlock() + + return nil + }) } - slices.Sort(res) - return slices.Compact(res), annos, nil + if err := g.Wait(); err != nil { + return nil, nil, fmt.Errorf("unable to query label values: %w", err) + } + + limit := hints.Limit + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + return res, annos, nil } -func (BlockQuerier) LabelNames(context.Context, *storage.LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { - // TODO - return nil, nil, nil +func (q BlockQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + var annos annotations.Annotations + + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + + res := make([]string, 0) + for _, s := range q.shards { + g.Go(func() error { + lvals, lannos, err := s.LabelNames(ctx, hints, matchers...) + if err != nil { + return fmt.Errorf("unable to query label names for shard: %w", err) + } + annos = annos.Merge(lannos) + mu.Lock() + res = append(res, lvals...) + mu.Unlock() + + return nil + }) + } + if err := g.Wait(); err != nil { + return nil, nil, fmt.Errorf("unable to query label values: %w", err) + } + + limit := hints.Limit + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + return res, annos, nil } func (q BlockQuerier) Select(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + return newLazySeriesSet(ctx, q.selectFn, sorted, hints, matchers...) +} + +func (q BlockQuerier) selectFn(ctx context.Context, _ bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + ctx, span := tracing.Tracer().Start(ctx, "Select Block") + defer span.End() + + span.SetAttributes(attribute.Bool("sorted", true)) + span.SetAttributes(attribute.StringSlice("matchers", matchersToStringSlice(matchers))) + span.SetAttributes(attribute.Int("block.shards", len(q.shards))) + span.SetAttributes(attribute.String("block.mint", time.UnixMilli(q.mint).String())) + span.SetAttributes(attribute.String("block.maxt", time.UnixMilli(q.maxt).String())) + sss := make([]storage.SeriesSet, 0, len(q.shards)) for _, q := range q.shards { - sss = append(sss, q.Select(ctx, sorted, hints, matchers...)) + // always sort since we need to merge later anyhow + sss = append(sss, q.Select(ctx, true, hints, matchers...)) + } + if len(sss) == 0 { + return storage.EmptySeriesSet() } - return newVerticalSeriesSet(sss...) + return storage.NewMergeSeriesSet(sss, hints.Limit, storage.ChainedSeriesMerge) } diff --git a/db/db.go b/db/db.go index 1b25207..c7729e0 100644 --- a/db/db.go +++ b/db/db.go @@ -6,20 +6,26 @@ package db import ( "context" + "errors" "fmt" "math" - "slices" + "sync" - "github.com/hashicorp/go-multierror" + "github.com/alecthomas/units" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" + "go.opentelemetry.io/otel/attribute" + "golang.org/x/sync/errgroup" + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" ) // DB is a horizontal partitioning of multiple non-overlapping blocks that are -// aligned to 24h and span exactly 24h. +// aligned to 24h and span exactely 24h. type DB struct { syncer syncer extLabels labels.Labels @@ -67,23 +73,80 @@ func (db *DB) Extlabels() labels.Labels { return db.extLabels } -// Queryable returns a storage.Queryable to evaluate queries with. -func (db *DB) Queryable() storage.Queryable { - return &DBQueryable{ - blocks: db.syncer.Blocks(), - extLabels: db.extLabels, +type queryableConfig struct { + replicaLabelsNames []string + selectChunkBytesQuota *limits.Quota + selectRowCountQuota *limits.Quota + + selectChunkPartitionMaxRange uint64 + selectChunkPartitionMaxGap uint64 + selectChunkPartitionMaxConcurrency int +} + +type QueryableOption func(*queryableConfig) + +func DropReplicaLabels(s ...string) QueryableOption { + return func(cfg *queryableConfig) { + cfg.replicaLabelsNames = append(cfg.replicaLabelsNames, s...) } } -// ReplicaQueryable returns a storage.Queryable that drops replica labels at runtime. Replica labels are +func SelectChunkBytesQuota(maxBytes units.Base2Bytes) QueryableOption { + return func(cfg *queryableConfig) { + cfg.selectChunkBytesQuota = limits.NewQuota(int64(maxBytes)) + } +} + +func SelectRowCountQuota(maxRows int64) QueryableOption { + return func(cfg *queryableConfig) { + cfg.selectRowCountQuota = limits.NewQuota(int64(maxRows)) + } +} + +func SelectChunkPartitionMaxRange(maxRange units.Base2Bytes) QueryableOption { + return func(cfg *queryableConfig) { + cfg.selectChunkPartitionMaxRange = uint64(maxRange) + } +} + +func SelectChunkPartitionMaxGap(maxGap units.Base2Bytes) QueryableOption { + return func(cfg *queryableConfig) { + cfg.selectChunkPartitionMaxGap = uint64(maxGap) + } +} + +func SelectChunkPartitionMaxConcurrency(n int) QueryableOption { + return func(cfg *queryableConfig) { + cfg.selectChunkPartitionMaxConcurrency = n + } +} + +// Queryable returns a storage.Queryable that drops replica labels at runtime. Replica labels are // labels that identify a replica, i.e. one member of an HA pair of Prometheus servers. Thanos // might request at query time to drop those labels so that we can deduplicate results into one view. // Common replica labels are 'prometheus', 'host', etc. -func (db *DB) ReplicaQueryable(replicaLabelNames []string) storage.Queryable { +// It also enforces various quotas over its lifetime. +func (db *DB) Queryable(opts ...QueryableOption) storage.Queryable { + cfg := queryableConfig{ + selectChunkBytesQuota: limits.UnlimitedQuota(), + selectRowCountQuota: limits.UnlimitedQuota(), + selectChunkPartitionMaxRange: math.MaxUint64, + selectChunkPartitionMaxGap: math.MaxUint64, + selectChunkPartitionMaxConcurrency: 0, + } + for i := range opts { + opts[i](&cfg) + } + return &DBQueryable{ - blocks: db.syncer.Blocks(), - extLabels: db.extLabels, - replicaLabelNames: replicaLabelNames, + blocks: db.syncer.Blocks(), + extLabels: db.extLabels, + replicaLabelNames: cfg.replicaLabelsNames, + selectChunkBytesQuota: cfg.selectChunkBytesQuota, + selectRowCountQuota: cfg.selectRowCountQuota, + selectChunkPartitionMaxRange: cfg.selectChunkPartitionMaxRange, + selectChunkPartitionMaxGap: cfg.selectChunkPartitionMaxGap, + selectChunkPartitionMaxConcurrency: cfg.selectChunkPartitionMaxConcurrency, } } @@ -96,6 +159,22 @@ type DBQueryable struct { // replicaLabelNames are names of labels that identify replicas, they are dropped // after extLabels were applied. replicaLabelNames []string + + // selectChunkBytesQuota is the limit of bytes that "Select" calls can fetch from chunk columns. + selectChunkBytesQuota *limits.Quota + + // selectRowCountQuota is the limit of rows that "Select" calls can touch. + selectRowCountQuota *limits.Quota + + // selectChunkPartitionMaxRange is the maximum range of chunk pages that get coalesced into a + // range that is concurrently scheduled to be fetched from object storage. + selectChunkPartitionMaxRange uint64 + + // selectChunkPartitionMaxGap is the maximum gap that we tolerate when coalescing nearby pages into ranges. + selectChunkPartitionMaxGap uint64 + + // selectChunkPartitionMaxConcurrency is the maximum amount of parallel object storage requests we run per select. + selectChunkPartitionMaxConcurrency int } func (db *DBQueryable) Querier(mint, maxt int64) (storage.Querier, error) { @@ -106,9 +185,17 @@ func (db *DBQueryable) Querier(mint, maxt int64) (storage.Querier, error) { continue } start, end := util.Intersection(mint, maxt, bmint, bmaxt) - q, err := blk.Queryable(db.extLabels, db.replicaLabelNames).Querier(start, end) + q, err := blk.Queryable( + db.extLabels, + db.replicaLabelNames, + db.selectChunkBytesQuota, + db.selectRowCountQuota, + db.selectChunkPartitionMaxRange, + db.selectChunkPartitionMaxGap, + db.selectChunkPartitionMaxConcurrency, + ).Querier(start, end) if err != nil { - return nil, fmt.Errorf("unable to get block querier: %s", err) + return nil, fmt.Errorf("unable to get block querier: %w", err) } qs = append(qs, q) } @@ -124,33 +211,85 @@ type DBQuerier struct { var _ storage.Querier = &DBQuerier{} func (q DBQuerier) Close() error { - var err *multierror.Error - for _, q := range q.blocks { - err = multierror.Append(err, q.Close()) + errs := make([]error, 0) + for i, q := range q.blocks { + if err := q.Close(); err != nil { + errs = append(errs, fmt.Errorf("unable to close block %q: %w", i, err)) + } } - return err.ErrorOrNil() + return errors.Join(errs...) } -func (q DBQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, ms ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q DBQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { var annos annotations.Annotations + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + res := make([]string, 0) for _, blk := range q.blocks { - lvals, lannos, err := blk.LabelValues(ctx, name, hints, ms...) - if err != nil { - return nil, nil, fmt.Errorf("unable to query label values for block: %w", err) - } - annos = annos.Merge(lannos) - res = append(res, lvals...) + g.Go(func() error { + lvals, lannos, err := blk.LabelValues(ctx, name, hints, matchers...) + if err != nil { + return fmt.Errorf("unable to query label values for block: %w", err) + } + annos = annos.Merge(lannos) + mu.Lock() + res = append(res, lvals...) + mu.Unlock() + + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, nil, fmt.Errorf("unable to query label values: %w", err) } - slices.Sort(res) - return slices.Compact(res), annos, nil + limit := hints.Limit + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + return res, annos, nil } -func (DBQuerier) LabelNames(context.Context, *storage.LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { - // TODO - return nil, nil, nil +func (q DBQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + var annos annotations.Annotations + + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + + res := make([]string, 0) + for _, blk := range q.blocks { + g.Go(func() error { + lnames, lannos, err := blk.LabelNames(ctx, hints, matchers...) + if err != nil { + return fmt.Errorf("unable to query label names for block: %w", err) + } + annos = annos.Merge(lannos) + mu.Lock() + res = append(res, lnames...) + mu.Unlock() + + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, nil, fmt.Errorf("unable to query label names: %w", err) + } + + limit := hints.Limit + + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + return res, annos, nil } func (q DBQuerier) Select(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { @@ -158,6 +297,13 @@ func (q DBQuerier) Select(ctx context.Context, sorted bool, hints *storage.Selec } func (q DBQuerier) selectFn(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + ctx, span := tracing.Tracer().Start(ctx, "Select DB") + defer span.End() + + span.SetAttributes(attribute.Bool("sorted", sorted)) + span.SetAttributes(attribute.StringSlice("matchers", matchersToStringSlice(matchers))) + span.SetAttributes(attribute.Int("block.shards", len(q.blocks))) + // If we need to merge multiple series sets vertically we need them sorted sorted = sorted || len(q.blocks) > 1 @@ -169,5 +315,5 @@ func (q DBQuerier) selectFn(ctx context.Context, sorted bool, hints *storage.Sel if len(sss) == 0 { return storage.EmptySeriesSet() } - return storage.NewMergeSeriesSet(sss, storage.ChainedSeriesMerge) + return storage.NewMergeSeriesSet(sss, hints.Limit, storage.ChainedSeriesMerge) } diff --git a/db/db_bench_test.go b/db/db_bench_test.go new file mode 100644 index 0000000..2537d5e --- /dev/null +++ b/db/db_bench_test.go @@ -0,0 +1,300 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package db_test + +import ( + "context" + "fmt" + "io" + "math/rand" + "sync/atomic" + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/util/teststorage" + "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" +) + +type countingBucket struct { + objstore.Bucket + + nGet atomic.Int32 + nGetRange atomic.Int32 + bsGetRange atomic.Int64 +} + +func (b *countingBucket) ResetCounters() { + b.nGet.Store(0) + b.nGetRange.Store(0) + b.bsGetRange.Store(0) +} + +func (b *countingBucket) Get(ctx context.Context, name string) (io.ReadCloser, error) { + b.nGet.Add(1) + return b.Bucket.Get(ctx, name) +} + +func (b *countingBucket) GetRange(ctx context.Context, name string, off int64, length int64) (io.ReadCloser, error) { + b.nGetRange.Add(1) + b.bsGetRange.Add(length) + return b.Bucket.GetRange(ctx, name, off, length) +} + +func BenchmarkSelect(b *testing.B) { + ctx := b.Context() + + type queryableCreate func(tb testing.TB, bkt objstore.Bucket, st *teststorage.TestStorage) storage.Queryable + + for k, qc := range map[string]queryableCreate{ + "parquet": func(tb testing.TB, bkt objstore.Bucket, st *teststorage.TestStorage) storage.Queryable { + return storageToDBWithBkt(tb, st, bkt).Queryable() + }, + "prometheus": func(_ testing.TB, _ objstore.Bucket, st *teststorage.TestStorage) storage.Queryable { + return st + }, + } { + b.Run(k, func(b *testing.B) { + b.Run("80k cardinality series", func(b *testing.B) { + st := teststorage.New(b) + b.Cleanup(func() { _ = st.Close() }) + bkt, err := filesystem.NewBucket(b.TempDir()) + if err != nil { + b.Fatal("error creating bucket: ", err) + } + b.Cleanup(func() { _ = bkt.Close() }) + cbkt := &countingBucket{Bucket: bkt} + + app := st.Appender(b.Context()) + for i := range 10_000 { + for _, sc := range []string{"200", "202", "300", "404", "400", "429", "500", "503"} { + app.Append(0, labels.FromStrings("__name__", "foo", "idx", fmt.Sprintf("%d", i), "status_code", sc), 0, rand.Float64()) + } + } + if err := app.Commit(); err != nil { + b.Fatal("error committing samples: ", err) + } + + q, err := qc(b, cbkt, st).Querier(0, 120) + if err != nil { + b.Fatal("error building querier: ", err) + } + + for _, bc := range []struct { + name string + matchers []*labels.Matcher + }{ + { + name: "ShreddedByStatusCode", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "__name__", "foo"), labels.MustNewMatcher(labels.MatchRegexp, "status_code", "4..")}, + }, + { + name: "ShreddedByIdx", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "__name__", "foo"), labels.MustNewMatcher(labels.MatchRegexp, "idx", ".*2")}, + }, + { + name: "AllSeries", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "__name__", "foo")}, + }, + { + name: "SingleSeries", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "idx", "500")}, + }, + { + name: "FirstAndLastSeries", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "idx", "0|9999")}, + }, + } { + b.Run(bc.name, func(b *testing.B) { + cbkt.ResetCounters() + b.ReportAllocs() + + for b.Loop() { + ss := q.Select(ctx, true, &storage.SelectHints{}, bc.matchers...) + for ss.Next() { + s := ss.At() + it := s.Iterator(nil) + for it.Next() != chunkenc.ValNone { //revive:disable-line:empty-block + } + } + if err := ss.Err(); err != nil { + b.Error(err) + } + } + + b.ReportMetric(float64(cbkt.nGet.Load())/float64(b.N), "get/op") + b.ReportMetric(float64(cbkt.nGetRange.Load())/float64(b.N), "get_range/op") + b.ReportMetric(float64(cbkt.bsGetRange.Load())/float64(b.N), "bytes_get_range/op") + }) + } + }) + b.Run("Realistic Series", func(b *testing.B) { + b.Run(k, func(b *testing.B) { + st := teststorage.New(b) + b.Cleanup(func() { _ = st.Close() }) + + bkt, err := filesystem.NewBucket(b.TempDir()) + if err != nil { + b.Fatal("error creating bucket: ", err) + } + b.Cleanup(func() { _ = bkt.Close() }) + cbkt := &countingBucket{Bucket: bkt} + + app := st.Appender(b.Context()) + + // 5 metrics × 100 instances × 5 regions × 10 zones × 20 services × 3 environments = 1,500,000 series + metrics := 5 + instances := 100 + regions := 5 + zones := 10 + services := 20 + environments := 3 + + seriesCount := 0 + for m := range metrics { + for i := range instances { + for r := range regions { + for z := range zones { + for s := range services { + for e := range environments { + lbls := labels.FromStrings( + "__name__", fmt.Sprintf("test_metric_%d", m), + "instance", fmt.Sprintf("instance-%d", i), + "region", fmt.Sprintf("region-%d", r), + "zone", fmt.Sprintf("zone-%d", z), + "service", fmt.Sprintf("service-%d", s), + "environment", fmt.Sprintf("environment-%d", e), + ) + app.Append(0, lbls, 0, rand.Float64()) + seriesCount++ + } + } + } + } + } + } + if err := app.Commit(); err != nil { + b.Fatal("error committing samples: ", err) + } + + q, err := qc(b, cbkt, st).Querier(0, 120) + if err != nil { + b.Fatal("error building querier: ", err) + } + + for _, bc := range []struct { + name string + matchers []*labels.Matcher + }{ + { + name: "SingleMetricAllSeries", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + }, + }, + { + name: "SingleMetricReducedSeries", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchEqual, "instance", "instance-1"), + }, + }, + { + name: "SingleMetricOneSeries", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchEqual, "instance", "instance-2"), + labels.MustNewMatcher(labels.MatchEqual, "region", "region-1"), + labels.MustNewMatcher(labels.MatchEqual, "zone", "zone-3"), + labels.MustNewMatcher(labels.MatchEqual, "service", "service-10"), + labels.MustNewMatcher(labels.MatchEqual, "environment", "environment-1"), + }, + }, + { + name: "SingleMetricSparseSeries", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchEqual, "service", "service-1"), + labels.MustNewMatcher(labels.MatchEqual, "environment", "environment-0"), + }, + }, + { + name: "NonExistentSeries", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchEqual, "environment", "non-existent-environment"), + }, + }, + { + name: "MultipleMetricsRange", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "__name__", "test_metric_[1-5]"), + }, + }, + { + name: "MultipleMetricsSparse", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "__name__", "test_metric_(1|5|10|15|20)"), + }, + }, + { + name: "NegativeRegexSingleMetric", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchNotRegexp, "instance", "(instance-1.*|instance-2.*)"), + }, + }, + { + name: "NegativeRegexMultipleMetrics", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "__name__", "test_metric_[1-3]"), + labels.MustNewMatcher(labels.MatchNotRegexp, "instance", "(instance-1.*|instance-2.*)"), + }, + }, + { + name: "ExpensiveRegexSingleMetric", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "test_metric_1"), + labels.MustNewMatcher(labels.MatchRegexp, "instance", "(container-1|instance-2|container-3|instance-4|container-5)"), + }, + }, + { + name: "ExpensiveRegexMultipleMetrics", + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "__name__", "test_metric_[1-3]"), + labels.MustNewMatcher(labels.MatchRegexp, "instance", "(container-1|container-2|container-3|container-4|container-5)"), + }, + }, + } { + b.Run(bc.name, func(b *testing.B) { + cbkt.ResetCounters() + b.ResetTimer() + + for b.Loop() { + ss := q.Select(ctx, true, &storage.SelectHints{}, bc.matchers...) + for ss.Next() { + s := ss.At() + it := s.Iterator(nil) + for it.Next() != chunkenc.ValNone { //revive:disable-line:empty-block + } + } + if err := ss.Err(); err != nil { + b.Error(err) + } + } + + b.ReportMetric(float64(cbkt.nGet.Load())/float64(b.N), "get/op") + b.ReportMetric(float64(cbkt.nGetRange.Load())/float64(b.N), "get_range/op") + b.ReportMetric(float64(cbkt.bsGetRange.Load())/float64(b.N), "bytes_get_range/op") + }) + } + + }) + }) + }) + } +} diff --git a/db/db_test.go b/db/db_test.go index 78d3999..f16b3c2 100644 --- a/db/db_test.go +++ b/db/db_test.go @@ -2,13 +2,15 @@ // Licensed under the Apache 2.0 license found in the LICENSE file or at: // https://opensource.org/licenses/Apache-2.0 -package db +package db_test import ( "context" + "errors" "fmt" + "math" + "slices" "sort" - "strings" "testing" "time" @@ -18,20 +20,290 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/promqltest" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/teststorage" + "github.com/prometheus/prometheus/util/testutil" "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" "github.com/cloudflare/parquet-tsdb-poc/convert" - "github.com/cloudflare/parquet-tsdb-poc/schema" + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" + "github.com/cloudflare/parquet-tsdb-poc/locate" ) -var opts = promql.EngineOpts{ - Timeout: 1 * time.Hour, - MaxSamples: 1e10, - EnableNegativeOffset: true, - EnableAtModifier: true, - NoStepSubqueryIntervalFn: func(_ int64) int64 { return 30 * time.Second.Milliseconds() }, - LookbackDelta: 5 * time.Minute, +var ( + opts = promql.EngineOpts{ + Timeout: 1 * time.Hour, + MaxSamples: 1e10, + EnableNegativeOffset: true, + EnableAtModifier: true, + NoStepSubqueryIntervalFn: func(_ int64) int64 { return 30 * time.Second.Milliseconds() }, + LookbackDelta: 5 * time.Minute, + } +) + +func TestPromQLAcceptance(t *testing.T) { + if testing.Short() { + // There are still some tests failing because we dont support regex matcher on + // the name label, or matching without a name label in the first place. + t.Skip("Skipping, because 'short' flag was set") + } + + engine := promql.NewEngine(opts) + t.Cleanup(func() { engine.Close() }) + + st := &testHelper{ + skipTests: []string{ + "testdata/name_label_dropping.test", // feature unsupported in promql-engine + }, + TBRun: t, + } + + promqltest.RunBuiltinTestsWithStorage(st, engine, func(tt testutil.T) storage.Storage { + return &acceptanceTestStorage{tb: t, st: teststorage.New(tt)} + }) +} + +type testHelper struct { + skipTests []string + promqltest.TBRun +} + +func (s *testHelper) Run(name string, t func(*testing.T)) bool { + if slices.Contains(s.skipTests, name) { + return true + } + + return s.TBRun.Run(name+"-concurrent", func(tt *testing.T) { + tt.Parallel() + s.TBRun.Run(name, t) + }) +} + +type acceptanceTestStorage struct { + tb testing.TB + st *teststorage.TestStorage +} + +func (st *acceptanceTestStorage) Appender(ctx context.Context) storage.Appender { + return st.st.Appender(ctx) +} + +func (st *acceptanceTestStorage) ChunkQuerier(int64, int64) (storage.ChunkQuerier, error) { + return nil, errors.New("unimplemented") +} + +func (st *acceptanceTestStorage) Querier(from, to int64) (storage.Querier, error) { + if st.st.Head().NumSeries() == 0 { + // parquet-go panics when writing an empty parquet file + return st.st.Querier(from, to) + } + return storageToDB(st.tb, st.st).Queryable().Querier(from, to) +} + +func (st *acceptanceTestStorage) Close() error { + return st.st.Close() +} + +func (st *acceptanceTestStorage) StartTime() (int64, error) { + return st.st.StartTime() +} + +func TestSelect(t *testing.T) { + ts := promqltest.LoadedStorage(t, `load 30s + foo{bar="baz"} 1 + abc{def="ghi"} 1 + jkl{ext="doesntmatter1"} 1 + jkl{ext="doesntmatter2"} 1 + `) + t.Cleanup(func() { ts.Close() }) + + database := storageToDB(t, ts, db.ExternalLabels(labels.FromStrings("ext", "test", "rep", "1"))) + qry, err := database.Queryable(db.DropReplicaLabels("rep")).Querier(math.MinInt64, math.MaxInt64) + if err != nil { + t.Fatalf("unable to construct querier: %s", err) + } + for _, tc := range []struct { + matchers []*labels.Matcher + want []labels.Labels + warns []error + }{ + { + want: []labels.Labels{ + labels.FromStrings("__name__", "abc", "def", "ghi", "ext", "test"), + labels.FromStrings("__name__", "foo", "bar", "baz", "ext", "test"), + labels.FromStrings("__name__", "jkl", "ext", "test"), + }, + warns: []error{ + warnings.ErrorDroppedSeriesAfterExternalLabelMangling, + }, + }, + { + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchNotEqual, "ext", "test"), + }, + }, + { + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "ext", "doesntmatter.*"), + }, + }, + { + matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "__name__", "foo"), + }, + want: []labels.Labels{ + labels.FromStrings("__name__", "foo", "bar", "baz", "ext", "test"), + }, + }, + } { + t.Run("", func(tt *testing.T) { + ss := qry.Select(tt.Context(), false, &storage.SelectHints{}, tc.matchers...) + + series := make([]labels.Labels, 0) + for ss.Next() { + series = append(series, ss.At().Labels()) + } + if err := ss.Err(); err != nil { + tt.Fatalf("unable to query label names: %s", err) + } + if !slices.EqualFunc(tc.want, series, func(l, r labels.Labels) bool { return l.Hash() == r.Hash() }) { + tt.Fatalf("expected %q, got %q", tc.want, series) + } + if errs := ss.Warnings().AsErrors(); !slices.Equal(tc.warns, errs) { + tt.Errorf("expected %q, got %q", tc.warns, errs) + } + }) + } +} + +func TestLabelNames(t *testing.T) { + ts := promqltest.LoadedStorage(t, `load 30s + foo{bar="baz"} 1 + abc{def="ghi"} 1 + `) + t.Cleanup(func() { ts.Close() }) + + database := storageToDB(t, ts, db.ExternalLabels(labels.FromStrings("ext", "test", "rep", "1"))) + qry, err := database.Queryable(db.DropReplicaLabels("rep")).Querier(math.MinInt64, math.MaxInt64) + if err != nil { + t.Fatalf("unable to construct querier: %s", err) + } + + for _, tc := range []struct { + matchers []*labels.Matcher + want []string + }{ + { + want: []string{"__name__", "bar", "def", "ext"}, + }, + { + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "ext", "test")}, + }, + { + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "ext", "test")}, + want: []string{"__name__", "bar", "def", "ext"}, + }, + { + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "fizz", "buzz")}, + }, + { + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "__name__", "abc")}, + want: []string{"__name__", "def", "ext"}, + }, + } { + t.Run("", func(tt *testing.T) { + res, _, err := qry.LabelNames(tt.Context(), &storage.LabelHints{}, tc.matchers...) + if err != nil { + tt.Fatalf("unable to query label names: %s", err) + } + if !slices.Equal(tc.want, res) { + tt.Fatalf("expected %q, got %q", tc.want, res) + } + }) + } +} + +func TestLabelValues(t *testing.T) { + ts := promqltest.LoadedStorage(t, `load 30s + foo{bar="baz"} 1 + abc{ext="internal"} 1 + `) + t.Cleanup(func() { ts.Close() }) + + database := storageToDB(t, ts, db.ExternalLabels(labels.FromStrings("ext", "test", "rep", "1"))) + qry, err := database.Queryable(db.DropReplicaLabels("rep")).Querier(math.MinInt64, math.MaxInt64) + if err != nil { + t.Fatalf("unable to construct querier: %s", err) + } + + for _, tc := range []struct { + name string + matchers []*labels.Matcher + want []string + warns []error + }{ + { + name: "bar", + want: []string{"baz"}, + }, + { + name: "ext", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "bar", "baz")}, + want: []string{"test"}, + }, + { + name: "ext", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "fizz", "buzz")}, + }, + { + name: "ext", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "__name__", "abc")}, + want: []string{"test"}, + warns: []error{warnings.ErrorDroppedLabelValuesAfterExternalLabelMangling}, + }, + { + name: "bar", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "ext", "test")}, + }, + { + name: "bar", + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "ext", "test")}, + want: []string{"baz"}, + }, + { + name: "ext", + want: []string{"test"}, + warns: []error{warnings.ErrorDroppedLabelValuesAfterExternalLabelMangling}, + }, + { + name: "rep", + }, + } { + t.Run("", func(tt *testing.T) { + res, warns, err := qry.LabelValues(tt.Context(), tc.name, &storage.LabelHints{}, tc.matchers...) + if err != nil { + tt.Fatalf("unable to query label names: %s", err) + } + if !slices.Equal(tc.want, res) { + tt.Fatalf("expected %q, got %q", tc.want, res) + } + if errs := warns.AsErrors(); !slices.Equal(tc.warns, errs) { + tt.Errorf("expected %q, got %q", tc.warns, errs) + } + }) + } + + t.Run("No matchers, replica label", func(tt *testing.T) { + res, _, err := qry.LabelValues(tt.Context(), "rep", &storage.LabelHints{}) + if err != nil { + tt.Fatalf("unable to query label values: %s", err) + } + if want := []string{}; !slices.Equal(want, res) { + tt.Fatalf("expected %q, got %q", want, res) + } + }) } func TestInstantQuery(t *testing.T) { @@ -45,6 +317,12 @@ func TestInstantQuery(t *testing.T) { queries []string queryTime time.Time }{ + { + name: "vector selector with empty matcher on nonexistent column", + load: `load 10s + metric{pod="nginx-1", a=""} 1+2x40`, + queries: []string{`metric{foo=""}`}, + }, { name: "vector selector with empty labels", load: `load 10s @@ -52,7 +330,7 @@ func TestInstantQuery(t *testing.T) { queries: []string{`metric`}, }, { - name: "vector selector with different labelnames", + name: "vector selector with differnt labelnames", load: `load 10s metric{pod="nginx-1", a="foo"} 1+1x40 metric{pod="nginx-1", b="bar"} 1+1x40`, @@ -536,9 +814,8 @@ func TestInstantQuery(t *testing.T) { testmetric1{src="a",dst="b"} 0 testmetric2{src="a",dst="b"} 1`, queryTime: time.Unix(0, 0), - queries: []string{ - // TODO(kdeems): re-enable when regex name match is supported - // `ceil({__name__=~'testmetric1|testmetric2'})`, + queries: []string{ + `ceil({__name__=~'testmetric1|testmetric2'})`, }, }, { @@ -779,8 +1056,6 @@ func TestInstantQuery(t *testing.T) { }, } for _, tc := range cases { - - tc := tc t.Run(tc.name, func(t *testing.T) { queryTime := defaultQueryTime if !tc.queryTime.IsZero() { @@ -791,14 +1066,14 @@ func TestInstantQuery(t *testing.T) { testStorage := promqltest.LoadedStorage(t, tc.load) t.Cleanup(func() { testStorage.Close() }) - db := storageToDB(t, testStorage) + database := storageToDB(t, testStorage) ctx := context.Background() for _, query := range tc.queries { - t.Run(query, func(t *testing.T) { + t.Run("", func(t *testing.T) { t.Parallel() - q1, err := engine.NewInstantQuery(ctx, db.Queryable(), nil, query, queryTime) + q1, err := engine.NewInstantQuery(ctx, database.Queryable(), nil, query, queryTime) if err != nil { t.Fatalf("error for query `%s`: %v", query, err) } @@ -943,16 +1218,62 @@ func TestRangeQuery(t *testing.T) { }, }, }, + { + name: "empty and nonempty matchers", + load: ` + load 30s + http_requests_total{pod="nginx-1", route="/"} 0+1x5 + http_requests_total{pod="nginx-2"} 0+2x5 + http_requests_total{pod="nginx-3", route="/"} 0+3x5 + http_requests_total{pod="nginx-4"} 0+4x5`, + queries: []rangeQuery{ + { + query: `http_requests_total{route=""}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + { + query: `http_requests_total{route!=""}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + { + query: `http_requests_total{route=~""}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + { + query: `http_requests_total{route!~""}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + { + query: `http_requests_total{route=~".+"}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + { + query: `http_requests_total{route!~".+"}`, + start: time.Unix(0, 0), + end: time.Unix(120, 0), + step: time.Minute, + }, + }, + }, } for _, tc := range cases { - tc := tc t.Run(tc.name, func(t *testing.T) { t.Parallel() testStorage := promqltest.LoadedStorage(t, tc.load) t.Cleanup(func() { testStorage.Close() }) db := storageToDB(t, testStorage) - ctx := context.Background() + ctx := t.Context() for _, query := range tc.queries { q1, err := engine.NewRangeQuery(ctx, db.Queryable(), nil, query.query, query.start, query.end, query.step) @@ -979,30 +1300,35 @@ func TestRangeQuery(t *testing.T) { } } -func TestMutipleSearchCallbacks(t *testing.T) { +func TestExternalAndReplicaLabels(t *testing.T) { + t.Parallel() + ctx := context.Background() st := teststorage.New(t) t.Cleanup(func() { st.Close() }) - bkt := objstore.NewInMemBucket() + bkt, err := filesystem.NewBucket(t.TempDir()) + if err != nil { + t.Fatalf("unable to create bucket: %s", err) + } t.Cleanup(func() { bkt.Close() }) engine := promql.NewEngine(opts) t.Cleanup(func() { engine.Close() }) app := st.Appender(ctx) - for i := 0; i != 10_000; i++ { - app.Append(0, labels.FromStrings("__name__", "foo", "bar", fmt.Sprintf("%d", 2*i)), 0, float64(i)) - app.Append(0, labels.FromStrings("__name__", "foo", "bar", fmt.Sprintf("%d", 2*i+1)), 0, float64(i)) + for i := range 100 { + app.Append(0, labels.FromStrings("__name__", "foo", "replica", "1", "bar", fmt.Sprintf("%d", 2*i)), 0, float64(i)) + app.Append(0, labels.FromStrings("__name__", "foo", "replica", "2", "bar", fmt.Sprintf("%d", 2*i)), 0, float64(i)) } if err := app.Commit(); err != nil { t.Fatalf("unable to commit: %s", err) } - db := storageToDB(t, st) + database := storageToDB(t, st) - expr := `foo{bar=~".*1"}` + expr := `foo{bar=~"0"}` - q1, err := engine.NewInstantQuery(ctx, db.Queryable(), nil, expr, time.Unix(5, 0)) + q1, err := engine.NewInstantQuery(ctx, database.Queryable(db.DropReplicaLabels("replica")), nil, expr, time.Unix(5, 0)) if err != nil { t.Fatalf("error for query: %v", err) } @@ -1018,43 +1344,39 @@ func TestMutipleSearchCallbacks(t *testing.T) { t.Fatalf("result was no vector: %v", err) } for i := range v { - if !strings.HasSuffix(v[i].Metric.Get("bar"), "1") { + if v[i].Metric.Get("replica") != "" { t.Fatalf("unexpected series in result for query %s: %s", expr, v[i].Metric) } } } -func storageToDB(tb testing.TB, st *teststorage.TestStorage) *DB { +func storageToDBWithBkt(tb testing.TB, st *teststorage.TestStorage, bkt objstore.Bucket, opts ...db.DBOption) *db.DB { ctx := context.Background() - bkt := objstore.NewInMemBucket() h := st.DB.Head() day := time.UnixMilli(h.MinTime()).UTC() - if err := convert.ConvertTSDBBlock(ctx, bkt, day, []convert.Convertible{h}); err != nil { + if err := convert.ConvertTSDBBlock(ctx, bkt, day, []convert.Convertable{h}); err != nil { tb.Fatal(err) } - blkname, err := schema.BlockNameForDay(time.Unix(st.DB.Head().Meta().MinTime, 0).UTC()) - if err != nil { - tb.Fatal(err) - } - meta, err := readMetafile(ctx, bkt, blkname) - if err != nil { + discoverer := locate.NewDiscoverer(bkt) + if err := discoverer.Discover(ctx); err != nil { tb.Fatal(err) } - blk, err := newBlockForMeta(ctx, bkt, meta) - if err != nil { + syncer := locate.NewSyncer(bkt, locate.BlockOptions(locate.LabelFilesDir(tb.TempDir()))) + if err := syncer.Sync(ctx, discoverer.Metas()); err != nil { tb.Fatal(err) } - return NewDB(singleBlockSyncer{blk: blk}) + return db.NewDB(syncer, opts...) } -type singleBlockSyncer struct { - blk *Block -} +func storageToDB(tb testing.TB, st *teststorage.TestStorage, opts ...db.DBOption) *db.DB { + bkt, err := filesystem.NewBucket(tb.TempDir()) + if err != nil { + tb.Fatalf("unable to create bucket: %s", err) + } + return storageToDBWithBkt(tb, st, bkt, opts...) -func (s singleBlockSyncer) Blocks() []*Block { - return []*Block{s.blk} } type seriesByLabels []promql.Series @@ -1104,7 +1426,7 @@ var ( sort.Sort(samplesByLabels(vx)) sort.Sort(samplesByLabels(vy)) - for i := 0; i < len(vx); i++ { + for i := range len(vx) { if !compareMetrics(vx[i].Metric, vy[i].Metric) { return false } @@ -1131,7 +1453,7 @@ var ( // Sort matrix before comparing. sort.Sort(seriesByLabels(mx)) sort.Sort(seriesByLabels(my)) - for i := 0; i < len(mx); i++ { + for i := range len(mx) { mxs := mx[i] mys := my[i] @@ -1145,7 +1467,7 @@ var ( if len(xps) != len(yps) { return false } - for j := 0; j < len(xps); j++ { + for j := range len(xps) { if xps[j].T != yps[j].T { return false } @@ -1159,7 +1481,7 @@ var ( if len(xph) != len(yph) { return false } - for j := 0; j < len(xph); j++ { + for j := range len(xph) { if xph[j].T != yph[j].T { return false } @@ -1182,40 +1504,3 @@ var ( return false }) ) - -func BenchmarkQuery(b *testing.B) { - ctx := context.Background() - st := teststorage.New(b) - b.Cleanup(func() { st.Close() }) - - bkt := objstore.NewInMemBucket() - b.Cleanup(func() { bkt.Close() }) - - engine := promql.NewEngine(opts) - b.Cleanup(func() { engine.Close() }) - - app := st.Appender(ctx) - for i := 0; i != 100_000; i++ { - for j := 0; j != 120; j++ { - app.Append(0, labels.FromStrings("__name__", "foo", "bar", "fizz", "buzz", fmt.Sprintf("%d", i)), int64(j), float64(j)) - } - } - if err := app.Commit(); err != nil { - b.Fatalf("unable to commit: %s", err) - } - db := storageToDB(b, st) - - qryable := db.Queryable() - q1, err := engine.NewInstantQuery(ctx, qryable, nil, "foo", time.Unix(5, 0)) - if err != nil { - b.Fatalf("error for query: %v", err) - } - defer q1.Close() - - b.ResetTimer() - b.ReportAllocs() - - for i := 0; i < b.N; i++ { - q1.Exec(ctx) - } -} diff --git a/db/discover.go b/db/discover.go deleted file mode 100644 index 35818fa..0000000 --- a/db/discover.go +++ /dev/null @@ -1,173 +0,0 @@ -// Copyright (c) 2025 Cloudflare, Inc. -// Licensed under the Apache 2.0 license found in the LICENSE file or at: -// https://opensource.org/licenses/Apache-2.0 - -package db - -import ( - "context" - "fmt" - "io" - "maps" - "slices" - "sync" - - "github.com/thanos-io/objstore" - "google.golang.org/protobuf/proto" - - "github.com/cloudflare/parquet-tsdb-poc/proto/metapb" - "github.com/cloudflare/parquet-tsdb-poc/schema" -) - -type Discoverer struct { - bkt objstore.Bucket - - mu sync.Mutex - metas map[string]Meta - - concurrency int -} - -type discoveryConfig struct { - concurrency int -} - -type DiscoveryOption func(*discoveryConfig) - -func MetaConcurrency(c int) DiscoveryOption { - return func(cfg *discoveryConfig) { - cfg.concurrency = c - } -} - -func NewDiscoverer(bkt objstore.Bucket, opts ...DiscoveryOption) *Discoverer { - cfg := discoveryConfig{ - concurrency: 1, - } - for _, o := range opts { - o(&cfg) - } - return &Discoverer{ - bkt: bkt, - metas: make(map[string]Meta), - concurrency: cfg.concurrency, - } -} - -func (s *Discoverer) Metas() map[string]Meta { - s.mu.Lock() - defer s.mu.Unlock() - - res := make(map[string]Meta, len(s.metas)) - maps.Copy(res, s.metas) - - return res -} - -func (s *Discoverer) Discover(ctx context.Context) error { - m := make(map[string][]string) - err := s.bkt.Iter(ctx, "", func(n string) error { - id, file, ok := schema.SplitBlockPath(n) - if !ok { - return nil - } - m[id] = append(m[id], file) - return nil - }, objstore.WithRecursiveIter()) - if err != nil { - return err - } - - type metaOrError struct { - m Meta - err error - } - - metaC := make(chan metaOrError) - go func() { - defer close(metaC) - - workerC := make(chan string, s.concurrency) - go func() { - defer close(workerC) - - for k, v := range m { - if _, ok := s.metas[k]; ok { - continue - } - if !slices.Contains(v, schema.MetaFile) { - // skip incomplete block - continue - } - workerC <- k - } - }() - - var wg sync.WaitGroup - defer wg.Wait() - - for i := 0; i < s.concurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - for k := range workerC { - meta, err := readMetafile(ctx, s.bkt, k) - if err != nil { - metaC <- metaOrError{err: fmt.Errorf("unable to read meta file for %q: %w", k, err)} - } else { - metaC <- metaOrError{m: meta} - } - } - }() - } - }() - - nm := make(map[string]Meta) - for m := range metaC { - if m.err != nil { - return fmt.Errorf("unable to read meta: %w", err) - } - nm[m.m.Name] = m.m - } - - s.mu.Lock() - defer s.mu.Unlock() - - maps.Copy(s.metas, nm) - - return nil -} - -func readMetafile(ctx context.Context, bkt objstore.Bucket, name string) (Meta, error) { - mfile := schema.MetaFileNameForBlock(name) - if _, err := bkt.Attributes(ctx, mfile); err != nil { - return Meta{}, fmt.Errorf("unable to attr %s: %w", mfile, err) - } - rdr, err := bkt.Get(ctx, mfile) - if err != nil { - return Meta{}, fmt.Errorf("unable to get %s: %w", mfile, err) - } - defer rdr.Close() - - metaBytes, err := io.ReadAll(rdr) - if err != nil { - return Meta{}, fmt.Errorf("unable to read %s: %w", mfile, err) - } - - metapb := &metapb.Metadata{} - if err := proto.Unmarshal(metaBytes, metapb); err != nil { - return Meta{}, fmt.Errorf("unable to read %s: %w", mfile, err) - } - - m := make(map[string][]string, 0) - for k, v := range metapb.GetColumnsForName() { - m[k] = v.GetColumns() - } - return Meta{ - Name: name, - Mint: metapb.GetMint(), - Maxt: metapb.GetMaxt(), - Shards: metapb.GetShards(), - ColumnsForName: m, - }, nil -} diff --git a/db/metrics.go b/db/metrics.go index 1f20aa9..72a6862 100644 --- a/db/metrics.go +++ b/db/metrics.go @@ -5,19 +5,41 @@ package db import ( - "github.com/hashicorp/go-multierror" + "errors" + "github.com/prometheus/client_golang/prometheus" ) +const ( + typeSelect = "select" + typeLabelValues = "label_values" + typeLabelNames = "label_names" + + // to avoid too high cardinality, we only measure on shard level + whereShard = "shard" +) + var ( - bucketRequests = prometheus.NewCounter(prometheus.CounterOpts{ - Name: "bucket_requests_total", - Help: "Total amount of requests to object storage", - }) + queryableOperationsTotal = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: "queryable_operations_total", + Help: "The total amount of query operations we evaluated", + }, []string{"type", "where"}) + queryableOperationsDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "queryable_operations_seconds", + Help: "Histogram of durations for queryable operations", + Buckets: prometheus.ExponentialBucketsRange(0.1, 30, 20), + }, []string{"type", "where"}) ) func RegisterMetrics(reg prometheus.Registerer) error { - return multierror.Append(nil, - reg.Register(bucketRequests), - ).ErrorOrNil() + for _, t := range []string{typeSelect, typeLabelNames, typeLabelValues} { + for _, w := range []string{whereShard} { + queryableOperationsTotal.WithLabelValues(t, w).Set(0) + queryableOperationsDuration.WithLabelValues(t, w).Observe(0) + } + } + return errors.Join( + reg.Register(queryableOperationsTotal), + reg.Register(queryableOperationsDuration), + ) } diff --git a/db/seriesset.go b/db/seriesset.go index af51f09..e635bdb 100644 --- a/db/seriesset.go +++ b/db/seriesset.go @@ -8,13 +8,13 @@ import ( "context" "sync" - "github.com/hashicorp/go-multierror" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" ) type hints struct { + Limit int By bool Func string Grouping []string @@ -24,6 +24,7 @@ func fromStorageHints(h *storage.SelectHints) hints { clone := make([]string, len(h.Grouping)) copy(clone, h.Grouping) return hints{ + Limit: h.Limit, Func: h.Func, By: h.By, Grouping: clone, @@ -31,7 +32,7 @@ func fromStorageHints(h *storage.SelectHints) hints { } func toStorageHints(h hints) *storage.SelectHints { - return &storage.SelectHints{Func: h.Func, By: h.By, Grouping: h.Grouping} + return &storage.SelectHints{Limit: h.Limit, Func: h.Func, By: h.By, Grouping: h.Grouping} } type selectFn func(context.Context, bool, *storage.SelectHints, ...*labels.Matcher) storage.SeriesSet @@ -91,48 +92,6 @@ func (c *lazySeriesSet) Warnings() annotations.Annotations { return c.set.Warnings() } -type verticalSeriesSet struct { - i int - sets []storage.SeriesSet -} - -func newVerticalSeriesSet(sets ...storage.SeriesSet) storage.SeriesSet { - if len(sets) == 0 { - return storage.EmptySeriesSet() - } - return &verticalSeriesSet{sets: sets, i: 0} -} - -func (ss *verticalSeriesSet) Next() bool { - if ss.sets[ss.i].Next() { - return true - } - for ss.i < len(ss.sets)-1 { - ss.i++ - if ss.sets[ss.i].Next() { - return true - } - } - return false -} - -func (ss *verticalSeriesSet) At() storage.Series { return ss.sets[ss.i].At() } -func (ss *verticalSeriesSet) Err() error { - var err *multierror.Error - for i := range ss.sets { - err = multierror.Append(err, ss.sets[i].Err()) - } - return err.ErrorOrNil() -} - -func (ss *verticalSeriesSet) Warnings() annotations.Annotations { - res := annotations.New() - for i := range ss.sets { - res.Merge(ss.sets[i].Warnings()) - } - return *res -} - type concatSeriesSet struct { i int series []storage.Series @@ -156,3 +115,18 @@ func (ss *concatSeriesSet) Next() bool { func (ss *concatSeriesSet) At() storage.Series { return ss.series[ss.i] } func (ss *concatSeriesSet) Err() error { return nil } func (ss *concatSeriesSet) Warnings() annotations.Annotations { return nil } + +type warningsSeriesSet struct { + storage.SeriesSet + + warns annotations.Annotations +} + +func newWarningsSeriesSet(ss storage.SeriesSet, warns annotations.Annotations) storage.SeriesSet { + return &warningsSeriesSet{SeriesSet: ss, warns: warns} +} + +func (ss *warningsSeriesSet) Warnings() annotations.Annotations { + original := ss.SeriesSet.Warnings() + return original.Merge(ss.warns) +} diff --git a/db/shard.go b/db/shard.go index 024be94..cca669e 100644 --- a/db/shard.go +++ b/db/shard.go @@ -6,55 +6,104 @@ package db import ( "context" - "encoding/binary" - "errors" "fmt" + "io" "slices" + "time" "github.com/parquet-go/parquet-go" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" + "go.opentelemetry.io/otel/attribute" - "github.com/cloudflare/parquet-tsdb-poc/internal/encoding" + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" "github.com/cloudflare/parquet-tsdb-poc/schema" "github.com/cloudflare/parquet-tsdb-poc/search" ) type Shard struct { - meta Meta + meta schema.Meta chunkspfile *parquet.File labelspfile *parquet.File + + chunkFileReaderFromCtx func(ctx context.Context) io.ReaderAt } -func (shd *Shard) Queryable(extlabels labels.Labels, replicaLabelNames []string) storage.Queryable { - return &ShardQueryable{extlabels: extlabels, replicaLabelNames: replicaLabelNames, shard: shd} +func NewShard( + meta schema.Meta, + chunkspfile *parquet.File, + labelspfile *parquet.File, + chunkFileReaderCtxFunc func(ctx context.Context) io.ReaderAt, +) *Shard { + return &Shard{ + meta: meta, + chunkspfile: chunkspfile, + labelspfile: labelspfile, + chunkFileReaderFromCtx: chunkFileReaderCtxFunc, + } +} + +func (shd *Shard) Queryable( + extlabels labels.Labels, + replicaLabelNames []string, + selectChunkBytesQuota *limits.Quota, + selectRowCountQuota *limits.Quota, + selectChunkPartitionMaxRange uint64, + selectChunkPartitionMaxGap uint64, + selectChunkPartitionMaxConcurrency int, +) storage.Queryable { + return &ShardQueryable{ + extlabels: extlabels, + replicaLabelNames: replicaLabelNames, + selectChunkBytesQuota: selectChunkBytesQuota, + selectRowCountQuota: selectRowCountQuota, + selectChunkPartitionMaxRange: selectChunkPartitionMaxRange, + selectChunkPartitionMaxGap: selectChunkPartitionMaxGap, + selectChunkPartitionMaxConcurrency: selectChunkPartitionMaxConcurrency, + shard: shd, + } } type ShardQueryable struct { - extlabels labels.Labels - replicaLabelNames []string + extlabels labels.Labels + replicaLabelNames []string + selectChunkBytesQuota *limits.Quota + selectRowCountQuota *limits.Quota + selectChunkPartitionMaxRange uint64 + selectChunkPartitionMaxGap uint64 + selectChunkPartitionMaxConcurrency int shard *Shard } func (q *ShardQueryable) Querier(mint, maxt int64) (storage.Querier, error) { return &ShardQuerier{ - mint: mint, - maxt: maxt, - shard: q.shard, - extlabels: q.extlabels, - replicaLabelNames: q.replicaLabelNames, + mint: mint, + maxt: maxt, + shard: q.shard, + extlabels: q.extlabels, + replicaLabelNames: q.replicaLabelNames, + selectChunkBytesQuota: q.selectChunkBytesQuota, + selectRowCountQuota: q.selectRowCountQuota, + selectChunkPartitionMaxRange: q.selectChunkPartitionMaxRange, + selectChunkPartitionMaxGap: q.selectChunkPartitionMaxGap, + selectChunkPartitionMaxConcurrency: q.selectChunkPartitionMaxConcurrency, }, nil } type ShardQuerier struct { - mint, maxt int64 - extlabels labels.Labels - replicaLabelNames []string + mint, maxt int64 + extlabels labels.Labels + replicaLabelNames []string + selectChunkBytesQuota *limits.Quota + selectRowCountQuota *limits.Quota + selectChunkPartitionMaxRange uint64 + selectChunkPartitionMaxGap uint64 + selectChunkPartitionMaxConcurrency int shard *Shard } @@ -63,263 +112,163 @@ var _ storage.Querier = &ShardQuerier{} func (ShardQuerier) Close() error { return nil } -func (q ShardQuerier) LabelValues(_ context.Context, name string, _ *storage.LabelHints, ms ...*labels.Matcher) ([]string, annotations.Annotations, error) { - if len(ms) != 0 { - return nil, nil, errors.New("label values with label matchers is not supported") +func (q ShardQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + ctx, span := tracing.Tracer().Start(ctx, "Label Values Shard") + defer span.End() + + span.SetAttributes(attribute.StringSlice("matchers", matchersToStringSlice(matchers))) + span.SetAttributes(attribute.StringSlice("shard.replica_labels", q.replicaLabelNames)) + span.SetAttributes(attribute.String("shard.external_labels", q.extlabels.String())) + + queryableOperationsTotal.WithLabelValues(typeLabelValues, whereShard).Inc() + + start := time.Now() + defer func() { + queryableOperationsDuration.WithLabelValues(typeLabelValues, whereShard).Observe(float64(time.Since(start).Seconds())) + }() + + labelValues, warns, err := search.LabelValues( + ctx, + search.LabelValuesReadMeta{ + Meta: q.shard.meta, + LabelPfile: q.shard.labelspfile, + ExternalLabels: q.extlabels, + ReplicaLabelNames: q.replicaLabelNames, + }, + name, + hints, + matchers...) + if err != nil { + return nil, nil, fmt.Errorf("unable to query label values: %w", err) } - if name != model.MetricNameLabel { - return nil, nil, errors.New("label values for label names other then __name__ is not supported") - } + return labelValues, warns, nil +} - res := make([]string, 0, len(q.shard.meta.ColumnsForName)) - for name := range q.shard.meta.ColumnsForName { - res = append(res, name) +func (q ShardQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + ctx, span := tracing.Tracer().Start(ctx, "Label Names Shard") + defer span.End() + + span.SetAttributes(attribute.StringSlice("matchers", matchersToStringSlice(matchers))) + span.SetAttributes(attribute.StringSlice("shard.replica_labels", q.replicaLabelNames)) + span.SetAttributes(attribute.String("shard.external_labels", q.extlabels.String())) + + queryableOperationsTotal.WithLabelValues(typeLabelNames, whereShard).Inc() + + start := time.Now() + defer func() { + queryableOperationsDuration.WithLabelValues(typeLabelNames, whereShard).Observe(float64(time.Since(start).Seconds())) + }() + + labelNames, warns, err := search.LabelNames( + ctx, + search.LabelNamesReadMeta{ + Meta: q.shard.meta, + LabelPfile: q.shard.labelspfile, + ExternalLabels: q.extlabels, + ReplicaLabelNames: q.replicaLabelNames, + }, + hints, + matchers..., + ) + if err != nil { + return nil, nil, fmt.Errorf("unable to query label names: %w", err) } - return res, nil, nil -} - -func (ShardQuerier) LabelNames(context.Context, *storage.LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { - // TODO - return nil, nil, nil + return labelNames, warns, nil } func (q ShardQuerier) Select(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { return newLazySeriesSet(ctx, q.selectFn, sorted, hints, matchers...) } -func (q ShardQuerier) selectFn(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { - metricName, err := metricNameFromMatchers(matchers) - if err != nil { - return storage.ErrSeriesSet(fmt.Errorf("unable to get metric name from matchers: %s", err)) +func matchersToStringSlice(matchers []*labels.Matcher) []string { + res := make([]string, len(matchers)) + for i := range matchers { + res[i] = matchers[i].String() } - columnsForName := slices.Clone(q.shard.meta.ColumnsForName[metricName]) + return res +} - constraint, err := constraintForMatchers(matchers, columnsForName) +func (q ShardQuerier) selectFn(ctx context.Context, sorted bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + ctx, span := tracing.Tracer().Start(ctx, "Select Shard") + defer span.End() + + span.SetAttributes(attribute.Bool("sorted", sorted)) + span.SetAttributes(attribute.StringSlice("matchers", matchersToStringSlice(matchers))) + span.SetAttributes(attribute.StringSlice("shard.replica_labels", q.replicaLabelNames)) + span.SetAttributes(attribute.String("shard.external_labels", q.extlabels.String())) + + queryableOperationsTotal.WithLabelValues(typeSelect, whereShard).Inc() + + start := time.Now() + defer func() { + queryableOperationsDuration.WithLabelValues(typeSelect, whereShard).Observe(float64(time.Since(start).Seconds())) + }() + + seriesChunks, warns, err := search.Select( + ctx, + search.SelectReadMeta{ + Meta: q.shard.meta, + LabelPfile: q.shard.labelspfile, + ChunkPfile: q.shard.chunkspfile, + RowCountQuota: q.selectRowCountQuota, + ChunkBytesQuota: q.selectChunkBytesQuota, + ChunkPagePartitionMaxRange: q.selectChunkPartitionMaxRange, + ChunkPagePartitionMaxGap: q.selectChunkPartitionMaxGap, + ChunkPagePartitionMaxConcurrency: q.selectChunkPartitionMaxConcurrency, + ChunkFileReaderFromContext: q.shard.chunkFileReaderFromCtx, + ExternalLabels: q.extlabels, + ReplicaLabelNames: q.replicaLabelNames, + }, + q.mint, + q.maxt, + hints, + matchers..., + ) if err != nil { - return storage.ErrSeriesSet(fmt.Errorf("unable to compute constraint for matchers: %s", err)) + return storage.ErrSeriesSet(fmt.Errorf("unable to select: %w", err)) } - // we cannot do projections in presence of replica/external labels as its not clear how to resolve conflicts - labelProjections := labelProjections(columnsForName) - chunkProjections := chunkProjections(q.shard.meta.Mint, q.mint, q.maxt) - - // no need to fetch chunks for /api/v1/series - if hints.Func == "series" { - chunkProjections = nil + series, dropped := q.seriesFromSeriesChunks(seriesChunks) + if dropped { + warns = warns.Add(warnings.ErrorDroppedSeriesAfterExternalLabelMangling) } - labelpfile := q.shard.labelspfile - labelReadSchema := schema.Projection(labelpfile.Schema(), labelProjections) - chunkspfile := q.shard.chunkspfile - chunksReadSchema := schema.Projection(chunkspfile.Schema(), chunkProjections) - - rr, err := search.Match(ctx, constraint, labelpfile, labelReadSchema, chunkspfile, chunksReadSchema) - if err != nil { - return storage.ErrSeriesSet(err) - } - defer rr.Close() - - ssb := &seriesSetBuilder{ - schema: schema.Joined(labelReadSchema, chunksReadSchema), - mint: q.mint, - maxt: q.maxt, - m: make(map[uint64]struct{}), - s: make([]*chunkSeries, 0), - b: labels.NewBuilder(labels.EmptyLabels()), - extlabels: q.extlabels, - replicaLabelNames: q.replicaLabelNames, - } - - if _, err := parquet.CopyRows(ssb, rr); err != nil { - return storage.ErrSeriesSet(err) - } - series := ssb.Series() if sorted { slices.SortFunc(series, func(l, r storage.Series) int { return labels.Compare(l.Labels(), r.Labels()) }) } - return newConcatSeriesSet(series...) + return newWarningsSeriesSet(newConcatSeriesSet(series...), warns) } -func metricNameFromMatchers(matchers []*labels.Matcher) (string, error) { - for i := range matchers { - if matchers[i].Name == labels.MetricName { - return matchers[i].Value, nil - } - } - return "", errors.New("metric name is required") -} +func (q ShardQuerier) seriesFromSeriesChunks(sc []search.SeriesChunks) ([]storage.Series, bool) { + res := make([]storage.Series, 0, len(sc)) + m := make(map[uint64]struct{}) + dropped := false -func constraintForMatchers(matchers []*labels.Matcher, columnsForName []string) (search.Constraint, error) { - constraints := make([]search.Constraint, 0) - for i := range matchers { - m := matchers[i] - col := schema.LabelNameToColumn(m.Name) - val := parquet.ValueOf(m.Value) - if m.Name == labels.MetricName { - if m.Type != labels.MatchEqual { - return nil, errors.New("only equal matches on metric name are allowed") - } - constraints = append(constraints, search.EqualWithPageCheck(col, val)) + for i := range sc { + lbls := sc[i].Lset + h := sc[i].LsetHash + if _, ok := m[h]; ok { + // We have seen this series before, skip it for now; we could be smarter and select + // chunks appropriately so that we fill in what might be missing but for now skipping is fine + dropped = true continue } - validColumn := slices.Contains(columnsForName, col) - var c search.Constraint - switch m.Type { - case labels.MatchEqual: - if !validColumn { - // equal match on a column that the series does not have; return nothing - return search.Null(), nil - } - c = search.Equal(col, val) - case labels.MatchNotEqual: - if !validColumn { - continue - } - c = search.Not(search.Equal(col, val)) - case labels.MatchRegexp: - if !validColumn { - // equal match on a column that the series does not have; return nothing - return search.Null(), nil - } - var err error - c, err = search.Regex(col, m.GetRegexString()) - if err != nil { - return nil, fmt.Errorf("unable to build regex constraint: %s", err) - } - case labels.MatchNotRegexp: - if !validColumn { - continue - } - var err error - c, err = search.Regex(col, m.GetRegexString()) - if err != nil { - return nil, fmt.Errorf("unable to build regex constraint: %s", err) - } - c = search.Not(c) - } - constraints = append(constraints, c) - } - return search.And(constraints...), nil -} - -func labelProjections(columnsForName []string) []string { - return append(columnsForName, schema.LabelNameToColumn(model.MetricNameLabel)) -} - -func chunkProjections(blkStart, mint, maxt int64) []string { - res := make([]string, 0) - - // TODO: This might be buggy since a block might start at the end of 0-8 and might - // leak into 8-16. Right now we would not catch this. This supposes that chunks - // are aligned to 8h. - cmin, cmax := blkStart, blkStart+schema.ChunkColumnLength.Milliseconds() - for i := 0; i != int(schema.ChunkColumnsPerDay); i++ { - if !util.Intersects(cmin, cmax, mint, maxt) { - goto C - } - switch i { - case 0: - res = append(res, schema.ChunksColumn0) - case 1: - res = append(res, schema.ChunksColumn1) - case 2: - res = append(res, schema.ChunksColumn2) - } - C: - cmin, cmax = cmax, cmax+schema.ChunkColumnLength.Milliseconds() - } - return res -} + m[h] = struct{}{} -type seriesSetBuilder struct { - schema *parquet.Schema - mint, maxt int64 - extlabels labels.Labels - replicaLabelNames []string - - s []*chunkSeries - b *labels.Builder - m map[uint64]struct{} -} - -func (ssb *seriesSetBuilder) WriteRows(rs []parquet.Row) (int, error) { - var ( - chksBytes [schema.ChunkColumnsPerDay][]byte - ) - cols := ssb.schema.Columns() - for i := range rs { - ssb.b.Reset(labels.EmptyLabels()) - rc := rs[i].Clone() - for j := range rc { - key := cols[j][0] - val := rc[j] - switch key { - case schema.ChunksColumn0: - chksBytes[0] = val.ByteArray() - case schema.ChunksColumn1: - chksBytes[1] = val.ByteArray() - case schema.ChunksColumn2: - chksBytes[2] = val.ByteArray() - default: - lblName := schema.ColumnToLabelName(key) - if !val.IsNull() { - ssb.b.Set(lblName, val.String()) - } - } + ss := &chunkSeries{ + lset: lbls, + mint: q.mint, + maxt: q.maxt, } - chks := make([]chunkenc.Chunk, 0, 12) - for _, bs := range chksBytes { - for len(bs) != 0 { - enc := chunkenc.Encoding(binary.BigEndian.Uint32(bs[:4])) - bs = bs[4:] - mint := encoding.ZigZagDecode(binary.BigEndian.Uint64(bs[:8])) - bs = bs[8:] - maxt := encoding.ZigZagDecode(binary.BigEndian.Uint64(bs[:8])) - bs = bs[8:] - l := binary.BigEndian.Uint32(bs[:4]) - bs = bs[4:] - if util.Intersects(mint, maxt, ssb.mint, ssb.maxt) { - chk, err := chunkenc.FromData(enc, bs[:l]) - if err != nil { - return i, fmt.Errorf("unable to create chunk from data: %s", err) - } - chks = append(chks, chk) - } - bs = bs[l:] + for j := range sc[i].Chunks { + if !util.Intersects(q.mint, q.maxt, sc[i].Chunks[j].MinTime, sc[i].Chunks[j].MaxTime) { + continue } + ss.chunks = append(ss.chunks, sc[i].Chunks[j].Chunk) } - - ssb.extlabels.Range(func(lbl labels.Label) { ssb.b.Set(lbl.Name, lbl.Value) }) - for _, lbl := range ssb.replicaLabelNames { - ssb.b.Del(lbl) - } - - lbls := ssb.b.Labels() - - h := lbls.Hash() - if _, ok := ssb.m[h]; ok { - // We have seen this series before, skip it for now; we could be smarter and select - // chunks appropriately so that we fill in what might be missing but for now skipping is fine - continue - } - ssb.m[h] = struct{}{} - - ssb.s = append(ssb.s, &chunkSeries{ - lset: lbls, - mint: ssb.mint, - maxt: ssb.maxt, - chunks: chks, - }) + res = append(res, ss) } - return len(rs), nil -} - -func (ssb *seriesSetBuilder) Series() []storage.Series { - res := make([]storage.Series, 0, len(ssb.s)) - for _, v := range ssb.s { - res = append(res, v) - } - return res + return res, dropped } diff --git a/db/syncer.go b/db/syncer.go deleted file mode 100644 index e58e773..0000000 --- a/db/syncer.go +++ /dev/null @@ -1,319 +0,0 @@ -// Copyright (c) 2025 Cloudflare, Inc. -// Licensed under the Apache 2.0 license found in the LICENSE file or at: -// https://opensource.org/licenses/Apache-2.0 - -package db - -import ( - "bytes" - "context" - "fmt" - "io" - "maps" - "slices" - "sort" - "sync" - "time" - - "github.com/alecthomas/units" - "github.com/parquet-go/parquet-go" - "github.com/thanos-io/objstore" - "github.com/thanos-io/thanos/pkg/info/infopb" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - - "github.com/cloudflare/parquet-tsdb-poc/internal/util" - "github.com/cloudflare/parquet-tsdb-poc/schema" -) - -type Syncer struct { - bkt objstore.Bucket - - blockOpts []BlockOption - metaFilter MetaFilter - concurrency int - - mu sync.Mutex - blocks map[string]*Block - - cached []*Block -} - -type SyncerOption func(*syncerConfig) - -type syncerConfig struct { - blockOpts []BlockOption - metaFilter MetaFilter - concurrency int -} - -func BlockOptions(opts ...BlockOption) SyncerOption { - return func(cfg *syncerConfig) { - cfg.blockOpts = opts - } -} - -func FilterMetas(f MetaFilter) SyncerOption { - return func(cfg *syncerConfig) { - cfg.metaFilter = f - } -} - -func BlockConcurrency(c int) SyncerOption { - return func(cfg *syncerConfig) { - cfg.concurrency = c - } -} - -type BlockOption func(*blockConfig) - -type blockConfig struct { - readBufferSize units.Base2Bytes -} - -func ReadBufferSize(sz units.Base2Bytes) BlockOption { - return func(cfg *blockConfig) { - cfg.readBufferSize = sz - } -} - -func NewSyncer(bkt objstore.Bucket, opts ...SyncerOption) *Syncer { - cfg := syncerConfig{ - metaFilter: AllMetasMetaFilter, - concurrency: 1, - } - - for _, o := range opts { - o(&cfg) - } - - return &Syncer{ - bkt: bkt, - blocks: make(map[string]*Block), - blockOpts: cfg.blockOpts, - metaFilter: cfg.metaFilter, - concurrency: cfg.concurrency, - } -} - -func (s *Syncer) Blocks() []*Block { - s.mu.Lock() - defer s.mu.Unlock() - - return s.filterBlocks(s.cached) -} - -func (s *Syncer) Sync(ctx context.Context, metas map[string]Meta) error { - type blockOrError struct { - blk *Block - err error - } - - blkC := make(chan blockOrError) - go func() { - defer close(blkC) - - workerC := make(chan Meta, s.concurrency) - go func() { - defer close(workerC) - - for k, v := range s.filterMetas(metas) { - if _, ok := s.blocks[k]; ok { - continue - } - workerC <- v - } - }() - - var wg sync.WaitGroup - defer wg.Wait() - - for i := 0; i < s.concurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - for m := range workerC { - blk, err := newBlockForMeta(ctx, s.bkt, m, s.blockOpts...) - if err != nil { - blkC <- blockOrError{err: fmt.Errorf("unable to read block %q: %w", m.Name, err)} - } else { - blkC <- blockOrError{blk: blk} - } - } - }() - } - }() - - blocks := make(map[string]*Block, 0) - for b := range blkC { - if b.err != nil { - return fmt.Errorf("unable to read block: %w", b.err) - } - blocks[b.blk.meta.Name] = b.blk - } - - s.mu.Lock() - defer s.mu.Unlock() - - // delete blocks that are not in meta map - maps.DeleteFunc(s.blocks, func(k string, _ *Block) bool { _, ok := metas[k]; return !ok }) - - // add new blocks that we just loaded - maps.Copy(s.blocks, blocks) - - s.cached = slices.Collect(maps.Values(s.blocks)) - sort.Slice(s.cached, func(i, j int) bool { - ls, _ := s.cached[i].Timerange() - rs, _ := s.cached[j].Timerange() - return ls < rs - }) - return nil -} - -func (s *Syncer) filterMetas(metas map[string]Meta) map[string]Meta { - return s.metaFilter.filterMetas(metas) -} - -func (s *Syncer) filterBlocks(blks []*Block) []*Block { - return s.metaFilter.filterBlocks(blks) -} - -type MetaFilter interface { - filterMetas(map[string]Meta) map[string]Meta - filterBlocks([]*Block) []*Block -} - -var AllMetasMetaFilter = allMetas{} - -type allMetas struct { -} - -func (mf allMetas) filterMetas(metas map[string]Meta) map[string]Meta { return metas } -func (mf allMetas) filterBlocks(blocks []*Block) []*Block { return blocks } - -type ThanosBackfillMetaFilter struct { - endpoint string - - mu sync.Mutex - mint, maxt int64 -} - -func NewThanosBackfillMetaFilter(endpoint string) *ThanosBackfillMetaFilter { - return &ThanosBackfillMetaFilter{endpoint: endpoint} -} - -func (tp *ThanosBackfillMetaFilter) filterMetas(metas map[string]Meta) map[string]Meta { - tp.mu.Lock() - defer tp.mu.Unlock() - - res := make(map[string]Meta, len(metas)) - for k, v := range metas { - if util.Contains(tp.mint, tp.maxt, v.Mint, v.Maxt) { - continue - } - res[k] = v - } - return res -} - -func (tp *ThanosBackfillMetaFilter) filterBlocks(blks []*Block) []*Block { - tp.mu.Lock() - defer tp.mu.Unlock() - - res := make([]*Block, 0, len(blks)) - for _, blk := range blks { - blkMint, blkMaxt := blk.Timerange() - if util.Contains(tp.mint, tp.maxt, blkMint, blkMaxt) { - continue - } - res = append(res, blk) - } - return res -} - -func (tp *ThanosBackfillMetaFilter) Update(ctx context.Context) error { - // Note: we assume that thanos runs close to this server, we dont need TLS here - cc, err := grpc.NewClient(tp.endpoint, grpc.WithTransportCredentials(insecure.NewCredentials())) - if err != nil { - return fmt.Errorf("unable to connect: %w", err) - } - client := infopb.NewInfoClient(cc) - - info, err := client.Info(ctx, &infopb.InfoRequest{}) - if err != nil { - return fmt.Errorf("unable to get store time range from thanos: %w", err) - } - - tp.mu.Lock() - defer tp.mu.Unlock() - - tp.mint = info.Store.MinTime - tp.maxt = info.Store.MaxTime - - return nil -} - -func newBlockForMeta(ctx context.Context, bkt objstore.Bucket, m Meta, opts ...BlockOption) (*Block, error) { - cfg := blockConfig{ - readBufferSize: 8 * units.MiB, - } - for _, o := range opts { - o(&cfg) - } - - shards, err := readShards(ctx, bkt, m, cfg) - if err != nil { - return nil, fmt.Errorf("unable to read shards: %w", err) - } - - return &Block{meta: m, shards: shards}, nil -} - -func readShards(ctx context.Context, bkt objstore.Bucket, m Meta, cfg blockConfig) ([]*Shard, error) { - shards := make([]*Shard, 0, m.Shards) - for i := 0; i != int(m.Shards); i++ { - shard, err := readShard(ctx, bkt, m, i, cfg) - if err != nil { - return nil, fmt.Errorf("unable to read shard %d: %w", i, err) - } - shards = append(shards, shard) - } - return shards, nil -} - -func readShard(ctx context.Context, bkt objstore.Bucket, m Meta, i int, cfg blockConfig) (*Shard, error) { - chunkspfile := schema.ChunksPfileNameForShard(m.Name, i) - attrs, err := bkt.Attributes(ctx, chunkspfile) - if err != nil { - return nil, fmt.Errorf("unable to attr %s: %w", chunkspfile, err) - } - - bktRdrAt := newBucketReaderAt(bkt, chunkspfile, 1*time.Minute) - - chunkspf, err := parquet.OpenFile(bktRdrAt, attrs.Size, - parquet.FileReadMode(parquet.ReadModeAsync), - parquet.ReadBufferSize(int(cfg.readBufferSize)), - ) - if err != nil { - return nil, fmt.Errorf("unable to open parquet file %s: %w", chunkspfile, err) - } - - labelspfile := schema.LabelsPfileNameForShard(m.Name, i) - rdr, err := bkt.Get(ctx, labelspfile) - if err != nil { - return nil, fmt.Errorf("unable to get %s: %w", labelspfile, err) - } - defer rdr.Close() - - labelspfileBs, err := io.ReadAll(rdr) - if err != nil { - return nil, fmt.Errorf("unable to read %s: %w", labelspfile, err) - } - - labelspf, err := parquet.OpenFile(bytes.NewReader(labelspfileBs), int64(len(labelspfileBs))) - if err != nil { - return nil, fmt.Errorf("unable to open parquet file %s: %w", labelspfile, err) - } - - return &Shard{meta: m, chunkspfile: chunkspf, labelspfile: labelspf}, nil -} diff --git a/db/util.go b/db/util.go deleted file mode 100644 index 006705c..0000000 --- a/db/util.go +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) 2025 Cloudflare, Inc. -// Licensed under the Apache 2.0 license found in the LICENSE file or at: -// https://opensource.org/licenses/Apache-2.0 - -package db - -import ( - "context" - "fmt" - "io" - "time" - - "github.com/thanos-io/objstore" -) - -// TODO: timeouts retries whatnot -type bucketReaderAt struct { - bkt objstore.Bucket - name string - - timeout time.Duration -} - -func newBucketReaderAt(bkt objstore.Bucket, name string, timeout time.Duration) *bucketReaderAt { - return &bucketReaderAt{ - bkt: bkt, - name: name, - timeout: timeout, - } -} - -func (br *bucketReaderAt) ReadAt(p []byte, off int64) (n int, err error) { - ctx, cancel := context.WithTimeout(context.Background(), br.timeout) - defer cancel() - - bucketRequests.Inc() - - rdc, err := br.bkt.GetRange(ctx, br.name, off, int64(len(p))) - if err != nil { - return 0, fmt.Errorf("unable to read range for %s: %w", br.name, err) - } - defer rdc.Close() - - return io.ReadFull(rdc, p) -} diff --git a/go.mod b/go.mod index 81d7704..f5baef5 100644 --- a/go.mod +++ b/go.mod @@ -1,58 +1,111 @@ module github.com/cloudflare/parquet-tsdb-poc -go 1.23.1 +go 1.24.2 require ( - github.com/KimMachineGun/automemlimit v0.6.1 - github.com/google/go-cmp v0.6.0 - github.com/hashicorp/go-multierror v1.1.1 + github.com/AdaLogics/go-fuzz-headers v0.0.0-20240806141605-e8a1dd7889d6 + github.com/KimMachineGun/automemlimit v0.7.1 + github.com/google/go-cmp v0.7.0 + github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1 + github.com/json-iterator/go v1.1.12 github.com/mostynb/go-grpc-compression v1.2.3 github.com/oklog/run v1.1.0 - github.com/parquet-go/parquet-go v0.25.0 - github.com/prometheus/client_golang v1.20.5 - github.com/prometheus/common v0.61.0 - github.com/prometheus/prometheus v0.300.0-beta.0.0.20241007135006-65f610353919 - github.com/thanos-io/objstore v0.0.0-20241111205755-d1dd89d41f97 - github.com/thanos-io/promql-engine v0.0.0-20241203103240-2f49f80c7c68 + github.com/oklog/ulid/v2 v2.1.0 + github.com/parquet-go/parquet-go v0.25.1 + github.com/prometheus/client_golang v1.22.0 + github.com/prometheus/common v0.63.0 + github.com/prometheus/prometheus v0.303.1 + github.com/thanos-io/objstore v0.0.0-20250317105316-a0136a6f898d + github.com/thanos-io/promql-engine v0.0.0-20250428192047-2eb963d84212 github.com/thanos-io/thanos v0.37.2 - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.57.0 - go.opentelemetry.io/otel v1.32.0 + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.60.0 + go.opentelemetry.io/contrib/propagators/autoprop v0.54.0 + go.opentelemetry.io/otel v1.35.0 go.opentelemetry.io/otel/exporters/jaeger v1.17.0 go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.32.0 - go.opentelemetry.io/otel/sdk v1.32.0 - google.golang.org/protobuf v1.35.2 + go.opentelemetry.io/otel/sdk v1.35.0 + google.golang.org/protobuf v1.36.6 gopkg.in/alecthomas/kingpin.v2 v2.2.6 gopkg.in/yaml.v3 v3.0.1 ) require ( + github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect + github.com/envoyproxy/go-control-plane/envoy v1.32.4 // indirect + github.com/fatih/color v1.18.0 // indirect github.com/go-kit/log v0.2.1 // indirect + github.com/go-openapi/analysis v0.23.0 // indirect + github.com/go-openapi/errors v0.22.0 // indirect + github.com/go-openapi/jsonpointer v0.21.0 // indirect + github.com/go-openapi/jsonreference v0.21.0 // indirect + github.com/go-openapi/loads v0.22.0 // indirect + github.com/go-openapi/spec v0.21.0 // indirect + github.com/go-openapi/strfmt v0.23.0 // indirect + github.com/go-openapi/swag v0.23.0 // indirect + github.com/go-openapi/validate v0.24.0 // indirect + github.com/go-viper/mapstructure/v2 v2.2.1 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 // indirect + github.com/josharian/intern v1.0.0 // indirect + github.com/knadh/koanf/maps v0.1.1 // indirect + github.com/knadh/koanf/providers/confmap v0.1.0 // indirect + github.com/knadh/koanf/v2 v2.1.2 // indirect + github.com/mailru/easyjson v0.7.7 // indirect + github.com/mattn/go-colorable v0.1.14 // indirect + github.com/minio/sha256-simd v1.0.1 // 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.2 // indirect + github.com/oklog/ulid v1.3.1 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.121.0 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.121.0 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.121.0 // indirect + github.com/prometheus/alertmanager v0.28.0 // indirect + github.com/prometheus/sigv4 v0.1.2 // indirect + go.mongodb.org/mongo-driver v1.14.0 // indirect + go.opentelemetry.io/auto/sdk v1.1.0 // indirect + go.opentelemetry.io/collector/component v1.27.0 // indirect + go.opentelemetry.io/collector/confmap v1.27.0 // indirect + go.opentelemetry.io/collector/confmap/xconfmap v0.121.0 // indirect + go.opentelemetry.io/collector/consumer v1.27.0 // indirect + go.opentelemetry.io/collector/pdata v1.27.0 // indirect + go.opentelemetry.io/collector/pipeline v0.121.0 // indirect + go.opentelemetry.io/collector/processor v0.121.0 // indirect + go.opentelemetry.io/collector/semconv v0.121.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.60.0 // indirect + go.opentelemetry.io/contrib/propagators/aws v1.29.0 // indirect + go.opentelemetry.io/contrib/propagators/b3 v1.29.0 // indirect + go.opentelemetry.io/contrib/propagators/jaeger v1.29.0 // indirect + go.opentelemetry.io/contrib/propagators/ot v1.29.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.27.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect + golang.org/x/tools v0.34.0 // indirect ) require ( - cel.dev/expr v0.18.0 // indirect - cloud.google.com/go v0.116.0 // indirect - cloud.google.com/go/auth v0.11.0 // indirect - cloud.google.com/go/auth/oauth2adapt v0.2.6 // indirect - cloud.google.com/go/compute/metadata v0.5.2 // indirect - cloud.google.com/go/iam v1.2.2 // indirect - cloud.google.com/go/monitoring v1.21.2 // indirect - cloud.google.com/go/storage v1.47.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.16.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.0 // indirect + cel.dev/expr v0.19.2 // indirect + cloud.google.com/go v0.118.1 // indirect + cloud.google.com/go/auth v0.15.0 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.7 // indirect + cloud.google.com/go/compute/metadata v0.6.0 // indirect + cloud.google.com/go/iam v1.3.1 // indirect + cloud.google.com/go/monitoring v1.24.0 // indirect + cloud.google.com/go/storage v1.50.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.17.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.2 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.5.0 // indirect - github.com/AzureAD/microsoft-authentication-library-for-go v1.3.2 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0 // indirect + github.com/AzureAD/microsoft-authentication-library-for-go v1.3.3 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.26.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.50.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.50.0 // indirect github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 // indirect github.com/alecthomas/units v0.0.0-20240927000941-0f3dac36c52b github.com/aliyun/aliyun-oss-go-sdk v3.0.2+incompatible // indirect github.com/andybalholm/brotli v1.1.1 // indirect - github.com/aws/aws-sdk-go v1.55.5 // indirect + github.com/aws/aws-sdk-go v1.55.6 // indirect github.com/aws/aws-sdk-go-v2 v1.32.5 // indirect github.com/aws/aws-sdk-go-v2/config v1.28.5 // indirect github.com/aws/aws-sdk-go-v2/credentials v1.17.46 // indirect @@ -69,42 +122,32 @@ require ( github.com/baidubce/bce-sdk-go v0.9.201 // indirect github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect - github.com/cilium/ebpf v0.16.0 // indirect github.com/clbanning/mxj v1.8.4 // indirect - github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 // indirect - github.com/containerd/cgroups/v3 v3.0.4 // indirect - github.com/containerd/log v0.1.0 // indirect - github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dennwc/varint v1.0.0 // indirect - github.com/docker/go-units v0.5.0 // indirect github.com/dustin/go-humanize v1.0.1 // indirect github.com/edsrzf/mmap-go v1.2.0 // indirect github.com/efficientgo/core v1.0.0-rc.3 // indirect - github.com/envoyproxy/go-control-plane v0.13.1 // indirect - github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect + github.com/envoyproxy/protoc-gen-validate v1.2.1 // indirect github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/go-ini/ini v1.67.0 // indirect github.com/go-logfmt/logfmt v0.6.0 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/goccy/go-json v0.10.3 // indirect - github.com/godbus/dbus/v5 v5.1.0 // indirect + github.com/goccy/go-json v0.10.4 // indirect github.com/gofrs/flock v0.12.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt/v5 v5.2.1 // indirect - github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/go-querystring v1.1.0 // indirect - github.com/google/s2a-go v0.1.8 // 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.4 // indirect - github.com/googleapis/gax-go/v2 v2.14.0 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.5 // indirect + github.com/googleapis/gax-go/v2 v2.14.1 // indirect github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc // indirect - github.com/hashicorp/errwrap v1.1.0 // indirect github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/jpillora/backoff v1.0.0 // indirect @@ -112,18 +155,13 @@ require ( github.com/klauspost/compress v1.18.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect github.com/kylelemons/godebug v1.1.0 // indirect - github.com/mattn/go-runewidth v0.0.16 // indirect github.com/minio/md5-simd v1.1.2 // indirect - github.com/minio/minio-go/v7 v7.0.81 // indirect + github.com/minio/minio-go/v7 v7.0.83 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect - github.com/moby/sys/userns v0.1.0 // indirect github.com/mozillazg/go-httpheader v0.4.0 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect github.com/ncw/swift v1.0.53 // indirect - github.com/oklog/ulid v1.3.1 // indirect - github.com/olekukonko/tablewriter v0.0.5 // indirect - github.com/opencontainers/runtime-spec v1.2.0 // indirect github.com/oracle/oci-go-sdk/v65 v65.79.0 // indirect github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect @@ -131,43 +169,37 @@ require ( github.com/pkg/errors v0.9.1 // 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_model v0.6.1 // indirect - github.com/prometheus/common/sigv4 v0.1.0 // indirect - github.com/prometheus/procfs v0.15.1 // indirect - github.com/rivo/uniseg v0.4.7 // indirect - github.com/rogpeppe/go-internal v1.13.1 // indirect + github.com/prometheus/client_model v0.6.2 // indirect + github.com/prometheus/procfs v0.16.1 // indirect github.com/rs/xid v1.6.0 // indirect - github.com/sirupsen/logrus v1.9.3 // indirect github.com/sony/gobreaker v1.0.0 // indirect github.com/stretchr/testify v1.10.0 // indirect github.com/tencentyun/cos-go-sdk-v5 v0.7.59 // indirect github.com/zhangyunhao116/umap v0.0.0-20241028085443-797b8ba93f64 // indirect - go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.32.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.57.0 // indirect - go.opentelemetry.io/otel/metric v1.32.0 // indirect - go.opentelemetry.io/otel/sdk/metric v1.32.0 // indirect - go.opentelemetry.io/otel/trace v1.32.0 + go.opentelemetry.io/contrib/detectors/gcp v1.34.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 + go.opentelemetry.io/otel/metric v1.35.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.35.0 // indirect + go.opentelemetry.io/otel/trace v1.35.0 go.uber.org/atomic v1.11.0 // indirect go.uber.org/goleak v1.3.0 // indirect - golang.org/x/crypto v0.35.0 // indirect - golang.org/x/exp v0.0.0-20241108190413-2d47ceb2692f // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/oauth2 v0.24.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect - golang.org/x/time v0.8.0 // indirect + golang.org/x/crypto v0.39.0 // indirect + golang.org/x/exp v0.0.0-20250106191152-7588d65b2ba8 // indirect + golang.org/x/net v0.41.0 // indirect + golang.org/x/oauth2 v0.27.0 // indirect + golang.org/x/sync v0.15.0 + golang.org/x/sys v0.33.0 // indirect + golang.org/x/text v0.26.0 // indirect + golang.org/x/time v0.10.0 // indirect gonum.org/v1/gonum v0.15.1 // indirect - google.golang.org/api v0.209.0 // indirect - google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697 // indirect - google.golang.org/grpc v1.68.0 - google.golang.org/grpc/stats/opentelemetry v0.0.0-20241028142157-ada6787961b3 // indirect + google.golang.org/api v0.224.0 // indirect + google.golang.org/genproto v0.0.0-20250204164813-702378808489 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20250218202821-56aae31c358a // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250227231956-55c901821b1e // indirect + google.golang.org/grpc v1.71.0 gopkg.in/yaml.v2 v2.4.0 // indirect - k8s.io/apimachinery v0.31.3 // indirect - k8s.io/client-go v0.31.3 // indirect + k8s.io/apimachinery v0.32.2 // indirect + k8s.io/client-go v0.32.2 // indirect k8s.io/klog/v2 v2.130.1 // indirect k8s.io/utils v0.0.0-20241104163129-6fe5fd82f078 // indirect ) diff --git a/go.sum b/go.sum index 3567d4c..5953845 100644 --- a/go.sum +++ b/go.sum @@ -1,64 +1,33 @@ -cel.dev/expr v0.18.0 h1:CJ6drgk+Hf96lkLikr4rFf19WrU0BOWEihyZnI2TAzo= -cel.dev/expr v0.18.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= -cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= -cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= -cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= -cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= -cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= -cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= -cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= -cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go v0.116.0 h1:B3fRrSDkLRt5qSHWe40ERJvhvnQwdZiHu0bJOpldweE= -cloud.google.com/go v0.116.0/go.mod h1:cEPSRWPzZEswwdr9BxE6ChEn01dWlTaF05LiC2Xs70U= -cloud.google.com/go/auth v0.11.0 h1:Ic5SZz2lsvbYcWT5dfjNWgw6tTlGi2Wc8hyQSC9BstA= -cloud.google.com/go/auth v0.11.0/go.mod h1:xxA5AqpDrvS+Gkmo9RqrGGRh6WSNKKOXhY3zNOr38tI= -cloud.google.com/go/auth/oauth2adapt v0.2.6 h1:V6a6XDu2lTwPZWOawrAa9HUK+DB2zfJyTuciBG5hFkU= -cloud.google.com/go/auth/oauth2adapt v0.2.6/go.mod h1:AlmsELtlEBnaNTL7jCj8VQFLy6mbZv0s4Q7NGBeQ5E8= -cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= -cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= -cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= -cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= -cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= -cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/iam v1.2.2 h1:ozUSofHUGf/F4tCNy/mu9tHLTaxZFLOUiKzjcgWHGIA= -cloud.google.com/go/iam v1.2.2/go.mod h1:0Ys8ccaZHdI1dEUilwzqng/6ps2YB6vRsjIe00/+6JY= -cloud.google.com/go/logging v1.12.0 h1:ex1igYcGFd4S/RZWOCU51StlIEuey5bjqwH9ZYjHibk= -cloud.google.com/go/logging v1.12.0/go.mod h1:wwYBt5HlYP1InnrtYI0wtwttpVU1rifnMT7RejksUAM= -cloud.google.com/go/longrunning v0.6.2 h1:xjDfh1pQcWPEvnfjZmwjKQEcHnpz6lHjfy7Fo0MK+hc= -cloud.google.com/go/longrunning v0.6.2/go.mod h1:k/vIs83RN4bE3YCswdXC5PFfWVILjm3hpEUlSko4PiI= -cloud.google.com/go/monitoring v1.21.2 h1:FChwVtClH19E7pJ+e0xUhJPGksctZNVOk2UhMmblmdU= -cloud.google.com/go/monitoring v1.21.2/go.mod h1:hS3pXvaG8KgWTSz+dAdyzPrGUYmi2Q+WFX8g2hqVEZU= -cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= -cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= -cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= -cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= -cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= -cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= -cloud.google.com/go/storage v1.47.0 h1:ajqgt30fnOMmLfWfu1PWcb+V9Dxz6n+9WKjdNg5R4HM= -cloud.google.com/go/storage v1.47.0/go.mod h1:Ks0vP374w0PW6jOUameJbapbQKXqkjGd/OJRp2fb9IQ= -cloud.google.com/go/trace v1.11.2 h1:4ZmaBdL8Ng/ajrgKqY5jfvzqMXbrDcBsUGXOT9aqTtI= -cloud.google.com/go/trace v1.11.2/go.mod h1:bn7OwXd4pd5rFuAnTrzBuoZ4ax2XQeG3qNgYmfCy0Io= -dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.16.0 h1:JZg6HRh6W6U4OLl6lk7BZ7BLisIzM9dG1R50zUk9C/M= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.16.0/go.mod h1:YL1xnZ6QejvQHWJrX/AvhFl4WW4rqHVoKspWNVwFk0M= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.0 h1:B/dfvscEQtew9dVuoxqxrUKKv8Ih2f55PydknDamU+g= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.0/go.mod h1:fiPSssYvltE08HJchL04dOy+RD4hgrjph0cwGGMntdI= -github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.3.0 h1:+m0M/LFxN43KvULkDNfdXOgrjtg6UYJPFBJyuEcRCAw= -github.com/Azure/azure-sdk-for-go/sdk/azidentity/cache v0.3.0/go.mod h1:PwOyop78lveYMRs6oCxjiVyBdyCgIYH6XHIVZO9/SFQ= +cel.dev/expr v0.19.2 h1:V354PbqIXr9IQdwy4SYA4xa0HXaWq1BUPAGzugBY5V4= +cel.dev/expr v0.19.2/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= +cloud.google.com/go v0.118.1 h1:b8RATMcrK9A4BH0rj8yQupPXp+aP+cJ0l6H7V9osV1E= +cloud.google.com/go v0.118.1/go.mod h1:CFO4UPEPi8oV21xoezZCrd3d81K4fFkDTEJu4R8K+9M= +cloud.google.com/go/auth v0.15.0 h1:Ly0u4aA5vG/fsSsxu98qCQBemXtAtJf+95z9HK+cxps= +cloud.google.com/go/auth v0.15.0/go.mod h1:WJDGqZ1o9E9wKIL+IwStfyn/+s59zl4Bi+1KQNVXLZ8= +cloud.google.com/go/auth/oauth2adapt v0.2.7 h1:/Lc7xODdqcEw8IrZ9SvwnlLX6j9FHQM74z6cBk9Rw6M= +cloud.google.com/go/auth/oauth2adapt v0.2.7/go.mod h1:NTbTTzfvPl1Y3V1nPpOgl2w6d/FjO7NNUQaWSox6ZMc= +cloud.google.com/go/compute/metadata v0.6.0 h1:A6hENjEsCDtC1k8byVsgwvVcioamEHvZ4j01OwKxG9I= +cloud.google.com/go/compute/metadata v0.6.0/go.mod h1:FjyFAW1MW0C203CEOMDTu3Dk1FlqW3Rga40jzHL4hfg= +cloud.google.com/go/iam v1.3.1 h1:KFf8SaT71yYq+sQtRISn90Gyhyf4X8RGgeAVC8XGf3E= +cloud.google.com/go/iam v1.3.1/go.mod h1:3wMtuyT4NcbnYNPLMBzYRFiEfjKfJlLVLrisE7bwm34= +cloud.google.com/go/logging v1.13.0 h1:7j0HgAp0B94o1YRDqiqm26w4q1rDMH7XNRU34lJXHYc= +cloud.google.com/go/logging v1.13.0/go.mod h1:36CoKh6KA/M0PbhPKMq6/qety2DCAErbhXT62TuXALA= +cloud.google.com/go/longrunning v0.6.4 h1:3tyw9rO3E2XVXzSApn1gyEEnH2K9SynNQjMlBi3uHLg= +cloud.google.com/go/longrunning v0.6.4/go.mod h1:ttZpLCe6e7EXvn9OxpBRx7kZEB0efv8yBO6YnVMfhJs= +cloud.google.com/go/monitoring v1.24.0 h1:csSKiCJ+WVRgNkRzzz3BPoGjFhjPY23ZTcaenToJxMM= +cloud.google.com/go/monitoring v1.24.0/go.mod h1:Bd1PRK5bmQBQNnuGwHBfUamAV1ys9049oEPHnn4pcsc= +cloud.google.com/go/storage v1.50.0 h1:3TbVkzTooBvnZsk7WaAQfOsNrdoM8QHusXA1cpk6QJs= +cloud.google.com/go/storage v1.50.0/go.mod h1:l7XeiD//vx5lfqE3RavfmU9yvk5Pp0Zhcv482poyafY= +cloud.google.com/go/trace v1.11.3 h1:c+I4YFjxRQjvAhRmSsmjpASUKq88chOX854ied0K/pE= +cloud.google.com/go/trace v1.11.3/go.mod h1:pt7zCYiDSQjC9Y2oqCsh9jF4GStB/hmjrYLsxRR27q8= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20240806141605-e8a1dd7889d6 h1:He8afgbRMd7mFxO99hRNu+6tazq8nFF9lIwo9JFroBk= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20240806141605-e8a1dd7889d6/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.17.0 h1:g0EZJwz7xkXQiZAI5xi9f3WWFYBlX1CPTrR+NDToRkQ= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.17.0/go.mod h1:XCW7KnZet0Opnr7HccfUw1PLc4CjHqpcaxW8DHklNkQ= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.2 h1:F0gBpfdPLGsw+nsgk6aqqkZS1jiixa5WwFe3fk/T3Ys= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.8.2/go.mod h1:SqINnQ9lVVdRlyC8cd1lCI0SdX4n2paeABd2K8ggfnE= +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= github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 h1:ywEEhmNahHBihViHepv3xPBn1663uRv2t2q/ESv9seY= github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0/go.mod h1:iZDifYGJTIgIIkYRNWPENUnqx6bJ2xnSDFI2tjwZNuY= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.7.0 h1:LkHbJbgF3YyvC53aqYGR+wWQDn2Rdp9AQdGndf9QvY4= @@ -71,31 +40,25 @@ github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.5.0 h1:mlmW46Q0B79I+Aj4a github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.5.0/go.mod h1:PXe2h+LKcWTX9afWdZoHyODqR4fBa5boUM/8uJfZ0Jo= 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.3.2 h1:kYRSnvJju5gYVyhkij+RTJ/VR6QIUaCfWeaFm2ycsjQ= -github.com/AzureAD/microsoft-authentication-library-for-go v1.3.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/AzureAD/microsoft-authentication-library-for-go v1.3.3 h1:H5xDQaE3XowWfhZRUpnfC+rGZMEVoSiji+b+/HFAPU4= +github.com/AzureAD/microsoft-authentication-library-for-go v1.3.3/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= 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.25.0 h1:3c8yed4lgqTt+oTQ+JNMDo+F4xprBf+O/il4ZC0nRLw= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0 h1:o90wcURuxekmXrtxmYWTyNla0+ZEHhud6DI1ZTxd1vI= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0/go.mod h1:6fTWu4m3jocfUZLYF5KsZC1TUfRvEjs7lM4crme/irw= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.49.0 h1:jJKWl98inONJAr/IZrdFQUWcwUO95DLY1XMD1ZIut+g= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.49.0/go.mod h1:l2fIqmwB+FKSfvn3bAD/0i+AXAxhIZjTK2svT/mgUXs= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0 h1:GYUJLfvd++4DMuMhCFLgLXvFwofIxh/qOwoGuS/LTew= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0/go.mod h1:wRbFgBQUVm1YXrvWKofAEmq9HNJTDphbAaJSSX01KUI= -github.com/KimMachineGun/automemlimit v0.6.1 h1:ILa9j1onAAMadBsyyUJv5cack8Y1WT26yLj/V+ulKp8= -github.com/KimMachineGun/automemlimit v0.6.1/go.mod h1:T7xYht7B8r6AG/AqFcUdc7fzd2bIdBKmepfP2S1svPY= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.26.0 h1:f2Qw/Ehhimh5uO1fayV0QIW7DShEQqhtUfhYc+cBPlw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.26.0/go.mod h1:2bIszWvQRlJVmJLiuLhukLImRjKPcYdzzsx6darK02A= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.50.0 h1:5IT7xOdq17MtcdtL/vtl6mGfzhaq4m4vpollPRmlsBQ= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.50.0/go.mod h1:ZV4VOm0/eHR06JLrXWe09068dHpr3TRpY9Uo7T+anuA= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.50.0 h1:nNMpRpnkWDAaqcpxMJvxa/Ud98gjbYwayJY4/9bdjiU= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.50.0/go.mod h1:SZiPHWGOOk3bl8tkevxkoiwPgsIl6CwrWcbwjfHZpdM= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.50.0 h1:ig/FpDD2JofP/NExKQUbn7uOSZzJAQqogfqluZK4ed4= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.50.0/go.mod h1:otE2jQekW/PqXk1Awf5lmfokJx4uwuqcj1ab5SpGeW0= +github.com/KimMachineGun/automemlimit v0.7.1 h1:QcG/0iCOLChjfUweIMC3YL5Xy9C3VBeNmCZHrZfJMBw= +github.com/KimMachineGun/automemlimit v0.7.1/go.mod h1:QZxpHaGOQoYvFhv/r4u3U0JTC2ZcOwbSr11UZF46UBM= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/QcloudApi/qcloud_sign_golang v0.0.0-20141224014652-e4130a326409/go.mod h1:1pk82RBxDY/JZnPQrtqHlUFfCctgdorsd9M06fMynOM= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alecthomas/units v0.0.0-20240927000941-0f3dac36c52b h1:mimo19zliBX/vSQ6PWWSL9lK8qwHozUj03+zLoEB8O0= github.com/alecthomas/units v0.0.0-20240927000941-0f3dac36c52b/go.mod h1:fvzegU4vN3H1qMT+8wDmzjAcDONcgo2/SZ/TyfdUOFs= github.com/aliyun/aliyun-oss-go-sdk v3.0.2+incompatible h1:8psS8a+wKfiLt1iVDX79F7Y6wUM49Lcha2FMXt4UM8g= @@ -104,9 +67,10 @@ github.com/andybalholm/brotli v1.1.1 h1:PR2pgnyFznKEugtsUo0xLdDop5SKXd5Qf5ysW+7X github.com/andybalholm/brotli v1.1.1/go.mod h1:05ib4cKhjx3OQYUY22hTVd34Bc8upXjOLL2rKwwZBoA= 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 v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= -github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU= -github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= +github.com/aws/aws-sdk-go v1.55.6 h1:cSg4pvZ3m8dgYcgqB97MrcdjUmZ1BeMYKUxMMB89IPk= +github.com/aws/aws-sdk-go v1.55.6/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v1.32.5 h1:U8vdWJuY7ruAkzaOdD7guwJjD06YSKmnKCJs7s3IkIo= github.com/aws/aws-sdk-go-v2 v1.32.5/go.mod h1:P5WJBrYqqbWVaOxgH0X/FYYD47/nooaPOZPlQdmiN2U= github.com/aws/aws-sdk-go-v2/config v1.28.5 h1:Za41twdCXbuyyWv9LndXxZZv3QhTG1DinqlFsSuvtI0= @@ -137,37 +101,18 @@ github.com/baidubce/bce-sdk-go v0.9.201 h1:gIuvsE6azuwICmPc/P3IfyUk/vU3IqkwkT3J9 github.com/baidubce/bce-sdk-go v0.9.201/go.mod h1:zbYJMQwE4IZuyrJiFO8tO8NbtYiKTFTbwh4eIsqjVdg= 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 v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= 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/bluele/gcache v0.0.2 h1:WcbfdXICg7G/DGBh1PFfcirkWOQV+v077yF1pSy3DGw= github.com/bluele/gcache v0.0.2/go.mod h1:m15KV+ECjptwSPxKhOhQoAFQVtUFjTVkc3H8o0t/fp0= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMrBo8f1j86j5WHzznCCQxV/b8g= -github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= -github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/cilium/ebpf v0.16.0 h1:+BiEnHL6Z7lXnlGUsXQPPAE7+kenAd4ES8MQ5min0Ok= -github.com/cilium/ebpf v0.16.0/go.mod h1:L7u2Blt2jMM/vLAVgjxluxtBKlz3/GWjB0dMOEngfwE= github.com/clbanning/mxj v1.8.4 h1:HuhwZtbyvyOw+3Z1AowPkU87JkJUSv751ELWaiTpj8I= github.com/clbanning/mxj v1.8.4/go.mod h1:BVjHeAH+rl9rs6f+QIpeRl0tfu10SXn1pUSa5PVGJng= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 h1:QVw89YDxXxEe+l8gU8ETbOasdwEV+avkR75ZzsVV9WI= -github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= -github.com/containerd/cgroups/v3 v3.0.4 h1:2fs7l3P0Qxb1nKWuJNFiwhp2CqiKzho71DQkDrHJIo4= -github.com/containerd/cgroups/v3 v3.0.4/go.mod h1:SA5DLYnXO8pTGYiAHXz94qvLQTKfVM5GEVisn4jpins= -github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= -github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= -github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= -github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e h1:nOWmgQD3L/Z0bmm29iDxB7nlqjMnh7yD/PNOx9rnZmA= -github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e/go.mod h1:+bSqRETXJ1uk2S93m//htzTVqu8DJPvlGEb3bSE9PzI= +github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42 h1:Om6kYQYDUk5wWbT0t0q6pvyM49i9XZAv9dDrkDA7gjk= +github.com/cncf/xds/go v0.0.0-20250121191232-2f005788dc42/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= +github.com/cortexproject/promqlsmith v0.0.0-20250407233056-90db95b1a4e4 h1:dpo7kQ24uFSV6Zgm9/kB34TIUWjGmadlbKrM6fNfQko= +github.com/cortexproject/promqlsmith v0.0.0-20250407233056-90db95b1a4e4/go.mod h1:jh6POgN18lXU133HBMfwr/1TjvBp8e5kL4ZtRsAPvGY= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= @@ -176,12 +121,12 @@ github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgziApxA= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= -github.com/digitalocean/godo v1.122.0 h1:ziytLQi8QKtDp2K1A+YrYl2dWLHLh2uaMzWvcz9HkKg= -github.com/digitalocean/godo v1.122.0/go.mod h1:WQVH83OHUy6gC4gXpEVQKtxTd4L5oCp+5OialidkPLY= +github.com/digitalocean/godo v1.136.0 h1:DTxugljFJSMBPfEGq4KeXpnKeAHicggNqogcrw/YdZw= +github.com/digitalocean/godo v1.136.0/go.mod h1:PU8JB6I1XYkQIdHFop8lLAY9ojp6M0XcU0TWaQSxbrc= github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v27.2.0+incompatible h1:Rk9nIVdfH3+Vz4cyI/uhbINhEZ/oLmc+CBXmH6fbNk4= -github.com/docker/docker v27.2.0+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v28.0.1+incompatible h1:FCHjSRdXhNRFjlHMTv4jUNlIBbTeRjrWfeFuJp7jpo0= +github.com/docker/docker v28.0.1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -196,39 +141,30 @@ github.com/efficientgo/e2e v0.14.1-0.20230710114240-c316eb95ae5b h1:8VX23BNufsa4 github.com/efficientgo/e2e v0.14.1-0.20230710114240-c316eb95ae5b/go.mod h1:plsKU0YHE9uX+7utvr7SiDtVBSHJyEfHRO4UnUgDmts= github.com/emicklei/go-restful/v3 v3.11.0 h1:rAQeMHw1c7zTmncogyy8VvRZwtkmkZ4FxERmMY4rD+g= github.com/emicklei/go-restful/v3 v3.11.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.13.1 h1:vPfJZCkob6yTMEgS+0TwfTUfbHjfy/6vOJ8hUWX/uXE= -github.com/envoyproxy/go-control-plane v0.13.1/go.mod h1:X45hY0mufo6Fd0KW3rqsGvQMw58jvjymeCzBU3mWyHw= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/envoyproxy/protoc-gen-validate v1.1.0 h1:tntQDh69XqOCOZsDz0lVJQez/2L6Uu2PdjCQwWCJ3bM= -github.com/envoyproxy/protoc-gen-validate v1.1.0/go.mod h1:sXRDRVmzEbkM7CVcM06s9shE/m23dg3wzjl0UWqJ2q4= +github.com/envoyproxy/go-control-plane v0.13.4 h1:zEqyPVyku6IvWCFwux4x9RxkLOMUL+1vC9xUFv5l2/M= +github.com/envoyproxy/go-control-plane v0.13.4/go.mod h1:kDfuBlDVsSj2MjrLEtRWtHlsWIFcGyB2RMO44Dc5GZA= +github.com/envoyproxy/go-control-plane/envoy v1.32.4 h1:jb83lalDRZSpPWW2Z7Mck/8kXZ5CQAFYVjQcdVIr83A= +github.com/envoyproxy/go-control-plane/envoy v1.32.4/go.mod h1:Gzjc5k8JcJswLjAx1Zm+wSYE20UrLtt7JZMWiWQXQEw= +github.com/envoyproxy/go-control-plane/ratelimit v0.1.0 h1:/G9QYbddjL25KvtKTv3an9lx6VBE2cnb8wp1vEGNYGI= +github.com/envoyproxy/go-control-plane/ratelimit v0.1.0/go.mod h1:Wk+tMFAFbCXaJPzVVHnPgRKdUdwW/KdbRt94AzgRee4= +github.com/envoyproxy/protoc-gen-validate v1.2.1 h1:DEo3O99U8j4hBFwbJfrz9VtgcDfUKS7KJ7spH3d86P8= +github.com/envoyproxy/protoc-gen-validate v1.2.1/go.mod h1:d/C80l/jxXLdfEIhX1W2TmLfsJ31lvEjwamM4DxlWXU= github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb h1:IT4JYU7k4ikYg1SCxNI1/Tieq/NFvh6dzLdgi7eu0tM= github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb/go.mod h1:bH6Xx7IW64qjjJq8M2u4dxNaBiDfKK+z/3eGDpXEQhc= -github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= -github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= 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.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA= -github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= +github.com/fsnotify/fsnotify v1.8.0 h1:dAwr6QBTBZIkG8roQaJjGof0pp0EeF+tNV7YBP3F/8M= +github.com/fsnotify/fsnotify v1.8.0/go.mod h1:8jBTzvmWwFyi3Pb8djgCCO5IBqzKJ/Jwo8TRcHyHii0= github.com/fullstorydev/emulators/storage v0.0.0-20240401123056-edc69752f474 h1:TufioMBjkJ6/Oqmlye/ReuxHFS35HyLmypj/BNy/8GY= github.com/fullstorydev/emulators/storage v0.0.0-20240401123056-edc69752f474/go.mod h1:PQwxF4UU8wuL+srGxr3BOhIW5zXqgucwVlO/nPZLsxw= github.com/fxamacker/cbor/v2 v2.7.0 h1:iM5WgngdRBanHcxugY4JySA0nk1wZorNOpTgCMedv5E= github.com/fxamacker/cbor/v2 v2.7.0/go.mod h1:pxXPTn3joSm21Gbwsv0w9OSA2y1HFR9qXEeXQVeNoDQ= -github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-ini/ini v1.67.0 h1:z6ZrTEZqSWOTyH2FlglNbNgARyHG8oLW9gMELqKr06A= github.com/go-ini/ini v1.67.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-kit/log v0.2.1 h1:MRVx0/zhvdseW+Gza6N9rVzU/IVzaeE1SFI4raAhmBU= github.com/go-kit/log v0.2.1/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logfmt/logfmt v0.6.0 h1:wGYYu3uicYdqXVgoYbvnkrPVXkuLM1p1ifugDMEdRi4= github.com/go-logfmt/logfmt v0.6.0/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -236,124 +172,81 @@ github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/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.20.2 h1:mQc3nmndL8ZBzStEo3JYF8wzmeWffDH4VbXz58sAx6Q= -github.com/go-openapi/jsonpointer v0.20.2/go.mod h1:bHen+N0u1KEO3YlmqOjTT9Adn1RfD91Ar825/PuiRVs= -github.com/go-openapi/jsonreference v0.20.4 h1:bKlDxQxQJgwpUSgOENiMPzCTBVuc7vTdXSSgNeAhojU= -github.com/go-openapi/jsonreference v0.20.4/go.mod h1:5pZJyJP2MnYCpoeoMAql78cCHauHj0V9Lhc506VOpw4= -github.com/go-openapi/swag v0.22.9 h1:XX2DssF+mQKM2DHsbgZK74y/zj4mo9I99+89xUmuZCE= -github.com/go-openapi/swag v0.22.9/go.mod h1:3/OXnFfnMAwBD099SwYRk7GD3xOrr1iL7d/XNLXVVwE= -github.com/go-quicktest/qt v1.101.0 h1:O1K29Txy5P2OK0dGo59b7b0LR6wKfIhttaAhHUyn7eI= -github.com/go-quicktest/qt v1.101.0/go.mod h1:14Bz/f7NwaXPtdYEgzsx46kqSxVwTbzVZsDC26tQJow= -github.com/go-resty/resty/v2 v2.13.1 h1:x+LHXBI2nMB1vqndymf26quycC4aggYJ7DECYbiz03g= -github.com/go-resty/resty/v2 v2.13.1/go.mod h1:GznXlLxkq6Nh4sU59rPmUw3VtgpO3aS96ORAI6Q7d+0= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-openapi/analysis v0.23.0 h1:aGday7OWupfMs+LbmLZG4k0MYXIANxcuBTYUC03zFCU= +github.com/go-openapi/analysis v0.23.0/go.mod h1:9mz9ZWaSlV8TvjQHLl2mUW2PbZtemkE8yA5v22ohupo= +github.com/go-openapi/errors v0.22.0 h1:c4xY/OLxUBSTiepAg3j/MHuAv5mJhnf53LLMWFB+u/w= +github.com/go-openapi/errors v0.22.0/go.mod h1:J3DmZScxCDufmIMsdOuDHxJbdOGC0xtUynjIx092vXE= +github.com/go-openapi/jsonpointer v0.21.0 h1:YgdVicSA9vH5RiHs9TZW5oyafXZFc6+2Vc1rr/O9oNQ= +github.com/go-openapi/jsonpointer v0.21.0/go.mod h1:IUyH9l/+uyhIYQ/PXVA41Rexl+kOkAPDdXEYns6fzUY= +github.com/go-openapi/jsonreference v0.21.0 h1:Rs+Y7hSXT83Jacb7kFyjn4ijOuVGSvOdF2+tg1TRrwQ= +github.com/go-openapi/jsonreference v0.21.0/go.mod h1:LmZmgsrTkVg9LG4EaHeY8cBDslNPMo06cago5JNLkm4= +github.com/go-openapi/loads v0.22.0 h1:ECPGd4jX1U6NApCGG1We+uEozOAvXvJSF4nnwHZ8Aco= +github.com/go-openapi/loads v0.22.0/go.mod h1:yLsaTCS92mnSAZX5WWoxszLj0u+Ojl+Zs5Stn1oF+rs= +github.com/go-openapi/spec v0.21.0 h1:LTVzPc3p/RzRnkQqLRndbAzjY0d0BCL72A6j3CdL9ZY= +github.com/go-openapi/spec v0.21.0/go.mod h1:78u6VdPw81XU44qEWGhtr982gJ5BWg2c0I5XwVMotYk= +github.com/go-openapi/strfmt v0.23.0 h1:nlUS6BCqcnAk0pyhi9Y+kdDVZdZMHfEKQiS4HaMgO/c= +github.com/go-openapi/strfmt v0.23.0/go.mod h1:NrtIpfKtWIygRkKVsxh7XQMDQW5HKQl6S5ik2elW+K4= +github.com/go-openapi/swag v0.23.0 h1:vsEVJDUo2hPJ2tu0/Xc+4noaxyEffXNIs3cOULZ+GrE= +github.com/go-openapi/swag v0.23.0/go.mod h1:esZ8ITTYEsH1V2trKHjAN8Ai7xHb8RV+YSZ577vPjgQ= +github.com/go-openapi/validate v0.24.0 h1:LdfDKwNbpB6Vn40xhTdNZAnfLECL81w+VX3BumrGD58= +github.com/go-openapi/validate v0.24.0/go.mod h1:iyeX1sEufmv3nPbBdX3ieNviWnOZaJ1+zquzJEf2BAQ= +github.com/go-resty/resty/v2 v2.16.3 h1:zacNT7lt4b8M/io2Ahj6yPypL7bqx9n1iprfQuodV+E= +github.com/go-resty/resty/v2 v2.16.3/go.mod h1:hkJtXbA2iKHzJheXYvQ8snQES5ZLGKMwQ07xAwp/fiA= +github.com/go-viper/mapstructure/v2 v2.2.1 h1:ZAaOCxANMuZx5RCeg0mBdEZk7DZasvvZIxtHqx8aGss= +github.com/go-viper/mapstructure/v2 v2.2.1/go.mod h1:oJDH3BJKyqBA2TXFhDsKDGDTlndYOZ6rGS0BRZIxGhM= github.com/go-zookeeper/zk v1.0.4 h1:DPzxraQx7OrPyXq2phlGlNSIyWEsAox0RJmjTseMV6I= github.com/go-zookeeper/zk v1.0.4/go.mod h1:nOB03cncLtlp4t+UAkGSV+9beXP/akpekBwL+UX1Qcw= -github.com/goccy/go-json v0.10.3 h1:KZ5WoDbxAIgm2HNbYckL0se1fHD6rz5j4ywS6ebzDqA= -github.com/goccy/go-json v0.10.3/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk= -github.com/godbus/dbus/v5 v5.1.0/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/goccy/go-json v0.10.4 h1:JSwxQzIqKfmFX1swYPpUThQZp/Ka4wzJdK0LWVytLPM= +github.com/goccy/go-json v0.10.4/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= github.com/gofrs/flock v0.12.1 h1:MTLVXXHf8ekldpJk3AKicLij9MdwOWkZ+a/jHHZby9E= github.com/gofrs/flock v0.12.1/go.mod h1:9zxTsyu5xtJ9DK+1tFZyibEV7y3uwDxPPfbxeeHCoD0= -github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= -github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= github.com/google/gnostic-models v0.6.8 h1:yo/ABAfM5IMRsS1VnXjTBvUb61tFIHozhlYvRgGre9I= github.com/google/gnostic-models v0.6.8/go.mod h1:5n7qKqH0f5wFt+aWF8CW6pZLLNOfYuF5OpfBSENuI8U= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= -github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= +github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/go-querystring v1.1.0 h1:AnCroh3fv4ZBgVIf1Iwtovgjaw/GiKJo8M8yD/fhyJ8= github.com/google/go-querystring v1.1.0/go.mod h1:Kcdr2DB4koayq7X8pmAG4sNG59So17icRSOU623lUBU= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= -github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/martian/v3 v3.3.3 h1:DIhPTQrbPkgs2yJYdXU/eNACCG5DVQjySNRNlflZ9Fc= github.com/google/martian/v3 v3.3.3/go.mod h1:iEPrYcgCF7jA9OtScMFQyAlZZ4YXTKEtJ1E6RWzmBA0= -github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/s2a-go v0.1.8 h1:zZDs9gcbt9ZPLV0ndSyQk6Kacx2g/X+SKYovpnz3SMM= -github.com/google/s2a-go v0.1.8/go.mod h1:6iNWHTpQ+nfNRN5E00MSdfDwVesa8hhS32PhPO8deJA= +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.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= 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.4 h1:XYIDZApgAnrN1c855gTgghdIA6Stxb52D5RnLI1SLyw= -github.com/googleapis/enterprise-certificate-proxy v0.3.4/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= -github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= -github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/googleapis/gax-go/v2 v2.14.0 h1:f+jMrjBPl+DL9nI4IQzLUxMq7XrAqFYB7hBPqMNIe8o= -github.com/googleapis/gax-go/v2 v2.14.0/go.mod h1:lhBCnjdLrWRaPvLWhmc8IS24m9mr07qSYnHncrgo+zk= -github.com/gophercloud/gophercloud v1.14.1 h1:DTCNaTVGl8/cFu58O1JwWgis9gtISAFONqpMKNg/Vpw= -github.com/gophercloud/gophercloud v1.14.1/go.mod h1:aAVqcocTSXh2vYFZ1JTvx4EQmfgzxRcNupUfxZbBNDM= +github.com/googleapis/enterprise-certificate-proxy v0.3.5 h1:VgzTY2jogw3xt39CusEnFJWm7rlsq5yL5q9XdLOuP5g= +github.com/googleapis/enterprise-certificate-proxy v0.3.5/go.mod h1:MkHOF77EYAE7qfSuSS9PU6g4Nt4e11cnsDUowfwewLA= +github.com/googleapis/gax-go/v2 v2.14.1 h1:hb0FFeiPaQskmvakKu5EbCbpntQn48jyHuvrkurSS/Q= +github.com/googleapis/gax-go/v2 v2.14.1/go.mod h1:Hb/NubMaVM88SrNkvl8X/o8XWwDJEPqouaLeN2IUxoA= +github.com/gophercloud/gophercloud/v2 v2.6.0 h1:XJKQ0in3iHOZHVAFMXq/OhjCuvvG+BKR0unOqRfG1EI= +github.com/gophercloud/gophercloud/v2 v2.6.0/go.mod h1:Ki/ILhYZr/5EPebrPL9Ej+tUg4lqx71/YH2JWVeU+Qk= github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc h1:GN2Lv3MGO7AS6PrRoT6yV5+wkrOpcszoIsO4+4ds248= github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc/go.mod h1:+JKpmjMGhpgPL+rXZ5nsZieVzvarn86asRlBg4uNGnk= -github.com/hashicorp/consul/api v1.29.4 h1:P6slzxDLBOxUSj3fWo2o65VuKtbtOXFi7TSSgtXutuE= -github.com/hashicorp/consul/api v1.29.4/go.mod h1:HUlfw+l2Zy68ceJavv2zAyArl2fqhGWnMycyt56sBgg= +github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1 h1:qnpSQwGEnkcRpTqNOIR6bJbR0gAorgP9CSALpRcKoAA= +github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1/go.mod h1:lXGCsh6c22WGtjr+qGHj1otzZpV/1kwTMAqkwZsnWRU= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 h1:pRhl55Yx1eC7BZ1N+BBWwnKaMyD8uC+34TLdndZMAKk= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0/go.mod h1:XKMd7iuf/RGPSMJ/U4HP0zS2Z9Fh8Ps9a+6X26m/tmI= +github.com/hashicorp/consul/api v1.31.2 h1:NicObVJHcCmyOIl7Z9iHPvvFrocgTYo9cITSGg0/7pw= +github.com/hashicorp/consul/api v1.31.2/go.mod h1:Z8YgY0eVPukT/17ejW+l+C7zJmKwgPHtjU1q16v/Y40= github.com/hashicorp/cronexpr v1.1.2 h1:wG/ZYIKT+RT3QkOdgYc+xsKWVRgnxJ1OJtjjy84fJ9A= github.com/hashicorp/cronexpr v1.1.2/go.mod h1:P4wA0KBl9C5q2hABiMO7cp6jcIg96CDh1Efb3g1PWA4= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ= @@ -368,45 +261,30 @@ github.com/hashicorp/go-retryablehttp v0.7.7 h1:C8hUCYzor8PIfXHa4UrZkU4VvK8o9ISH github.com/hashicorp/go-retryablehttp v0.7.7/go.mod h1:pkQpWZeYWskR+D1tR2O5OcBFOxfA7DoAO6xtkuQnHTk= github.com/hashicorp/go-rootcerts v1.0.2 h1:jzhAVGtqPKbwpyCPELlgNWhE1znq+qwJtW5Oi2viEzc= github.com/hashicorp/go-rootcerts v1.0.2/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= 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-20240717122358-3d93bd3778f3 h1:fgVfQ4AC1avVOnu2cfms8VAiD8lUq3vWI8mTocOXN/w= -github.com/hashicorp/nomad/api v0.0.0-20240717122358-3d93bd3778f3/go.mod h1:svtxn6QnrQ69P23VvIWMR34tg3vmwLz4UdUzm1dSCgE= +github.com/hashicorp/nomad/api v0.0.0-20241218080744-e3ac00f30eec h1:+YBzb977VrmffaCX/OBm17dEVJUcWn5dW+eqs3aIJ/A= +github.com/hashicorp/nomad/api v0.0.0-20241218080744-e3ac00f30eec/go.mod h1:svtxn6QnrQ69P23VvIWMR34tg3vmwLz4UdUzm1dSCgE= 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.13.1 h1:jq0GP4QaYE5d8xR/Zw17s9qoaESRJMXfGmtD1a/qckQ= -github.com/hetznercloud/hcloud-go/v2 v2.13.1/go.mod h1:dhix40Br3fDiBhwaSG/zgaYOFFddpfBm/6R1Zz0IiF0= +github.com/hetznercloud/hcloud-go/v2 v2.19.1 h1:UU/7h3uc/rdgspM8xkQF7wokmwZXePWDXcLqrQRRzzY= +github.com/hetznercloud/hcloud-go/v2 v2.19.1/go.mod h1:r5RTzv+qi8IbLcDIskTzxkFIji7Ovc8yNgepQR9M+UA= github.com/hexops/gotextdiff v1.0.3 h1:gitA9+qJrrTCsiCl7+kh75nPqQt1cx4ZkudSTLoUqJM= github.com/hexops/gotextdiff v1.0.3/go.mod h1:pSWU5MAI3yDq+fZBTazCSJysOMbxWL1BSow5/V2vxeg= github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible h1:XQVXdk+WAJ4fSNB6mMRuYNvFWou7BZs6SZB925hPrnk= github.com/huaweicloud/huaweicloud-sdk-go-obs v3.24.9+incompatible/go.mod h1:l7VUhRbTKCzdOacdT4oWCwATKyvZqUOlOqr0Ous3k4s= -github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4= -github.com/imdario/mergo v0.3.16/go.mod h1:WBLT9ZmE3lPoWsEzCh9LPo3TiwVN+ZKEjmz+hD27ysY= -github.com/ionos-cloud/sdk-go/v6 v6.2.1 h1:mxxN+frNVmbFrmmFfXnBC3g2USYJrl6mc1LW2iNYbFY= -github.com/ionos-cloud/sdk-go/v6 v6.2.1/go.mod h1:SXrO9OGyWjd2rZhAhEpdYN6VUAODzzqRdqA9BCviQtI= +github.com/ionos-cloud/sdk-go/v6 v6.3.2 h1:2mUmrZZz6cPyT9IRX0T8fBLc/7XU/eTxP2Y5tS7/09k= +github.com/ionos-cloud/sdk-go/v6 v6.3.2/go.mod h1:SXrO9OGyWjd2rZhAhEpdYN6VUAODzzqRdqA9BCviQtI= 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/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= -github.com/josharian/native v1.1.0 h1:uuaP0hAbW7Y4l0ZRQ6C9zfb7Mg1mbFKry/xzDAfmtLA= -github.com/josharian/native v1.1.0/go.mod h1:7X/raswPFr05uY3HiLlYeyQntB6OO7E/d2Cu7qoaN2w= 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/jsimonetti/rtnetlink/v2 v2.0.1 h1:xda7qaHDSVOsADNouv7ukSuicKZO7GgVUCXxpaIEIlM= -github.com/jsimonetti/rtnetlink/v2 v2.0.1/go.mod h1:7MoNYNbb3UaDHtF8udiJo/RH6VsTKP1pqKLUTVCvToE= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= 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/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-keychain v0.0.0-20231219164618-57a3676c3af6 h1:IsMZxCuZqKuao2vNdfD82fjjgPLfyHLpR41Z88viRWs= @@ -418,56 +296,50 @@ github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYW github.com/klauspost/cpuid/v2 v2.0.1/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.9 h1:66ze0taIn2H33fBvCkXuv9BmCwDfafmiIVpKV9kKGuY= github.com/klauspost/cpuid/v2 v2.2.9/go.mod h1:rqkxqrZ1EhYM9G+hXH7YdowN5R5RGN6NK4QwQ3WMXF8= +github.com/knadh/koanf/maps v0.1.1 h1:G5TjmUh2D7G2YWf5SQQqSiHRJEjaicvU0KpypqB3NIs= +github.com/knadh/koanf/maps v0.1.1/go.mod h1:npD/QZY3V6ghQDdcQzl1W4ICNVTkohC8E73eI2xW4yI= +github.com/knadh/koanf/providers/confmap v0.1.0 h1:gOkxhHkemwG4LezxxN8DMOFopOPghxRVp7JbIvdvqzU= +github.com/knadh/koanf/providers/confmap v0.1.0/go.mod h1:2uLhxQzJnyHKfxG927awZC7+fyHFdQkd697K4MdLnIU= +github.com/knadh/koanf/v2 v2.1.2 h1:I2rtLRqXRy1p01m/utEtpZSSA6dcJbgGVuE27kW2PzQ= +github.com/knadh/koanf/v2 v2.1.2/go.mod h1:Gphfaen0q1Fc1HTgJgSTC4oRX9R2R5ErYMZJy8fLJBo= 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/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= 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.41.0 h1:GcP7JIBr9iLRJ9FwAtb9/WCT1DuPJS/xUApapfdjtiY= -github.com/linode/linodego v1.41.0/go.mod h1:Ow4/XZ0yvWBzt3iAHwchvhSx30AyLintsSMvvQ2/SJY= +github.com/linode/linodego v1.47.0 h1:6MFNCyzWbr8Rhl4r7d5DwZLwxvFIsM4ARH6W0KS/R0U= +github.com/linode/linodego v1.47.0/go.mod h1:vyklQRzZUWhFVBZdYx4dcYJU/gG9yKB9VUcUs6ub0Lk= github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= -github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= -github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +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.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= -github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= -github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/mdlayher/netlink v1.7.2 h1:/UtM3ofJap7Vl4QWCPDGXY8d3GIY2UGSDbK+QWmY8/g= -github.com/mdlayher/netlink v1.7.2/go.mod h1:xraEF7uJbxLhc5fpHL4cPe221LI2bdttWlU+ZGLfQSw= -github.com/mdlayher/socket v0.4.1 h1:eM9y2/jlbs1M615oshPQOHZzj6R6wMT7bX5NPiQvn2U= -github.com/mdlayher/socket v0.4.1/go.mod h1:cAqeGjoufqdxWkD7DkpyS+wcefOtmu5OQ8KuoJGIReA= -github.com/miekg/dns v1.1.62 h1:cN8OuEF1/x5Rq6Np+h1epln8OiyPWV+lROx9LxcGgIQ= -github.com/miekg/dns v1.1.62/go.mod h1:mvDlcItzm+br7MToIKqkglaGhlFMHJ9DTNNWONWXbNQ= +github.com/miekg/dns v1.1.63 h1:8M5aAw6OMZfFXTT7K5V0Eu5YiiL8l7nUAkyN6C9YwaY= +github.com/miekg/dns v1.1.63/go.mod h1:6NGHfjhpmr5lt3XPLuyfDJi5AXbNIPM9PY6H6sF1Nfs= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= -github.com/minio/minio-go/v7 v7.0.81 h1:SzhMN0TQ6T/xSBu6Nvw3M5M8voM+Ht8RH3hE8S7zxaA= -github.com/minio/minio-go/v7 v7.0.81/go.mod h1:84gmIilaX4zcvAWWzJ5Z1WI5axN+hAbM5w25xf8xvC0= +github.com/minio/minio-go/v7 v7.0.83 h1:W4Kokksvlz3OKf3OqIlzDNKd4MERlC2oN8YptwJ0+GA= +github.com/minio/minio-go/v7 v7.0.83/go.mod h1:57YXpvc5l3rjPdhqNrDsvVlY0qPI6UTk1bflAe+9doY= +github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= +github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= +github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= +github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.4.3/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= +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/sys/userns v0.1.0 h1:tVLXkFOxVu9A64/yh59slHVv9ahO9UIev4JZusOLG/g= -github.com/moby/sys/userns v0.1.0/go.mod h1:IHUYgu/kao6N8YZlp9Cf444ySSvCmDlmzUcYfDHOl28= 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= -github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/mostynb/go-grpc-compression v1.2.3 h1:42/BKWMy0KEJGSdWvzqIyOZ95YcR9mLPqKctH7Uo//I= @@ -477,7 +349,6 @@ github.com/mozillazg/go-httpheader v0.4.0 h1:aBn6aRXtFzyDLZ4VIRLsZbbJloagQfMnCiY github.com/mozillazg/go-httpheader v0.4.0/go.mod h1:PuT8h0pw6efvp8ZeUec1Rs7dwjK08bt6gKSReGMqtdA= github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/ncw/swift v1.0.53 h1:luHjjTNtekIEvHg5KdAFIBaH7bWfNkefwFnpDffSIks= @@ -486,30 +357,35 @@ github.com/oklog/run v1.1.0 h1:GEenZ1cK0+q0+wsJew9qUg/DyD8k3JzYsZAi5gYi2mA= github.com/oklog/run v1.1.0/go.mod h1:sVPdnTZT1zYwAJeCMu2Th4T21pA3FPOQRfWjQlk7DVU= github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= -github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= -github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= +github.com/oklog/ulid/v2 v2.1.0 h1:+9lhoxAP56we25tyYETBBY1YLA2SaoLvUFgrP2miPJU= +github.com/oklog/ulid/v2 v2.1.0/go.mod h1:rcEKHmBBKfef9DhnvX7y1HZBYxjXb0cP5ExxNsTT1QQ= +github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.121.0 h1:I+F6xdXQsiXXdce7yjHN+y4LX5MrZI1kNmhBunJffdA= +github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.121.0/go.mod h1:cRh3l2emFBwW96dHnlPLr1psbEYjYJmn5qFujOkbfRo= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.121.0 h1:efEcUMbyFWBx56TQDz2IMsuI0kQ5g8Im0DjQc9w9HBU= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.121.0/go.mod h1:9ghLP9djsDo5xzmzkADqeJjZb3l92XIRhpAz/ToX2QM= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.121.0 h1:D7mQQKd4rncv3PSsbDGayNENqmVwN1dFvPo3wHFzhI4= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.121.0/go.mod h1:swPiDfFHEiy9x2TwNO3uexCkwppLWfPRVoJdpJvKIQE= +github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.121.0 h1:+wj+Sw08WDdL/9lD4OUy1PFgQMsiyLuSmlmb3HbKPv4= +github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.121.0/go.mod h1:YczZl2MmjOUdg5eXg+fAW0my/EG+77b27ue6vj7xPHU= 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.0.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM= github.com/opencontainers/image-spec v1.0.2/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= -github.com/opencontainers/runtime-spec v1.2.0 h1:z97+pHb3uELt/yiAWD691HNHQIF07bE7dzrbT927iTk= -github.com/opencontainers/runtime-spec v1.2.0/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/oracle/oci-go-sdk/v65 v65.79.0 h1:Tv9L1XTKWkdXtSViMbP+dA93WunquvW++/2s5pOvOgU= github.com/oracle/oci-go-sdk/v65 v65.79.0/go.mod h1:IBEV9l1qBzUpo7zgGaRUhbB05BVfcDGYRFBCPlTcPp0= -github.com/ovh/go-ovh v1.6.0 h1:ixLOwxQdzYDx296sXcgS35TOPEahJkpjMGtzPadCjQI= -github.com/ovh/go-ovh v1.6.0/go.mod h1:cTVDnl94z4tl8pP1uZ/8jlVxntjSIf09bNcQ5TJSC7c= -github.com/parquet-go/parquet-go v0.25.0 h1:GwKy11MuF+al/lV6nUsFw8w8HCiPOSAx1/y8yFxjH5c= -github.com/parquet-go/parquet-go v0.25.0/go.mod h1:OqBBRGBl7+llplCvDMql8dEKaDqjaFA/VAPw+OJiNiw= +github.com/ovh/go-ovh v1.7.0 h1:V14nF7FwDjQrZt9g7jzcvAAQ3HN6DNShRFRMC3jLoPw= +github.com/ovh/go-ovh v1.7.0/go.mod h1:cTVDnl94z4tl8pP1uZ/8jlVxntjSIf09bNcQ5TJSC7c= +github.com/parquet-go/parquet-go v0.25.1 h1:l7jJwNM0xrk0cnIIptWMtnSnuxRkwq53S+Po3KG8Xgo= +github.com/parquet-go/parquet-go v0.25.1/go.mod h1:AXBuotO1XiBtcqJb/FKFyjBG4aqa3aQAAWF3ZPzCanY= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y= +github.com/pborman/getopt v0.0.0-20170112200414-7148bc3a4c30/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= 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.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 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= @@ -517,69 +393,41 @@ github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1 github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= -github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= -github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= -github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.29.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/common v0.61.0 h1:3gv/GThfX0cV2lpO7gkTUwZru38mxevy90Bj8YFSRQQ= -github.com/prometheus/common v0.61.0/go.mod h1:zr29OCN/2BsJRaFwG8QOBr41D6kkchKbpeNH7pAjb/s= -github.com/prometheus/common/sigv4 v0.1.0 h1:qoVebwtwwEhS85Czm2dSROY5fTo2PAPEVdDeppTwGX4= -github.com/prometheus/common/sigv4 v0.1.0/go.mod h1:2Jkxxk9yYvCkE5G1sQT7GuEXm57JrvHu9k5YwTjsNtI= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= -github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= -github.com/prometheus/prometheus v0.300.0-beta.0.0.20241007135006-65f610353919 h1:5sFkY/h4vW7n7hR047vw27EQ0cosFkxdYrbhdXuTuXE= -github.com/prometheus/prometheus v0.300.0-beta.0.0.20241007135006-65f610353919/go.mod h1:T9MXHLcLt4uvXTL0fziwJDCj6w6W07odsQ29inDUZyk= -github.com/redis/go-redis/v9 v9.6.1 h1:HHDteefn6ZkTtY5fGUE8tj8uy85AHk6zP7CpzIAM0y4= -github.com/redis/go-redis/v9 v9.6.1/go.mod h1:0C0c6ycQsdpVNQpxb1njEQIqkx5UcsM8FJCQLgE9+RA= -github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= -github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= -github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/prometheus/alertmanager v0.28.0 h1:sLN+6HhZet8hrbmGHLAHWsTXgZSVCvq9Ix3U3wvivqc= +github.com/prometheus/alertmanager v0.28.0/go.mod h1:/okSnb2LlodbMlRoOWQEKtqI/coOo2NKZDm2Hu9QHLQ= +github.com/prometheus/client_golang v1.22.0 h1:rb93p9lokFEsctTys46VnV1kLCDpVZ0a/Y92Vm0Zc6Q= +github.com/prometheus/client_golang v1.22.0/go.mod h1:R7ljNsLXhuQXYZYtw6GAE9AZg8Y7vEW5scdCXrWRXC0= +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.63.0 h1:YR/EIY1o3mEFP/kZCD7iDMnLPlGyuU2Gb3HIcXnA98k= +github.com/prometheus/common v0.63.0/go.mod h1:VVFF/fBIoToEnWRVkYoXEkq3R3paCoxG9PXP74SnV18= +github.com/prometheus/procfs v0.16.1 h1:hZ15bTNuirocR6u0JZ6BAHHmwS1p8B4P6MRqxtzMyRg= +github.com/prometheus/procfs v0.16.1/go.mod h1:teAbpZRB1iIAJYREa1LsoWUXykVXA1KlTmWl8x/U+Is= +github.com/prometheus/prometheus v0.303.1 h1:He/2jRE6sB23Ew38AIoR1WRR3fCMgPlJA2E0obD2WSY= +github.com/prometheus/prometheus v0.303.1/go.mod h1:WEq2ogBPZoLjj9x5K67VEk7ECR0nRD9XCjaOt1lsYck= +github.com/prometheus/sigv4 v0.1.2 h1:R7570f8AoM5YnTUPFm3mjZH5q2k4D+I/phCWvZ4PXG8= +github.com/prometheus/sigv4 v0.1.2/go.mod h1:GF9fwrvLgkQwDdQ5BXeV9XUSCH/IPNqzvAoaohfjqMU= +github.com/redis/go-redis/v9 v9.7.0 h1:HhLSs+B6O021gwzl+locl0zEDnyNkxMtf/Z3NNBMa9E= +github.com/redis/go-redis/v9 v9.7.0/go.mod h1:f6zhXITC7JUJIlPEiBOTXxJgPLdZcA93GewI7inzyWw= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/rs/xid v1.6.0 h1:fV591PaemRlL6JfRxGDEPl69wICngIQ3shQtzfy2gxU= github.com/rs/xid v1.6.0/go.mod h1:7XoLgs4eV+QndskICGsho+ADou8ySMSjJKDIan90Nz0= -github.com/scaleway/scaleway-sdk-go v1.0.0-beta.30 h1:yoKAVkEVwAqbGbR8n87rHQ1dulL25rKloGadb3vm770= -github.com/scaleway/scaleway-sdk-go v1.0.0-beta.30/go.mod h1:sH0u6fq6x4R5M7WxkoQFY/o7UaiItec0o1LinLCJNq8= -github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= -github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/scaleway/scaleway-sdk-go v1.0.0-beta.32 h1:4+LP7qmsLSGbmc66m1s5dKRMBwztRppfxFKlYqYte/c= +github.com/scaleway/scaleway-sdk-go v1.0.0-beta.32/go.mod h1:kzh+BSAvpoyHHdHBCDhmSWtBc1NbLMZ2lWHqnBoxFks= github.com/sony/gobreaker v0.5.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/sony/gobreaker v1.0.0 h1:feX5fGGXSl3dYd4aHZItw+FpHLvvoaqkawKjVNiFMNQ= github.com/sony/gobreaker v1.0.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/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= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= @@ -588,10 +436,10 @@ github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.563/go.mod github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/kms v1.0.563/go.mod h1:uom4Nvi9W+Qkom0exYiJ9VWJjXwyxtPYTkKkaLMlfE0= github.com/tencentyun/cos-go-sdk-v5 v0.7.59 h1:bzB/0fj+gKCUvEfe+c4CNGkqTnrenxzTsLM2rMn3mHE= github.com/tencentyun/cos-go-sdk-v5 v0.7.59/go.mod h1:8+hG+mQMuRP/OIS9d83syAvXvrMj9HhkND6Q1fLghw0= -github.com/thanos-io/objstore v0.0.0-20241111205755-d1dd89d41f97 h1:VjG0mwhN1DkncwDHFvrpd12/2TLfgYNRmEQA48ikp+0= -github.com/thanos-io/objstore v0.0.0-20241111205755-d1dd89d41f97/go.mod h1:vyzFrBXgP+fGNG2FopEGWOO/zrIuoy7zt3LpLeezRsw= -github.com/thanos-io/promql-engine v0.0.0-20241203103240-2f49f80c7c68 h1:cChM/FbpXeYmrSmXO1/MmmSlONviLVxWAWCB0/g4JrY= -github.com/thanos-io/promql-engine v0.0.0-20241203103240-2f49f80c7c68/go.mod h1:wx0JlRZtsB2S10JYUgeg5GqLfMxw31SzArP+28yyE00= +github.com/thanos-io/objstore v0.0.0-20250317105316-a0136a6f898d h1:L4k+8i1cl0h3MscslVUAcBpvA5i9UYzE0DybcOxzvlM= +github.com/thanos-io/objstore v0.0.0-20250317105316-a0136a6f898d/go.mod h1:Nmy3+M2UM7wu2sEvg0h5M/c3mu1QaxmdyPvoGUPGlaU= +github.com/thanos-io/promql-engine v0.0.0-20250428192047-2eb963d84212 h1:gbSBZWFUNNB0NiRJKqqAzQmdxp2T9GQUUfAsy6EqwqI= +github.com/thanos-io/promql-engine v0.0.0-20250428192047-2eb963d84212/go.mod h1:67nlwY8gyrrH+xD91x47r01JtS3/q7LEXJaTPJ5XBLI= github.com/thanos-io/thanos v0.37.2 h1:eCv5JfqM5BQ+9+hZXZPeNgsaJRxjW2uMhstacNso8ok= github.com/thanos-io/thanos v0.37.2/go.mod h1:5Ni7Uc1Bc8UCGOYmZ/2f/LVvDkZKNDdqDJZqjDFG+rI= github.com/vultr/govultr/v2 v2.17.2 h1:gej/rwr91Puc/tgh+j33p/BLR16UrIPnSr+AIwYWZQs= @@ -600,388 +448,187 @@ 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/xyproto/randomstring v1.0.5 h1:YtlWPoRdgMu3NZtP45drfy1GKoojuR7hmRcnhZqKjWU= github.com/xyproto/randomstring v1.0.5/go.mod h1:rgmS5DeNXLivK7YprL0pY+lTuhNQW3iGxZ18UQApw/E= -github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/zhangyunhao116/umap v0.0.0-20241028085443-797b8ba93f64 h1:U5oIVLsZbkjon5J62Ej8gtb710oFSSIqijudhMDHZow= github.com/zhangyunhao116/umap v0.0.0-20241028085443-797b8ba93f64/go.mod h1:r86X1CnsDRrOeLtJlqRWdELPWpkcf933GTlojQlifQw= -go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= -go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= -go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.32.0 h1:P78qWqkLSShicHmAzfECaTgvslqHxblNE9j62Ws1NK8= -go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.57.0 h1:qtFISDHKolvIxzSs0gIaiPUPR0Cucb0F2coHC7ZLdps= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.57.0/go.mod h1:Y+Pop1Q6hCOnETWTW4NROK/q1hv50hM7yDaUTjG8lp8= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.57.0 h1:DheMAlT6POBP+gh8RUH19EOTnQIor5QE0uSRPtzCpSw= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.57.0/go.mod h1:wZcGmeVO9nzP67aYSLDqXNWK87EZWhi7JWj1v7ZXf94= -go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= -go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.mongodb.org/mongo-driver v1.14.0 h1:P98w8egYRjYe3XDjxhYJagTokP/H6HzlsnojRgZRd80= +go.mongodb.org/mongo-driver v1.14.0/go.mod h1:Vzb0Mk/pa7e6cWw85R4F/endUC3u0U9jGcNU603k65c= +go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= +go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= +go.opentelemetry.io/collector/component v1.27.0 h1:6wk0K23YT9lSprX8BH9x5w8ssAORE109ekH/ix2S614= +go.opentelemetry.io/collector/component v1.27.0/go.mod h1:fIyBHoa7vDyZL3Pcidgy45cx24tBe7iHWne097blGgo= +go.opentelemetry.io/collector/component/componentstatus v0.121.0 h1:G4KqBUuAqnQ1kB3fUxXPwspjwnhGZzdArlO7vc343og= +go.opentelemetry.io/collector/component/componentstatus v0.121.0/go.mod h1:ufRv8q15XNdbr9nNzdepMHlLl2aC3NHQgecCzp5VRns= +go.opentelemetry.io/collector/component/componenttest v0.121.0 h1:4q1/7WnP9LPKaY4HAd8/OkzhllZpRACKAOlWsqbrzqc= +go.opentelemetry.io/collector/component/componenttest v0.121.0/go.mod h1:H7bEXDPMYNeWcHal0xyKlVfRPByVxale7hCJ+Myjq3Q= +go.opentelemetry.io/collector/confmap v1.27.0 h1:OIjPcjij1NxkVQsQVmHro4+t1eYNFiUGib9+J9YBZhM= +go.opentelemetry.io/collector/confmap v1.27.0/go.mod h1:tmOa6iw3FJsEgfBHKALqvcdfRtf71JZGor0wSM5MoH8= +go.opentelemetry.io/collector/confmap/xconfmap v0.121.0 h1:pZ7SOl/i3kUIPdUwIeHHsYqzOHNLCwiyXZnwQ7rLO3E= +go.opentelemetry.io/collector/confmap/xconfmap v0.121.0/go.mod h1:YI1Sp8mbYro/H3rqH4csTq68VUuie5WVb7LI1o5+tVc= +go.opentelemetry.io/collector/consumer v1.27.0 h1:JoXdoCeFDJG3d9TYrKHvTT4eBhzKXDVTkWW5mDfnLiY= +go.opentelemetry.io/collector/consumer v1.27.0/go.mod h1:1B/+kTDUI6u3mCIOAkm5ityIpv5uC0Ll78IA50SNZ24= +go.opentelemetry.io/collector/consumer/consumertest v0.121.0 h1:EIJPAXQY0w9j1k/e5OzJqOYVEr6WljKpJBjgkkp/hWw= +go.opentelemetry.io/collector/consumer/consumertest v0.121.0/go.mod h1:Hmj+TizzsLU0EmS2n/rJYScOybNmm3mrAjis6ed7qTw= +go.opentelemetry.io/collector/consumer/xconsumer v0.121.0 h1:/FJ7L6+G++FvktXc/aBnnYDIKLoYsWLh0pKbvzFFwF8= +go.opentelemetry.io/collector/consumer/xconsumer v0.121.0/go.mod h1:KKy8Qg/vOnyseoi7A9/x1a1oEqSmf0WBHkJFlnQH0Ow= +go.opentelemetry.io/collector/pdata v1.27.0 h1:66yI7FYkUDia74h48Fd2/KG2Vk8DxZnGw54wRXykCEU= +go.opentelemetry.io/collector/pdata v1.27.0/go.mod h1:18e8/xDZsqyj00h/5HM5GLdJgBzzG9Ei8g9SpNoiMtI= +go.opentelemetry.io/collector/pdata/pprofile v0.121.0 h1:DFBelDRsZYxEaSoxSRtseAazsHJfqfC/Yl64uPicl2g= +go.opentelemetry.io/collector/pdata/pprofile v0.121.0/go.mod h1:j/fjrd7ybJp/PXkba92QLzx7hykUVmU8x/WJvI2JWSg= +go.opentelemetry.io/collector/pdata/testdata v0.121.0 h1:FFz+rdb7o6JRZ82Zmp6WKEdKnEMaoF3jLb7F1F21ijg= +go.opentelemetry.io/collector/pdata/testdata v0.121.0/go.mod h1:UhiSwmVpBbuKlPdmhBytiVTHipSz/JO6c4mbD4kWOPg= +go.opentelemetry.io/collector/pipeline v0.121.0 h1:SOiocdyWCJCjWAb96HIxsy9enp2qyQ1NRFo26qyHlCE= +go.opentelemetry.io/collector/pipeline v0.121.0/go.mod h1:TO02zju/K6E+oFIOdi372Wk0MXd+Szy72zcTsFQwXl4= +go.opentelemetry.io/collector/processor v0.121.0 h1:OcLrJ2F17cU0oDtXEYbGvL8vbku/kRQgAafSZ3+8jLY= +go.opentelemetry.io/collector/processor v0.121.0/go.mod h1:BoFEMvPn5/p53eWz+R9cibIxCXzaRZ/RtcBPtvqXNaQ= +go.opentelemetry.io/collector/processor/processortest v0.121.0 h1:1c3mEABELrxdC1obSQjIlfh5jZljJlzUravmzy1Mofo= +go.opentelemetry.io/collector/processor/processortest v0.121.0/go.mod h1:oL4S/eguZ6XTK6IxAQXhXD9yWuRrG5/Maiskbf9HL0o= +go.opentelemetry.io/collector/processor/xprocessor v0.121.0 h1:AiqDKzpEYZpiP9y3RRp4G9ym6fG2f9HByu3yWkSdd2E= +go.opentelemetry.io/collector/processor/xprocessor v0.121.0/go.mod h1:Puk+6YYKyqLVKqpftUXg0blMrd3BlH/Av+oiajp1sHQ= +go.opentelemetry.io/collector/semconv v0.121.0 h1:dtdgh5TsKWGZXIBMsyCMVrY1VgmyWlXHgWx/VH9tL1U= +go.opentelemetry.io/collector/semconv v0.121.0/go.mod h1:te6VQ4zZJO5Lp8dM2XIhDxDiL45mwX0YAQQWRQ0Qr9U= +go.opentelemetry.io/contrib/detectors/gcp v1.34.0 h1:JRxssobiPg23otYU5SbWtQC//snGVIM3Tx6QRzlQBao= +go.opentelemetry.io/contrib/detectors/gcp v1.34.0/go.mod h1:cV4BMFcscUR/ckqLkbfQmF0PRsq8w/lMGzdbCSveBHo= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 h1:rgMkmiGfix9vFJDcDi1PK8WEQP4FLQwLDfhp5ZLpFeE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0/go.mod h1:ijPqXp5P6IRRByFVVg9DY8P5HkxkHE5ARIa+86aXPf4= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.60.0 h1:0tY123n7CdWMem7MOVdKOt0YfshufLCwfE5Bob+hQuM= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.60.0/go.mod h1:CosX/aS4eHnG9D7nESYpV753l4j9q5j3SL/PUYd2lR8= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.60.0 h1:sbiXRNDSWJOTobXh5HyQKjq6wUC5tNybqjIqDpAY4CU= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.60.0/go.mod h1:69uWxva0WgAA/4bu2Yy70SLDBwZXuQ6PbBpbsa5iZrQ= +go.opentelemetry.io/contrib/propagators/autoprop v0.54.0 h1:h/O1OcNbqrFilsMKfG6MJWWpx8gzCDfn9D+1W7lU3lE= +go.opentelemetry.io/contrib/propagators/autoprop v0.54.0/go.mod h1:VIaPlErTgbng1UhrMA4N6Yy+f94PLA/qRPOCMATdoCs= +go.opentelemetry.io/contrib/propagators/aws v1.29.0 h1:mqadbdNBhn/MVOcNx0dEZAaOaomKKdnsM0QNBmFegiI= +go.opentelemetry.io/contrib/propagators/aws v1.29.0/go.mod h1:3RCUqtGbLbVr6REZv3pQbtqql9GNEpvyB7GiTJhP/nk= +go.opentelemetry.io/contrib/propagators/b3 v1.29.0 h1:hNjyoRsAACnhoOLWupItUjABzeYmX3GTTZLzwJluJlk= +go.opentelemetry.io/contrib/propagators/b3 v1.29.0/go.mod h1:E76MTitU1Niwo5NSN+mVxkyLu4h4h7Dp/yh38F2WuIU= +go.opentelemetry.io/contrib/propagators/jaeger v1.29.0 h1:+YPiqF5rR6PqHBlmEFLPumbSP0gY0WmCGFayXRcCLvs= +go.opentelemetry.io/contrib/propagators/jaeger v1.29.0/go.mod h1:6PD7q7qquWSp3Z4HeM3e/2ipRubaY1rXZO8NIHVDZjs= +go.opentelemetry.io/contrib/propagators/ot v1.29.0 h1:CaJU78FvXrA6ajjp1dOdcABBEjh529+hl396RTqc2LQ= +go.opentelemetry.io/contrib/propagators/ot v1.29.0/go.mod h1:Sc0omwLb4eptUhwOAfYXfmPmErHPu2HV6vkeDge/3sY= +go.opentelemetry.io/otel v1.35.0 h1:xKWKPxrxB6OtMCbmMY021CqC45J+3Onta9MqjhnusiQ= +go.opentelemetry.io/otel v1.35.0/go.mod h1:UEqy8Zp11hpkUrL73gSlELM0DupHoiq72dR+Zqel/+Y= go.opentelemetry.io/otel/exporters/jaeger v1.17.0 h1:D7UpUy2Xc2wsi1Ras6V40q806WM07rqoCWzXu7Sqy+4= go.opentelemetry.io/otel/exporters/jaeger v1.17.0/go.mod h1:nPCqOnEH9rNLKqH/+rrUjiMzHJdV1BlpKcTwRTyKkKI= go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.29.0 h1:WDdP9acbMYjbKIyJUhTvtzj601sVJOqgWdUxSdR/Ysc= go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.29.0/go.mod h1:BLbf7zbNIONBLPwvFnwNHGj4zge8uTCM/UPIVW1Mq2I= go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.32.0 h1:cC2yDI3IQd0Udsux7Qmq8ToKAx1XCilTQECZ0KDZyTw= go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.32.0/go.mod h1:2PD5Ex6z8CFzDbTdOlwyNIUywRr1DN0ospafJM1wJ+s= -go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= -go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= -go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= -go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= -go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= -go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= -go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= -go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +go.opentelemetry.io/otel/metric v1.35.0 h1:0znxYu2SNyuMSQT4Y9WDWej0VpcsxkuklLa4/siN90M= +go.opentelemetry.io/otel/metric v1.35.0/go.mod h1:nKVFgxBZ2fReX6IlyW28MgZojkoAkJGaE8CpgeAU3oE= +go.opentelemetry.io/otel/sdk v1.35.0 h1:iPctf8iprVySXSKJffSS79eOjl9pvxV9ZqOWT0QejKY= +go.opentelemetry.io/otel/sdk v1.35.0/go.mod h1:+ga1bZliga3DxJ3CQGg3updiaAJoNECOgJREo9KHGQg= +go.opentelemetry.io/otel/sdk/metric v1.35.0 h1:1RriWBmCKgkeHEhM7a2uMjMUfP7MsOF5JpUCaEqEI9o= +go.opentelemetry.io/otel/sdk/metric v1.35.0/go.mod h1:is6XYCUMpcKi+ZsOvfluY5YstFnhW0BidkR+gL+qN+w= +go.opentelemetry.io/otel/trace v1.35.0 h1:dPpEfJu1sDIqruz7BHFG3c7528f6ddfSWfFDVt/xgMs= +go.opentelemetry.io/otel/trace v1.35.0/go.mod h1:WUk7DtFp1Aw2MkvqGdwiXYDZZNvA/1J8o6xRXLrIkyc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= 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.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= go4.org/intern v0.0.0-20230525184215-6c62f75575cb h1:ae7kzL5Cfdmcecbh22ll7lYP3iuUdnfnhiPcSaDgH/8= go4.org/intern v0.0.0-20230525184215-6c62f75575cb/go.mod h1:Ycrt6raEcnF5FTsLiLKkhBTO6DPX3RCUCUVnks3gFJU= go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 h1:WJhcL4p+YeDxmZWg141nRm7XC8IDmhz7lk5GpadO1Sg= go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2/go.mod h1:FftLjUGFEDu5k8lt0ddY+HcrH/qU/0qk+H8j9/nTl3E= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= 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.35.0 h1:b15kiHdrGCHrP6LvwaQ3c03kgNhhiMgvlhxHQhmg2Xs= -golang.org/x/crypto v0.35.0/go.mod h1:dy7dXNW32cAb/6/PRuTNsix8T+vJAqvuIy5Bli/x0YQ= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= -golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= -golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= -golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20241108190413-2d47ceb2692f h1:XdNn9LlyWAhLVp6P/i8QYBW+hlyhrhei9uErw2B5GJo= -golang.org/x/exp v0.0.0-20241108190413-2d47ceb2692f/go.mod h1:D5SMRVC3C2/4+F/DB1wZsLRnSNimn2Sp/NPsCrsv8ak= -golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= -golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= -golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/crypto v0.39.0 h1:SHs+kF4LP+f+p14esP5jAoDpHU8Gu/v9lFRK6IT5imM= +golang.org/x/crypto v0.39.0/go.mod h1:L+Xg3Wf6HoL4Bn4238Z6ft6KfEpN0tJGo53AAPC632U= +golang.org/x/exp v0.0.0-20250106191152-7588d65b2ba8 h1:yqrTHse8TCMW1M1ZCP+VAR/l0kKxwaAIqN/il7x4voA= +golang.org/x/exp v0.0.0-20250106191152-7588d65b2ba8/go.mod h1:tujkw807nyEEAamNbDrEGzRav+ilXA7PCRAd6xsmwiU= 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.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= -golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/mod v0.25.0 h1:n7a+ZbQKQA/Ysbyb0/6IbB1H/X41mKgbhfv7AfG/44w= +golang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/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-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= -golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/net v0.41.0 h1:vBTly1HeNPEn3wtREYfy4GZ/NECgw2Cnl+nK6Nz3uvw= +golang.org/x/net v0.41.0/go.mod h1:B/K4NNqkfmg07DQYrbwvSluqCJOOXwUjeb/5lOisjbA= +golang.org/x/oauth2 v0.27.0 h1:da9Vo7/tDv5RH/7nZDz1eMGS/q1Vv1N/7FCrBhI9I3M= +golang.org/x/oauth2 v0.27.0/go.mod h1:onh5ek6nERTohokkhCD/y2cV4Do3fxFHFuAejCkRWT8= 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-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/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.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sync v0.15.0 h1:KWH3jNZsfyT6xfAfKiz6MRNmd46ByHDYaZ7KSkCtdW8= +golang.org/x/sync v0.15.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.29.0 h1:L6pJp37ocefwRRtYPKSWOWzOtWSxVajvz2ldH/xi3iU= -golang.org/x/term v0.29.0/go.mod h1:6bl4lRlvVuDgSf3179VpIxBF0o10JUpXWOnI7nErv7s= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw= +golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= +golang.org/x/term v0.32.0 h1:DR4lr0TjUs3epypdhTOkMmuF5CDFJ/8pOnbzMZPQ7bg= +golang.org/x/term v0.32.0/go.mod h1:uZG1FhGx848Sqfsq4/DlJr3xGGsYMu/L5GW4abiaEPQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= -golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.8.0 h1:9i3RxcPv3PZnitoVGMPDKZSq1xW1gK1Xy3ArNOGZfEg= -golang.org/x/time v0.8.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/text v0.26.0 h1:P42AVeLghgTYr4+xUnTRKDMqpar+PtX7KWuNQL21L8M= +golang.org/x/text v0.26.0/go.mod h1:QK15LZJUUQVJxhz7wXgxSy/CJaTFjd0G+YLonydOVQA= +golang.org/x/time v0.10.0 h1:3usCWA8tQn0L8+hFJQNgzpWbd89begxN66o1Ojdn5L4= +golang.org/x/time v0.10.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.27.0 h1:qEKojBykQkQ4EynWy4S8Weg69NumxKdn40Fce3uc/8o= -golang.org/x/tools v0.27.0/go.mod h1:sUi0ZgbwW9ZPAq26Ekut+weQPR5eIM6GQLQ1Yjm1H0Q= +golang.org/x/tools v0.34.0 h1:qIpSLOxeCYGg9TrcJokLBG4KFA6d795g0xkBkiESGlo= +golang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg= 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.15.1 h1:FNy7N6OUZVUaWG9pTiD+jlhdQ3lMP+/LcTpJ6+a8sQ0= gonum.org/v1/gonum v0.15.1/go.mod h1:eZTZuRFrzu5pcyjN5wJhcIhnUdNijYxX1T2IcrOGY0o= -google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= -google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= -google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= -google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/api v0.209.0 h1:Ja2OXNlyRlWCWu8o+GgI4yUn/wz9h/5ZfFbKz+dQX+w= -google.golang.org/api v0.209.0/go.mod h1:I53S168Yr/PNDNMi5yPnDc0/LGRZO6o7PoEbl/HY3CM= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= -google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= -google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= -google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 h1:ToEetK57OidYuqD4Q5w+vfEnPvPpuTwedCNVohYJfNk= -google.golang.org/genproto v0.0.0-20241118233622-e639e219e697/go.mod h1:JJrvXBWRZaFMxBufik1a4RpFw4HhgVtBBWQeQgUj2cc= -google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 h1:pgr/4QbFyktUv9CtQ/Fq4gzEE6/Xs7iCXbktaGzLHbQ= -google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697/go.mod h1:+D9ySVjN8nY8YCVjc5O7PZDIdZporIDY3KaGfJunh88= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697 h1:LWZqQOEjDyONlF1H6afSWpAL/znlREo2tHfLoe+8LMA= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.68.0 h1:aHQeeJbo8zAkAa3pRzrVjZlbz6uSfeOXlJNQM0RAbz0= -google.golang.org/grpc v1.68.0/go.mod h1:fmSPC5AsjSBCK54MyHRx48kpOti1/jRfOlwEWywNjWA= -google.golang.org/grpc/stats/opentelemetry v0.0.0-20241028142157-ada6787961b3 h1:hUfOButuEtpc0UvYiaYRbNwxVYr0mQQOWq6X8beJ9Gc= -google.golang.org/grpc/stats/opentelemetry v0.0.0-20241028142157-ada6787961b3/go.mod h1:jzYlkSMbKypzuu6xoAEijsNVo9ZeDF1u/zCfFgsx7jg= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= -google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= -google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/api v0.224.0 h1:Ir4UPtDsNiwIOHdExr3fAj4xZ42QjK7uQte3lORLJwU= +google.golang.org/api v0.224.0/go.mod h1:3V39my2xAGkodXy0vEqcEtkqgw2GtrFL5WuBZlCTCOQ= +google.golang.org/genproto v0.0.0-20250204164813-702378808489 h1:nQcbCCOg2h2CQ0yA8SY3AHqriNKDvsetuq9mE/HFjtc= +google.golang.org/genproto v0.0.0-20250204164813-702378808489/go.mod h1:wkQ2Aj/xvshAUDtO/JHvu9y+AaN9cqs28QuSVSHtZSY= +google.golang.org/genproto/googleapis/api v0.0.0-20250218202821-56aae31c358a h1:nwKuGPlUAt+aR+pcrkfFRrTU1BVrSmYyYMxYbUIVHr0= +google.golang.org/genproto/googleapis/api v0.0.0-20250218202821-56aae31c358a/go.mod h1:3kWAYMk1I75K4vykHtKt2ycnOgpA6974V7bREqbsenU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250227231956-55c901821b1e h1:YA5lmSs3zc/5w+xsRcHqpETkaYyK63ivEPzNTcUUlSA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250227231956-55c901821b1e/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= +google.golang.org/grpc v1.71.0 h1:kF77BGdPTQ4/JZWMlb9VpJ5pa25aqvVqogsxNHHdeBg= +google.golang.org/grpc v1.71.0/go.mod h1:H0GRtasmQOh9LkFoCPDu3ZrwUtD1YGE+b2vYBYd/8Ec= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/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= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/evanphx/json-patch.v4 v4.12.0 h1:n6jtcsulIzXPJaxegRbvFNNrZDjbij7ny3gmSPG+6V4= +gopkg.in/evanphx/json-patch.v4 v4.12.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.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 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= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -k8s.io/api v0.31.3 h1:umzm5o8lFbdN/hIXbrK9oRpOproJO62CV1zqxXrLgk8= -k8s.io/api v0.31.3/go.mod h1:UJrkIp9pnMOI9K2nlL6vwpxRzzEX5sWgn8kGQe92kCE= -k8s.io/apimachinery v0.31.3 h1:6l0WhcYgasZ/wk9ktLq5vLaoXJJr5ts6lkaQzgeYPq4= -k8s.io/apimachinery v0.31.3/go.mod h1:rsPdaZJfTfLsNJSQzNHQvYoTmxhoOEofxtOsF3rtsMo= -k8s.io/client-go v0.31.3 h1:CAlZuM+PH2cm+86LOBemaJI/lQ5linJ6UFxKX/SoG+4= -k8s.io/client-go v0.31.3/go.mod h1:2CgjPUTpv3fE5dNygAr2NcM8nhHzXvxB8KL5gYc3kJs= +k8s.io/api v0.32.2 h1:bZrMLEkgizC24G9eViHGOPbW+aRo9duEISRIJKfdJuw= +k8s.io/api v0.32.2/go.mod h1:hKlhk4x1sJyYnHENsrdCWw31FEmCijNGPJO5WzHiJ6Y= +k8s.io/apimachinery v0.32.2 h1:yoQBR9ZGkA6Rgmhbp/yuT9/g+4lxtsGYwW6dR6BDPLQ= +k8s.io/apimachinery v0.32.2/go.mod h1:GpHVgxoKlTxClKcteaeuF1Ul/lDVb74KpZcxcmLDElE= +k8s.io/client-go v0.32.2 h1:4dYCD4Nz+9RApM2b/3BtVvBHw54QjMFUl1OLcJG5yOA= +k8s.io/client-go v0.32.2/go.mod h1:fpZ4oJXclZ3r2nDOv+Ux3XcJutfrwjKTCHz2H3sww94= k8s.io/klog/v2 v2.130.1 h1:n9Xl7H1Xvksem4KFG4PYbdQCQxqc/tTUyrgXaOhHSzk= k8s.io/klog/v2 v2.130.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= -k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340 h1:BZqlfIlq5YbRMFko6/PM7FjZpUb45WallggurYhKGag= -k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340/go.mod h1:yD4MZYeKMBwQKVht279WycxKyM84kkAx2DPrTXaeb98= +k8s.io/kube-openapi v0.0.0-20241105132330-32ad38e42d3f h1:GA7//TjRY9yWGy1poLzYYJJ4JRdzg3+O6e8I+e+8T5Y= +k8s.io/kube-openapi v0.0.0-20241105132330-32ad38e42d3f/go.mod h1:R/HEjbvWI0qdfb8viZUeVZm0X6IZnxAydC7YU42CMw4= k8s.io/utils v0.0.0-20241104163129-6fe5fd82f078 h1:jGnCPejIetjiy2gqaJ5V0NLwTpF4wbQ6cZIItJCSHno= k8s.io/utils v0.0.0-20241104163129-6fe5fd82f078/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= -rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= -rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= -sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= -sigs.k8s.io/structured-merge-diff/v4 v4.4.1 h1:150L+0vs/8DA78h1u02ooW1/fFq/Lwr+sGiqlzvrtq4= -sigs.k8s.io/structured-merge-diff/v4 v4.4.1/go.mod h1:N8hJocpFajUSSeSJ9bOZ77VzejKZaXsTtZo4/u7Io08= +sigs.k8s.io/json v0.0.0-20241010143419-9aa6b5e7a4b3 h1:/Rv+M11QRah1itp8VhT6HoVx1Ray9eB4DBr+K+/sCJ8= +sigs.k8s.io/json v0.0.0-20241010143419-9aa6b5e7a4b3/go.mod h1:18nIHnGi6636UCz6m8i4DhaJ65T6EruyzmoQqI2BVDo= +sigs.k8s.io/structured-merge-diff/v4 v4.4.2 h1:MdmvkGuXi/8io6ixD5wud3vOLwc1rj0aNqRlpuvjmwA= +sigs.k8s.io/structured-merge-diff/v4 v4.4.2/go.mod h1:N8f93tFZh9U6vpxwRArLiikrE5/2tiu1w1AGfACIGE4= sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= diff --git a/go.tools.mod b/go.tools.mod new file mode 100644 index 0000000..f366833 --- /dev/null +++ b/go.tools.mod @@ -0,0 +1,29 @@ +module github.com/cloudflare/parquet-tsdb-poc + +go 1.24.2 + +tool ( + github.com/mgechev/revive + golang.org/x/tools/cmd/goimports + golang.org/x/tools/gopls/internal/analysis/modernize/cmd/modernize +) + +require ( + codeberg.org/chavacava/garif v0.2.0 // indirect + github.com/BurntSushi/toml v1.5.0 // indirect + github.com/fatih/color v1.18.0 // indirect + github.com/fatih/structtag v1.2.0 // indirect + github.com/hashicorp/go-version v1.7.0 // indirect + github.com/mattn/go-colorable v0.1.14 // indirect + github.com/mattn/go-isatty v0.0.20 // indirect + github.com/mgechev/dots v1.0.0 // indirect + github.com/mgechev/revive v1.10.0 // indirect + github.com/spf13/afero v1.14.0 // indirect + golang.org/x/mod v0.25.0 // indirect + golang.org/x/sync v0.15.0 // indirect + golang.org/x/sys v0.33.0 // indirect + golang.org/x/telemetry v0.0.0-20250417124945-06ef541f3fa3 // indirect + golang.org/x/text v0.25.0 // indirect + golang.org/x/tools v0.34.0 // indirect + golang.org/x/tools/gopls v0.19.0-pre.2 // indirect +) diff --git a/go.tools.sum b/go.tools.sum new file mode 100644 index 0000000..cc3e88c --- /dev/null +++ b/go.tools.sum @@ -0,0 +1,43 @@ +codeberg.org/chavacava/garif v0.2.0 h1:F0tVjhYbuOCnvNcU3YSpO6b3Waw6Bimy4K0mM8y6MfY= +codeberg.org/chavacava/garif v0.2.0/go.mod h1:P2BPbVbT4QcvLZrORc2T29szK3xEOlnl0GiPTJmEqBQ= +github.com/BurntSushi/toml v1.5.0 h1:W5quZX/G/csjUnuI8SUYlsHs9M38FC7znL0lIO+DvMg= +github.com/BurntSushi/toml v1.5.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= +github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= +github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= +github.com/hashicorp/go-version v1.7.0 h1:5tqGy27NaOTB8yJKUZELlFAS/LTKJkrmONwQKeRZfjY= +github.com/hashicorp/go-version v1.7.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +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.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mgechev/dots v1.0.0 h1:o+4OJ3OjWzgQHGJXKfJ8rbH4dqDugu5BiEy84nxg0k4= +github.com/mgechev/dots v1.0.0/go.mod h1:rykuMydC9t3wfkM+ccYH3U3ss03vZGg6h3hmOznXLH0= +github.com/mgechev/revive v1.10.0 h1:x2oJsd7yrDp0mC6IgZqSKBTjSUC9Zk5Ob2WfBwZic2I= +github.com/mgechev/revive v1.10.0/go.mod h1:1MRO9zUV7Yukhqh/nGRKSaw6xC5XDzPWPja5GMPWoSE= +github.com/spf13/afero v1.14.0 h1:9tH6MapGnn/j0eb0yIXiLjERO8RB6xIVZRDCX7PtqWA= +github.com/spf13/afero v1.14.0/go.mod h1:acJQ8t0ohCGuMN3O+Pv0V0hgMxNYDlvdk+VTfyZmbYo= +golang.org/x/mod v0.24.0 h1:ZfthKaKaT4NrhGVZHO1/WDTwGES4De8KtWO0SIbNJMU= +golang.org/x/mod v0.24.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= +golang.org/x/mod v0.25.0 h1:n7a+ZbQKQA/Ysbyb0/6IbB1H/X41mKgbhfv7AfG/44w= +golang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= +golang.org/x/sync v0.14.0 h1:woo0S4Yywslg6hp4eUFjTVOyKt0RookbpAHG4c1HmhQ= +golang.org/x/sync v0.14.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= +golang.org/x/sync v0.15.0 h1:KWH3jNZsfyT6xfAfKiz6MRNmd46ByHDYaZ7KSkCtdW8= +golang.org/x/sync v0.15.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw= +golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= +golang.org/x/telemetry v0.0.0-20250417124945-06ef541f3fa3 h1:RXY2+rSHXvxO2Y+gKrPjYVaEoGOqh3VEXFhnWAt1Irg= +golang.org/x/telemetry v0.0.0-20250417124945-06ef541f3fa3/go.mod h1:RoaXAWDwS90j6FxVKwJdBV+0HCU+llrKUGgJaxiKl6M= +golang.org/x/text v0.25.0 h1:qVyWApTSYLk/drJRO5mDlNYskwQznZmkpV2c8q9zls4= +golang.org/x/text v0.25.0/go.mod h1:WEdwpYrmk1qmdHvhkSTNPm3app7v4rsT8F2UD6+VHIA= +golang.org/x/tools v0.33.0 h1:4qz2S3zmRxbGIhDIAgjxvFutSvH5EfnsYrRBj0UI0bc= +golang.org/x/tools v0.33.0/go.mod h1:CIJMaWEY88juyUfo7UbgPqbC8rU2OqfAV1h2Qp0oMYI= +golang.org/x/tools v0.33.1-0.20250604180551-4d89d8aef849 h1:y+3SPx7RmIoU/DCRF+k0+AhC/jvtMWEzb2m5j2j4OWA= +golang.org/x/tools v0.33.1-0.20250604180551-4d89d8aef849/go.mod h1:CIJMaWEY88juyUfo7UbgPqbC8rU2OqfAV1h2Qp0oMYI= +golang.org/x/tools v0.34.0 h1:qIpSLOxeCYGg9TrcJokLBG4KFA6d795g0xkBkiESGlo= +golang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg= +golang.org/x/tools/gopls v0.19.0-pre.2 h1:CUjqNCFGKHPMRZMd/tkBvtkae+eJuGNHOYsGpKxQNj4= +golang.org/x/tools/gopls v0.19.0-pre.2/go.mod h1:978AwBntyFmG+swdND5BfSs81hQdJQHR3jSUYtKpg0s= diff --git a/internal/encoding/label_column_index.go b/internal/encoding/label_column_index.go new file mode 100644 index 0000000..6ef07ed --- /dev/null +++ b/internal/encoding/label_column_index.go @@ -0,0 +1,49 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package encoding + +import ( + "bytes" + "encoding/binary" + "slices" +) + +func EncodeLabelColumnIndex(s []int) []byte { + l := make([]byte, binary.MaxVarintLen32) + r := make([]byte, 0, len(s)*binary.MaxVarintLen32) + + slices.Sort(s) + + n := binary.PutVarint(l[:], int64(len(s))) + r = append(r, l[:n]...) + + for i := range len(s) { + n := binary.PutVarint(l[:], int64(s[i])) + r = append(r, l[:n]...) + } + + return r +} + +func DecodeLabelColumnIndex(b []byte) ([]int, error) { + buffer := bytes.NewBuffer(b) + + s, err := binary.ReadVarint(buffer) + if err != nil { + return nil, err + } + + r := make([]int, 0, s) + + for range s { + v, err := binary.ReadVarint(buffer) + if err != nil { + return nil, err + } + r = append(r, int(v)) + } + + return r, nil +} diff --git a/internal/encoding/label_column_index_test.go b/internal/encoding/label_column_index_test.go new file mode 100644 index 0000000..365ec4e --- /dev/null +++ b/internal/encoding/label_column_index_test.go @@ -0,0 +1,31 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package encoding + +import ( + "slices" + "testing" + + fuzz "github.com/AdaLogics/go-fuzz-headers" +) + +func FuzzEncodeLabelColumnIndex(f *testing.F) { + f.Fuzz(func(t *testing.T, data []byte) { + fz := fuzz.NewConsumer(data) + + var ( + in []int + ) + fz.CreateSlice(&in) + + decoded, err := DecodeLabelColumnIndex(EncodeLabelColumnIndex(in)) + if err != nil { + t.Fatalf("unable to decode label column index: %s", err) + } + if slices.Compare(decoded, in) != 0 { + t.Fatalf("decoded %q did not match expected %q", decoded, in) + } + }) +} diff --git a/internal/limits/limit.go b/internal/limits/limit.go new file mode 100644 index 0000000..43f9713 --- /dev/null +++ b/internal/limits/limit.go @@ -0,0 +1,93 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package limits + +import ( + "context" + "errors" + "fmt" + "sync" +) + +type resourceExhausted struct { + used int64 +} + +func (re *resourceExhausted) Error() string { + return fmt.Sprintf("resouce exhausted (used %d)", re.used) +} + +func IsResourceExhausted(err error) bool { + var re *resourceExhausted + return errors.As(err, &re) +} + +type Semaphore struct { + n int + c chan struct{} +} + +func NewSempahore(n int) *Semaphore { + return &Semaphore{ + n: n, + c: make(chan struct{}, n), + } +} + +func UnlimitedSemaphore() *Semaphore { + return NewSempahore(0) +} + +func (s *Semaphore) Reserve(ctx context.Context) error { + if s.n == 0 { + return nil + } + select { + case s.c <- struct{}{}: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} + +func (s *Semaphore) Release() { + if s.n == 0 { + return + } + select { + case <-s.c: + default: + panic("semaphore would block on release?") + } +} + +type Quota struct { + mu sync.Mutex + q int64 + u int64 +} + +func NewQuota(n int64) *Quota { + return &Quota{q: n, u: n} +} + +func UnlimitedQuota() *Quota { + return NewQuota(0) +} + +func (q *Quota) Reserve(n int64) error { + if q.q == 0 { + return nil + } + + q.mu.Lock() + defer q.mu.Unlock() + + if q.u-n < 0 { + return &resourceExhausted{used: q.q} + } + q.u -= n + return nil +} diff --git a/internal/util/slice.go b/internal/util/slice.go new file mode 100644 index 0000000..5f5ff38 --- /dev/null +++ b/internal/util/slice.go @@ -0,0 +1,13 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package util + +import "slices" + +func SortUnique(ss []string) []string { + slices.Sort(ss) + + return slices.Compact(ss) +} diff --git a/internal/warnings/warnings.go b/internal/warnings/warnings.go new file mode 100644 index 0000000..bfdb3f2 --- /dev/null +++ b/internal/warnings/warnings.go @@ -0,0 +1,13 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package warnings + +import "errors" + +var ( + ErrorTruncatedResponse = errors.New("results truncated due to limit") + ErrorDroppedSeriesAfterExternalLabelMangling = errors.New("dropped series after external label mangling") + ErrorDroppedLabelValuesAfterExternalLabelMangling = errors.New("dropped label values after external label mangling") +) diff --git a/locate/bucket.go b/locate/bucket.go new file mode 100644 index 0000000..57cc27b --- /dev/null +++ b/locate/bucket.go @@ -0,0 +1,54 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "context" + "fmt" + "io" + + "github.com/alecthomas/units" + "github.com/thanos-io/objstore" + "go.opentelemetry.io/otel/attribute" + + "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" +) + +type bucketReaderAt struct { + ctx context.Context + bkt objstore.Bucket + name string +} + +func newBucketReaderAt(ctx context.Context, bkt objstore.Bucket, name string) *bucketReaderAt { + return &bucketReaderAt{ + ctx: ctx, + bkt: bkt, + name: name, + } +} + +func (br *bucketReaderAt) ReadAt(p []byte, off int64) (int, error) { + ctx, span := tracing.Tracer().Start(br.ctx, "Bucket Get Range") + defer span.End() + + span.SetAttributes(attribute.String("object", br.name)) + span.SetAttributes(attribute.Stringer("bytes", units.Base2Bytes(len(p)).Round(1))) + span.SetAttributes(attribute.Int64("offset", off)) + + bucketRequests.Inc() + + rdc, err := br.bkt.GetRange(ctx, br.name, off, int64(len(p))) + if err != nil { + return 0, fmt.Errorf("unable to read range for %s: %w", br.name, err) + } + defer rdc.Close() + + n, err := io.ReadFull(rdc, p) + if n == len(p) { + return n, nil + } + return n, err +} diff --git a/locate/discover.go b/locate/discover.go new file mode 100644 index 0000000..cf31058 --- /dev/null +++ b/locate/discover.go @@ -0,0 +1,419 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "context" + "encoding/json" + "fmt" + "io" + "maps" + "math" + "slices" + "strings" + "sync" + "time" + + "google.golang.org/protobuf/proto" + + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block/metadata" + + "github.com/cloudflare/parquet-tsdb-poc/proto/metapb" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +type discoveryConfig struct { + concurrency int +} + +type DiscoveryOption func(*discoveryConfig) + +func MetaConcurrency(c int) DiscoveryOption { + return func(cfg *discoveryConfig) { + cfg.concurrency = c + } +} + +type Discoverer struct { + bkt objstore.Bucket + + mu sync.Mutex + metas map[string]schema.Meta + + concurrency int +} + +func NewDiscoverer(bkt objstore.Bucket, opts ...DiscoveryOption) *Discoverer { + cfg := discoveryConfig{ + concurrency: 1, + } + for _, o := range opts { + o(&cfg) + } + return &Discoverer{ + bkt: bkt, + metas: make(map[string]schema.Meta), + concurrency: cfg.concurrency, + } +} + +func (s *Discoverer) Metas() map[string]schema.Meta { + s.mu.Lock() + defer s.mu.Unlock() + + res := make(map[string]schema.Meta, len(s.metas)) + maps.Copy(res, s.metas) + + return res +} + +func (s *Discoverer) Discover(ctx context.Context) error { + m := make(map[string][]string) + err := s.bkt.Iter(ctx, "", func(n string) error { + id, file, ok := schema.SplitBlockPath(n) + if !ok { + return nil + } + m[id] = append(m[id], file) + return nil + }, objstore.WithRecursiveIter()) + if err != nil { + return err + } + + type metaOrError struct { + m schema.Meta + err error + } + + metaC := make(chan metaOrError) + go func() { + defer close(metaC) + + workerC := make(chan string, s.concurrency) + go func() { + defer close(workerC) + + for k, v := range m { + if !slices.Contains(v, schema.MetaFile) { + // skip incomplete block + continue + } + if _, ok := s.metas[k]; ok { + // we already got the block + continue + } + workerC <- k + } + }() + + var wg sync.WaitGroup + defer wg.Wait() + + for range s.concurrency { + wg.Add(1) + go func() { + defer wg.Done() + for k := range workerC { + meta, err := readMetaFile(ctx, s.bkt, k) + if err != nil { + metaC <- metaOrError{err: fmt.Errorf("unable to read meta file for %q: %w", k, err)} + } else { + metaC <- metaOrError{m: meta} + } + } + }() + } + }() + + am := make(map[string]struct{}) + for k, v := range m { + if !slices.Contains(v, schema.MetaFile) { + // skip incomplete block + continue + } + am[k] = struct{}{} + } + + nm := make(map[string]schema.Meta) + for m := range metaC { + if m.err != nil { + return fmt.Errorf("unable to read meta: %w", m.err) + } + nm[m.m.Name] = m.m + } + + s.mu.Lock() + defer s.mu.Unlock() + + maps.Copy(s.metas, nm) + + // delete metas that are no longer in the bucket + maps.DeleteFunc(s.metas, func(k string, _ schema.Meta) bool { + _, ok := am[k] + return !ok + }) + + if len(s.metas) != 0 { + mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) + for _, v := range s.metas { + mint = min(mint, v.Mint) + maxt = max(maxt, v.Mint) + } + syncMinTime.WithLabelValues(whatDiscoverer).Set(float64(mint)) + syncMaxTime.WithLabelValues(whatDiscoverer).Set(float64(maxt)) + } + syncLastSuccessfulTime.WithLabelValues(whatDiscoverer).SetToCurrentTime() + + return nil +} + +func readMetaFile(ctx context.Context, bkt objstore.Bucket, name string) (schema.Meta, error) { + mfile := schema.MetaFileNameForBlock(name) + if _, err := bkt.Attributes(ctx, mfile); err != nil { + return schema.Meta{}, fmt.Errorf("unable to attr %s: %w", mfile, err) + } + rdr, err := bkt.Get(ctx, mfile) + if err != nil { + return schema.Meta{}, fmt.Errorf("unable to get %s: %w", mfile, err) + } + defer rdr.Close() + + metaBytes, err := io.ReadAll(rdr) + if err != nil { + return schema.Meta{}, fmt.Errorf("unable to read %s: %w", mfile, err) + } + + metapb := &metapb.Metadata{} + if err := proto.Unmarshal(metaBytes, metapb); err != nil { + return schema.Meta{}, fmt.Errorf("unable to read %s: %w", mfile, err) + } + + // for version == 0 + m := make(map[string][]string, len(metapb.GetColumnsForName())) + for k, v := range metapb.GetColumnsForName() { + m[k] = v.GetColumns() + } + return schema.Meta{ + Version: int(metapb.GetVersion()), + Name: name, + Mint: metapb.GetMint(), + Maxt: metapb.GetMaxt(), + Shards: metapb.GetShards(), + ColumnsForName: m, + }, nil +} + +type tsdbDiscoveryConfig struct { + concurrency int + + externalLabelMatchers []*labels.Matcher + minBlockAge time.Duration +} + +type TSDBDiscoveryOption func(*tsdbDiscoveryConfig) + +func TSDBMetaConcurrency(c int) TSDBDiscoveryOption { + return func(cfg *tsdbDiscoveryConfig) { + cfg.concurrency = c + } +} + +func TSDBMatchExternalLabels(ms ...*labels.Matcher) TSDBDiscoveryOption { + return func(cfg *tsdbDiscoveryConfig) { + cfg.externalLabelMatchers = ms + } +} + +func TSDBMinBlockAge(d time.Duration) TSDBDiscoveryOption { + return func(cfg *tsdbDiscoveryConfig) { + cfg.minBlockAge = d + } +} + +type TSDBDiscoverer struct { + bkt objstore.Bucket + + mu sync.Mutex + metas map[string]metadata.Meta + + externalLabelMatchers []*labels.Matcher + minBlockAge time.Duration + + concurrency int +} + +func NewTSDBDiscoverer(bkt objstore.Bucket, opts ...TSDBDiscoveryOption) *TSDBDiscoverer { + cfg := tsdbDiscoveryConfig{ + concurrency: 1, + } + for _, o := range opts { + o(&cfg) + } + return &TSDBDiscoverer{ + bkt: bkt, + metas: make(map[string]metadata.Meta), + concurrency: cfg.concurrency, + externalLabelMatchers: cfg.externalLabelMatchers, + minBlockAge: cfg.minBlockAge, + } +} + +func (s *TSDBDiscoverer) Metas() map[string]metadata.Meta { + s.mu.Lock() + defer s.mu.Unlock() + + res := make(map[string]metadata.Meta, len(s.metas)) + maps.Copy(res, s.metas) + + return res +} + +func (s *TSDBDiscoverer) Discover(ctx context.Context) error { + m := make(map[string][]string) + err := s.bkt.Iter(ctx, "", func(n string) error { + split := strings.Split(n, "/") + if len(split) != 2 { + return nil + } + id, f := split[0], split[1] + + m[id] = append(m[id], f) + return nil + }, objstore.WithRecursiveIter()) + if err != nil { + return err + } + + type metaOrError struct { + m metadata.Meta + err error + } + + metaC := make(chan metaOrError) + go func() { + defer close(metaC) + + workerC := make(chan string, s.concurrency) + go func() { + defer close(workerC) + + for k, v := range m { + if !slices.Contains(v, metadata.MetaFilename) { + // skip incomplete block + continue + } + if slices.Contains(v, metadata.DeletionMarkFilename) { + // skip block that is about to be deleted + continue + } + if _, ok := s.metas[k]; ok { + // we already got the block + continue + } + workerC <- k + } + }() + + var wg sync.WaitGroup + defer wg.Wait() + + for range s.concurrency { + wg.Add(1) + go func() { + defer wg.Done() + for k := range workerC { + meta, err := s.readMetaFile(ctx, k) + if err != nil { + metaC <- metaOrError{err: fmt.Errorf("unable to read meta file for %q: %w", k, err)} + } else { + metaC <- metaOrError{m: meta} + } + } + }() + } + }() + + am := make(map[string]struct{}) + for k, v := range m { + if !slices.Contains(v, metadata.MetaFilename) { + // skip incomplete block + continue + } + if slices.Contains(v, metadata.DeletionMarkFilename) { + // skip block that is about to be deleted + continue + } + am[k] = struct{}{} + } + + nm := make(map[string]metadata.Meta) + for m := range metaC { + if m.err != nil { + return fmt.Errorf("unable to read meta: %w", m.err) + } + nm[m.m.ULID.String()] = m.m + } + + // filter for metas that match our external labels + maps.DeleteFunc(nm, func(_ string, v metadata.Meta) bool { + series := labels.FromMap(v.Thanos.Labels) + + for _, m := range s.externalLabelMatchers { + if !m.Matches(series.Get(m.Name)) { + return true + } + } + return false + }) + + s.mu.Lock() + defer s.mu.Unlock() + + maps.Copy(s.metas, nm) + + // filter for metas that dont contain data after "now-minAge" + maps.DeleteFunc(s.metas, func(_ string, v metadata.Meta) bool { + return time.UnixMilli(v.MaxTime).After(time.Now().Add(-s.minBlockAge)) + }) + // delete metas that are no longer in the bucket + maps.DeleteFunc(s.metas, func(k string, _ metadata.Meta) bool { + _, ok := am[k] + return !ok + }) + + if len(s.metas) != 0 { + mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) + for _, v := range s.metas { + mint = min(mint, v.MinTime) + maxt = max(maxt, v.MaxTime) + } + syncMinTime.WithLabelValues(whatDiscoverer).Set(float64(mint)) + syncMaxTime.WithLabelValues(whatDiscoverer).Set(float64(maxt)) + } + syncLastSuccessfulTime.WithLabelValues(whatDiscoverer).SetToCurrentTime() + + return nil +} + +func (s *TSDBDiscoverer) readMetaFile(ctx context.Context, name string) (metadata.Meta, error) { + mfile := fmt.Sprintf("%s/%s", name, metadata.MetaFilename) + if _, err := s.bkt.Attributes(ctx, mfile); err != nil { + return metadata.Meta{}, fmt.Errorf("unable to attr %s: %w", mfile, err) + } + rdr, err := s.bkt.Get(ctx, mfile) + if err != nil { + return metadata.Meta{}, fmt.Errorf("unable to get %s: %w", mfile, err) + } + defer rdr.Close() + + var m metadata.Meta + if err := json.NewDecoder(rdr).Decode(&m); err != nil { + return metadata.Meta{}, fmt.Errorf("unable to decode %s: %w", mfile, err) + } + return m, nil +} diff --git a/locate/discover_test.go b/locate/discover_test.go new file mode 100644 index 0000000..ad3cde4 --- /dev/null +++ b/locate/discover_test.go @@ -0,0 +1,220 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "maps" + "path/filepath" + "slices" + "testing" + "time" + + "github.com/oklog/ulid/v2" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/util/teststorage" + "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" + "github.com/thanos-io/thanos/pkg/block/metadata" + + "github.com/cloudflare/parquet-tsdb-poc/convert" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" +) + +func TestDiscoverer(t *testing.T) { + t.Run("Discoverer discovers newly uploaded blocks", func(tt *testing.T) { + ctx := tt.Context() + bkt, err := filesystem.NewBucket(tt.TempDir()) + if err != nil { + tt.Fatalf("unable to create bucket: %s", err) + } + discoverer := NewDiscoverer(bkt) + + d := util.BeginOfDay(time.UnixMilli(0)).UTC() + if err := createBlockForDay(ctx, tt, bkt, d); err != nil { + tt.Fatalf("unable to create block for day: %s", err) + } + + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas := discoverer.Metas() + if expect, got := []string{"1970/01/01"}, slices.Sorted(maps.Keys(metas)); !slices.Equal(got, expect) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + + // Add another block + d = util.BeginOfDay(time.UnixMilli(0).AddDate(0, 0, 1)).UTC() + if err := createBlockForDay(ctx, tt, bkt, d); err != nil { + tt.Fatalf("unable to create block for day: %s", err) + } + + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas = discoverer.Metas() + if expect, got := []string{"1970/01/01", "1970/01/02"}, slices.Sorted(maps.Keys(metas)); !slices.Equal(got, expect) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + }) +} + +func TestTSDBDiscoverer(t *testing.T) { + t.Run("Discoverer skips blocks that are not matching", func(tt *testing.T) { + ctx := tt.Context() + bkt, err := filesystem.NewBucket(tt.TempDir()) + if err != nil { + tt.Fatalf("unable to create bucket: %s", err) + } + + for _, m := range []metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JS0DPYGA1HPW5RBZ1KBXCNXK"), + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{ + "foo": "bar", + }, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JT0DPYGA1HPW5RBZ1KBXCNXK"), + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{ + "foo": "not-bar", + }, + }, + }, + } { + buf := bytes.NewBuffer(nil) + if err := json.NewEncoder(buf).Encode(m); err != nil { + tt.Fatalf("unable to encode meta file: %s", err) + } + if err := bkt.Upload(ctx, filepath.Join(m.ULID.String(), metadata.MetaFilename), buf); err != nil { + tt.Fatalf("unable to upload meta file: %s", err) + } + } + + discoverer := NewTSDBDiscoverer(bkt, TSDBMatchExternalLabels(labels.MustNewMatcher(labels.MatchNotEqual, "foo", "bar"))) + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas := discoverer.Metas() + if expect, got := []string{"01JT0DPYGA1HPW5RBZ1KBXCNXK"}, slices.Collect(maps.Keys(metas)); !slices.Equal(got, expect) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + }) + t.Run("Discoverer skips blocks with deletion markers", func(tt *testing.T) { + ctx := tt.Context() + bkt, err := filesystem.NewBucket(tt.TempDir()) + if err != nil { + tt.Fatalf("unable to create bucket: %s", err) + } + + meta := metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JS0DPYGA1HPW5RBZ1KBXCNXK"), + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{ + "foo": "bar", + }, + }, + } + buf := bytes.NewBuffer(nil) + if err := json.NewEncoder(buf).Encode(meta); err != nil { + tt.Fatalf("unable to encode meta file: %s", err) + } + if err := bkt.Upload(ctx, filepath.Join(meta.ULID.String(), metadata.MetaFilename), buf); err != nil { + tt.Fatalf("unable to upload meta file: %s", err) + } + + buf.Reset() + if err := bkt.Upload(ctx, filepath.Join(meta.ULID.String(), metadata.DeletionMarkFilename), buf); err != nil { + tt.Fatalf("unable to upload deletion file: %s", err) + } + + discoverer := NewTSDBDiscoverer(bkt) + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas := discoverer.Metas() + if got := slices.Collect(maps.Keys(metas)); len(got) != 0 { + tt.Errorf("expected empty slice, got: %+v", got) + } + }) + t.Run("Discoverer forgets blocks that are no longer there", func(tt *testing.T) { + ctx := tt.Context() + bkt, err := filesystem.NewBucket(tt.TempDir()) + if err != nil { + tt.Fatalf("unable to create bucket: %s", err) + } + + meta := metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("01JS0DPYGA1HPW5RBZ1KBXCNXK"), + }, + } + buf := bytes.NewBuffer(nil) + if err := json.NewEncoder(buf).Encode(meta); err != nil { + tt.Fatalf("unable to encode meta file: %s", err) + } + if err := bkt.Upload(ctx, filepath.Join(meta.ULID.String(), metadata.MetaFilename), buf); err != nil { + tt.Fatalf("unable to upload meta file: %s", err) + } + + discoverer := NewTSDBDiscoverer(bkt) + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas := discoverer.Metas() + if expect, got := []string{"01JS0DPYGA1HPW5RBZ1KBXCNXK"}, slices.Collect(maps.Keys(metas)); !slices.Equal(got, expect) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + + // delete the block + if err := bkt.Delete(ctx, meta.BlockMeta.ULID.String()); err != nil { + tt.Fatalf("unable to delete block: %s", err) + } + if err := discoverer.Discover(ctx); err != nil { + tt.Fatalf("unable to discover tsdb metas: %s", err) + } + + metas = discoverer.Metas() + if got := slices.Collect(maps.Keys(metas)); len(got) != 0 { + tt.Errorf("expected empty slice, got: %+v", got) + } + + }) +} + +func createBlockForDay(ctx context.Context, t *testing.T, bkt objstore.Bucket, d time.Time) error { + st := teststorage.New(t) + t.Cleanup(func() { _ = st.Close() }) + + app := st.Appender(ctx) + app.Append(0, labels.FromStrings("foo", "bar"), d.UnixMilli(), 1) + if err := app.Commit(); err != nil { + return fmt.Errorf("unable to commit samples: %s", err) + } + + h := st.Head() + if err := convert.ConvertTSDBBlock(ctx, bkt, d, []convert.Convertable{h}); err != nil { + return fmt.Errorf("unable to convert blocks: %s", err) + } + return nil +} diff --git a/locate/filter.go b/locate/filter.go new file mode 100644 index 0000000..9b1c9a1 --- /dev/null +++ b/locate/filter.go @@ -0,0 +1,96 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/thanos-io/thanos/pkg/info/infopb" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +type MetaFilter interface { + filterMetas(map[string]schema.Meta) map[string]schema.Meta + filterBlocks([]*db.Block) []*db.Block +} + +var AllMetasMetaFilter = allMetas{} + +type allMetas struct { +} + +func (mf allMetas) filterMetas(metas map[string]schema.Meta) map[string]schema.Meta { return metas } +func (mf allMetas) filterBlocks(blocks []*db.Block) []*db.Block { return blocks } + +type ThanosBackfillMetaFilter struct { + endpoint string + overlap int64 + + mu sync.Mutex + mint, maxt int64 +} + +func NewThanosBackfillMetaFilter(endpoint string, overlap time.Duration) *ThanosBackfillMetaFilter { + return &ThanosBackfillMetaFilter{endpoint: endpoint, overlap: overlap.Milliseconds()} +} + +func (tp *ThanosBackfillMetaFilter) filterMetas(metas map[string]schema.Meta) map[string]schema.Meta { + tp.mu.Lock() + defer tp.mu.Unlock() + + res := make(map[string]schema.Meta, len(metas)) + for k, v := range metas { + if util.Contains(min(tp.mint+tp.overlap, tp.maxt), tp.maxt, v.Mint, v.Maxt) { + continue + } + res[k] = v + } + return res +} + +func (tp *ThanosBackfillMetaFilter) filterBlocks(blks []*db.Block) []*db.Block { + tp.mu.Lock() + defer tp.mu.Unlock() + + res := make([]*db.Block, 0, len(blks)) + for _, blk := range blks { + blkMint, blkMaxt := blk.Timerange() + if util.Contains(min(tp.mint+tp.overlap, tp.maxt), tp.maxt, blkMint, blkMaxt) { + continue + } + res = append(res, blk) + } + return res +} + +func (tp *ThanosBackfillMetaFilter) Update(ctx context.Context) error { + // Note: we assume that thanos runs close to this server, we dont need TLS here + cc, err := grpc.NewClient(tp.endpoint, grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return fmt.Errorf("unable to connect: %w", err) + } + client := infopb.NewInfoClient(cc) + + info, err := client.Info(ctx, &infopb.InfoRequest{}) + if err != nil { + return fmt.Errorf("unable to get store time range from thanos: %w", err) + } + + tp.mu.Lock() + defer tp.mu.Unlock() + + tp.mint = info.Store.MinTime + tp.maxt = info.Store.MaxTime + + return nil +} diff --git a/locate/metrics.go b/locate/metrics.go new file mode 100644 index 0000000..c0426f4 --- /dev/null +++ b/locate/metrics.go @@ -0,0 +1,57 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "errors" + + "github.com/prometheus/client_golang/prometheus" +) + +const ( + whatSyncer = "syncer" + whatDiscoverer = "discoverer" + whatTSDBDiscoverer = "tsdb_discoverer" +) + +var ( + bucketRequests = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "bucket_requests_total", + Help: "Total amount of requests to object storage", + }) + syncMinTime = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: "sync_min_time_unix_seconds", + Help: "The minimum timestamp that syncer knows", + }, []string{"what"}) + syncMaxTime = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: "sync_max_time_unix_seconds", + Help: "The minimum timestamp that syncer knows", + }, []string{"what"}) + syncLastSuccessfulTime = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: "sync_last_successful_update_time_unix_seconds", + Help: "The timestamp we last synced successfully", + }, []string{"what"}) + syncCorruptedLabelFile = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "sync_corrupted_label_parquet_files_total", + Help: "The amount of corrupted label parquet files we encountered", + }) +) + +func RegisterMetrics(reg prometheus.Registerer) error { + bucketRequests.Add(0) + for _, w := range []string{whatSyncer, whatDiscoverer, whatTSDBDiscoverer} { + syncMinTime.WithLabelValues(w).Set(0) + syncMaxTime.WithLabelValues(w).Set(0) + syncLastSuccessfulTime.WithLabelValues(w).Set(0) + } + + return errors.Join( + reg.Register(bucketRequests), + reg.Register(syncMinTime), + reg.Register(syncMaxTime), + reg.Register(syncLastSuccessfulTime), + reg.Register(syncCorruptedLabelFile), + ) +} diff --git a/locate/syncer.go b/locate/syncer.go new file mode 100644 index 0000000..f0738b6 --- /dev/null +++ b/locate/syncer.go @@ -0,0 +1,327 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "maps" + "net/url" + "os" + "path/filepath" + "slices" + "sort" + "sync" + + "github.com/alecthomas/units" + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/thanos-io/objstore" + + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +type Syncer struct { + bkt objstore.Bucket + + blockOpts []BlockOption + metaFilter MetaFilter + concurrency int + + mu sync.Mutex + blocks map[string]*db.Block + + cached []*db.Block +} + +type SyncerOption func(*syncerConfig) + +type syncerConfig struct { + blockOpts []BlockOption + metaFilter MetaFilter + concurrency int +} + +func BlockOptions(opts ...BlockOption) SyncerOption { + return func(cfg *syncerConfig) { + cfg.blockOpts = opts + } +} + +func FilterMetas(f MetaFilter) SyncerOption { + return func(cfg *syncerConfig) { + cfg.metaFilter = f + } +} + +func BlockConcurrency(c int) SyncerOption { + return func(cfg *syncerConfig) { + cfg.concurrency = c + } +} + +type BlockOption func(*blockConfig) + +type blockConfig struct { + readBufferSize units.Base2Bytes + labelFilesDir string +} + +func ReadBufferSize(sz units.Base2Bytes) BlockOption { + return func(cfg *blockConfig) { + cfg.readBufferSize = sz + } +} + +func LabelFilesDir(d string) BlockOption { + return func(cfg *blockConfig) { + cfg.labelFilesDir = d + } +} + +func NewSyncer(bkt objstore.Bucket, opts ...SyncerOption) *Syncer { + cfg := syncerConfig{ + metaFilter: AllMetasMetaFilter, + concurrency: 1, + } + + for _, o := range opts { + o(&cfg) + } + + return &Syncer{ + bkt: bkt, + blocks: make(map[string]*db.Block), + blockOpts: cfg.blockOpts, + metaFilter: cfg.metaFilter, + concurrency: cfg.concurrency, + } +} + +func (s *Syncer) Blocks() []*db.Block { + s.mu.Lock() + defer s.mu.Unlock() + + return s.filterBlocks(s.cached) +} + +func (s *Syncer) Sync(ctx context.Context, metas map[string]schema.Meta) error { + type blockOrError struct { + blk *db.Block + err error + } + + blkC := make(chan blockOrError) + go func() { + defer close(blkC) + + workerC := make(chan schema.Meta, s.concurrency) + go func() { + defer close(workerC) + + for k, v := range s.filterMetas(metas) { + if _, ok := s.blocks[k]; ok { + continue + } + workerC <- v + } + }() + + var wg sync.WaitGroup + defer wg.Wait() + + for range s.concurrency { + wg.Add(1) + go func() { + defer wg.Done() + for m := range workerC { + blk, err := newBlockForMeta(ctx, s.bkt, m, s.blockOpts...) + if err != nil { + blkC <- blockOrError{err: fmt.Errorf("unable to read block %q: %w", m.Name, err)} + } else { + blkC <- blockOrError{blk: blk} + } + } + }() + } + }() + + blocks := make(map[string]*db.Block, 0) + for b := range blkC { + if b.err != nil { + return fmt.Errorf("unable to read block: %w", b.err) + } + blocks[b.blk.Meta().Name] = b.blk + } + + s.mu.Lock() + defer s.mu.Unlock() + + // delete blocks that are not in meta map + maps.DeleteFunc(s.blocks, func(k string, _ *db.Block) bool { _, ok := metas[k]; return !ok }) + + // add new blocks that we just loaded + maps.Copy(s.blocks, blocks) + + s.cached = slices.Collect(maps.Values(s.blocks)) + sort.Slice(s.cached, func(i, j int) bool { + ls, _ := s.cached[i].Timerange() + rs, _ := s.cached[j].Timerange() + return ls < rs + }) + + if len(s.cached) != 0 { + syncMinTime.WithLabelValues(whatSyncer).Set(float64(s.cached[0].Meta().Mint)) + syncMaxTime.WithLabelValues(whatSyncer).Set(float64(s.cached[len(s.cached)-1].Meta().Maxt)) + } + syncLastSuccessfulTime.WithLabelValues(whatSyncer).SetToCurrentTime() + + return nil +} + +func (s *Syncer) filterMetas(metas map[string]schema.Meta) map[string]schema.Meta { + return s.metaFilter.filterMetas(metas) +} + +func (s *Syncer) filterBlocks(blks []*db.Block) []*db.Block { + return s.metaFilter.filterBlocks(blks) +} + +/* +TODO: the following functions should be abstracted into syncer somehow, the syncer should decide +where shards live (s3, disk, memory) and should also do regular maintenance on its on disk state, +i.e. downloading files that it wants on disk, deleting files that are out of retention, etc. +*/ + +func newBlockForMeta(ctx context.Context, bkt objstore.Bucket, m schema.Meta, opts ...BlockOption) (*db.Block, error) { + cfg := blockConfig{ + readBufferSize: 8 * units.MiB, + labelFilesDir: os.TempDir(), + } + for _, o := range opts { + o(&cfg) + } + + shards, err := readShards(ctx, bkt, m, cfg) + if err != nil { + return nil, fmt.Errorf("unable to read shards: %w", err) + } + + return db.NewBlock(m, shards...), nil +} + +func readShards(ctx context.Context, bkt objstore.Bucket, m schema.Meta, cfg blockConfig) ([]*db.Shard, error) { + shards := make([]*db.Shard, 0, m.Shards) + for i := range int(m.Shards) { + shard, err := readShard(ctx, bkt, m, i, cfg) + if err != nil { + return nil, fmt.Errorf("unable to read shard %d: %w", i, err) + } + shards = append(shards, shard) + } + return shards, nil +} + +func bucketReaderFromContext(bkt objstore.Bucket, name string) func(context.Context) io.ReaderAt { + return func(ctx context.Context) io.ReaderAt { + return newBucketReaderAt(ctx, bkt, name) + } +} + +func readShard(ctx context.Context, bkt objstore.Bucket, m schema.Meta, i int, cfg blockConfig) (s *db.Shard, err error) { + chunkspfile := schema.ChunksPfileNameForShard(m.Name, i) + attrs, err := bkt.Attributes(ctx, chunkspfile) + if err != nil { + return nil, fmt.Errorf("unable to attr chunks parquet file %q: %w", chunkspfile, err) + } + + bktRdrAtFromCtx := bucketReaderFromContext(bkt, chunkspfile) + + chunkspf, err := parquet.OpenFile(bktRdrAtFromCtx(ctx), attrs.Size, + parquet.FileReadMode(parquet.ReadModeAsync), + parquet.ReadBufferSize(int(cfg.readBufferSize)), + parquet.SkipMagicBytes(true), + parquet.SkipBloomFilters(true), + parquet.OptimisticRead(true), + ) + if err != nil { + return nil, fmt.Errorf("unable to open chunks parquet file %q: %w", chunkspfile, err) + } + + labelspfile := schema.LabelsPfileNameForShard(m.Name, i) + labelspfilePath := filepath.Join(cfg.labelFilesDir, url.PathEscape(labelspfile)) + + // If we were not able to read the file for any reason we delete it and retry. + // This is also executed on paths that have nothing to do with the file, i.e. loading + // its content from object storage, but just makes sure that we dont forget cleanup. + defer func() { + if err != nil { + if cerr := os.RemoveAll(labelspfilePath); cerr != nil { + err = errors.Join(err, fmt.Errorf("unable to remove labels parquet file %q: %w", labelspfilePath, cerr)) + } + } + }() + + // if the file was corrupted on its way to disk we remove it and will retry downloading it next try + if stat, err := os.Stat(labelspfilePath); err != nil { + if !os.IsNotExist(err) { + return nil, fmt.Errorf("unable to stat label parquet file %q from disk: %w", labelspfile, err) + // file didnt exist - we need to download and save it to disk + } + } else { + f, err := fileutil.OpenMmapFileWithSize(labelspfilePath, int(stat.Size())) + if err != nil { + return nil, fmt.Errorf("unable to mmap label parquet file %q: %w", labelspfile, err) + } + labelspf, err := parquet.OpenFile(bytes.NewReader(f.Bytes()), stat.Size()) + if err != nil { + syncCorruptedLabelFile.Add(1) + rerr := fmt.Errorf("unable to read label parquet file %q: %w", labelspfile, err) + if cerr := f.Close(); cerr != nil { + return nil, errors.Join(rerr, fmt.Errorf("unable to close memory mapped parquet file %q: %w", labelspfile, cerr)) + } + return nil, rerr + } + return db.NewShard(m, chunkspf, labelspf, bktRdrAtFromCtx), nil + } + rdr, err := bkt.Get(ctx, labelspfile) + if err != nil { + return nil, fmt.Errorf("unable to get %q: %w", labelspfile, err) + } + defer rdr.Close() + + f, err := os.Create(labelspfilePath) + if err != nil { + return nil, fmt.Errorf("unable to create label parquet file %q on disk: %w", labelspfile, err) + } + defer f.Close() + + n, err := io.Copy(f, rdr) + if err != nil { + return nil, fmt.Errorf("unable to copy label parquet file %q to disk: %w", labelspfile, err) + } + if err := f.Sync(); err != nil { + return nil, fmt.Errorf("unable to close label parquet file %q: %w", labelspfile, err) + } + + mf, err := fileutil.OpenMmapFileWithSize(labelspfilePath, int(n)) + if err != nil { + return nil, fmt.Errorf("unable to mmap label parquet file %q: %w", labelspfile, err) + } + labelspf, err := parquet.OpenFile(bytes.NewReader(mf.Bytes()), int64(n)) + if err != nil { + syncCorruptedLabelFile.Add(1) + rerr := fmt.Errorf("unable to read label parquet file %q: %w", labelspfile, err) + if cerr := f.Close(); cerr != nil { + return nil, errors.Join(rerr, fmt.Errorf("unable to close memory mapped parquet file %q: %w", labelspfile, cerr)) + } + return nil, rerr + } + return db.NewShard(m, chunkspf, labelspf, bktRdrAtFromCtx), nil +} diff --git a/locate/syncer_test.go b/locate/syncer_test.go new file mode 100644 index 0000000..cff4302 --- /dev/null +++ b/locate/syncer_test.go @@ -0,0 +1,60 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package locate + +import ( + "slices" + "testing" + "time" + + "github.com/thanos-io/objstore/providers/filesystem" + + "github.com/cloudflare/parquet-tsdb-poc/db" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +func TestSyncer(t *testing.T) { + t.Run("Syncer always has the blocks from the discovered metas", func(tt *testing.T) { + ctx := tt.Context() + bkt, err := filesystem.NewBucket(tt.TempDir()) + if err != nil { + tt.Fatalf("unable to create bucket: %s", err) + } + syncer := NewSyncer(bkt) + + d := util.BeginOfDay(time.UnixMilli(0)).UTC() + if err := createBlockForDay(ctx, tt, bkt, d); err != nil { + tt.Fatalf("unable to create block for day: %s", err) + } + + m := map[string]schema.Meta{ + "1970/01/01": { + Version: schema.V1, + Name: "1970/01/01", + Mint: d.UnixMilli(), + Maxt: d.AddDate(0, 0, 1).UnixMilli(), + Shards: 1, + }, + } + + if err := syncer.Sync(ctx, m); err != nil { + tt.Fatalf("unable to sync blocks: %s", err) + } + + if expect, got := []string{"1970/01/01"}, syncer.Blocks(); !slices.EqualFunc(expect, got, func(l string, r *db.Block) bool { return l == r.Meta().Name }) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + + delete(m, "1970/01/01") + if err := syncer.Sync(ctx, m); err != nil { + tt.Fatalf("unable to sync blocks: %s", err) + } + + if expect, got := []string{}, syncer.Blocks(); !slices.EqualFunc(expect, got, func(l string, r *db.Block) bool { return l == r.Meta().Name }) { + tt.Errorf("expected: %+v, got: %+v", expect, got) + } + }) +} diff --git a/proto/metapb/meta.pb.go b/proto/metapb/meta.pb.go index 9c8cac2..88fe230 100644 --- a/proto/metapb/meta.pb.go +++ b/proto/metapb/meta.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.2 -// protoc v5.28.3 +// protoc-gen-go v1.36.6 +// protoc v5.29.4 // source: meta.proto package metapb @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" sync "sync" + unsafe "unsafe" ) const ( @@ -21,14 +22,14 @@ const ( ) type Metadata struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Mint int64 `protobuf:"varint,1,opt,name=mint,proto3" json:"mint,omitempty"` - Maxt int64 `protobuf:"varint,2,opt,name=maxt,proto3" json:"maxt,omitempty"` - Shards int64 `protobuf:"varint,3,opt,name=shards,proto3" json:"shards,omitempty"` - ColumnsForName map[string]*Columns `protobuf:"bytes,4,rep,name=columnsForName,proto3" json:"columnsForName,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + state protoimpl.MessageState `protogen:"open.v1"` + Mint int64 `protobuf:"varint,1,opt,name=mint,proto3" json:"mint,omitempty"` + Maxt int64 `protobuf:"varint,2,opt,name=maxt,proto3" json:"maxt,omitempty"` + Shards int64 `protobuf:"varint,3,opt,name=shards,proto3" json:"shards,omitempty"` + ColumnsForName map[string]*Columns `protobuf:"bytes,4,rep,name=columnsForName,proto3" json:"columnsForName,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Metadata) Reset() { @@ -89,12 +90,18 @@ func (x *Metadata) GetColumnsForName() map[string]*Columns { return nil } +func (x *Metadata) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + type Columns struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"open.v1"` + Columns []string `protobuf:"bytes,1,rep,name=columns,proto3" json:"columns,omitempty"` unknownFields protoimpl.UnknownFields - - Columns []string `protobuf:"bytes,1,rep,name=columns,proto3" json:"columns,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Columns) Reset() { @@ -136,40 +143,30 @@ func (x *Columns) GetColumns() []string { var File_meta_proto protoreflect.FileDescriptor -var file_meta_proto_rawDesc = []byte{ - 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x6d, 0x65, - 0x74, 0x61, 0x22, 0xe8, 0x01, 0x0a, 0x08, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, - 0x12, 0x0a, 0x04, 0x6d, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x6d, - 0x69, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6d, 0x61, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x04, 0x6d, 0x61, 0x78, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, - 0x4a, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x46, 0x6f, 0x72, 0x4e, 0x61, 0x6d, - 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x46, - 0x6f, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x46, 0x6f, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x50, 0x0a, 0x13, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x46, 0x6f, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x23, 0x0a, - 0x07, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x66, 0x6c, 0x61, 0x72, 0x65, 0x2f, 0x70, 0x61, 0x72, 0x71, - 0x75, 0x65, 0x74, 0x2d, 0x74, 0x73, 0x64, 0x62, 0x2d, 0x70, 0x6f, 0x63, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, -} +const file_meta_proto_rawDesc = "" + + "\n" + + "\n" + + "meta.proto\x12\x04meta\"\x82\x02\n" + + "\bMetadata\x12\x12\n" + + "\x04mint\x18\x01 \x01(\x03R\x04mint\x12\x12\n" + + "\x04maxt\x18\x02 \x01(\x03R\x04maxt\x12\x16\n" + + "\x06shards\x18\x03 \x01(\x03R\x06shards\x12J\n" + + "\x0ecolumnsForName\x18\x04 \x03(\v2\".meta.Metadata.ColumnsForNameEntryR\x0ecolumnsForName\x12\x18\n" + + "\aversion\x18\x05 \x01(\x03R\aversion\x1aP\n" + + "\x13ColumnsForNameEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12#\n" + + "\x05value\x18\x02 \x01(\v2\r.meta.ColumnsR\x05value:\x028\x01\"#\n" + + "\aColumns\x12\x18\n" + + "\acolumns\x18\x01 \x03(\tR\acolumnsB5Z3github.com/cloudflare/parquet-tsdb-poc/proto/metapbb\x06proto3" var ( file_meta_proto_rawDescOnce sync.Once - file_meta_proto_rawDescData = file_meta_proto_rawDesc + file_meta_proto_rawDescData []byte ) func file_meta_proto_rawDescGZIP() []byte { file_meta_proto_rawDescOnce.Do(func() { - file_meta_proto_rawDescData = protoimpl.X.CompressGZIP(file_meta_proto_rawDescData) + file_meta_proto_rawDescData = protoimpl.X.CompressGZIP(unsafe.Slice(unsafe.StringData(file_meta_proto_rawDesc), len(file_meta_proto_rawDesc))) }) return file_meta_proto_rawDescData } @@ -199,7 +196,7 @@ func file_meta_proto_init() { out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), - RawDescriptor: file_meta_proto_rawDesc, + RawDescriptor: unsafe.Slice(unsafe.StringData(file_meta_proto_rawDesc), len(file_meta_proto_rawDesc)), NumEnums: 0, NumMessages: 3, NumExtensions: 0, @@ -210,7 +207,6 @@ func file_meta_proto_init() { MessageInfos: file_meta_proto_msgTypes, }.Build() File_meta_proto = out.File - file_meta_proto_rawDesc = nil file_meta_proto_goTypes = nil file_meta_proto_depIdxs = nil } diff --git a/proto/metapb/meta.proto b/proto/metapb/meta.proto index 6572f91..4613f49 100644 --- a/proto/metapb/meta.proto +++ b/proto/metapb/meta.proto @@ -9,6 +9,7 @@ message Metadata { int64 maxt = 2; int64 shards = 3; map columnsForName = 4; + int64 version = 5; } message Columns { diff --git a/revive.toml b/revive.toml index 087d775..c7825c1 100644 --- a/revive.toml +++ b/revive.toml @@ -13,7 +13,6 @@ warningCode = 1 [rule.error-naming] [rule.if-return] [rule.increment-decrement] -[rule.var-naming] [rule.var-declaration] [rule.range] [rule.receiver-naming] @@ -27,4 +26,6 @@ warningCode = 1 [rule.redefines-builtin-id] [rule.file-header] arguments = ["Copyright \\(c\\) 2025 Cloudflare, Inc."] +[rule.var-naming] +arguments = [[], [], [{ skip-package-name-checks = true }]] diff --git a/schema/block.go b/schema/block.go index baea5c0..ca932a7 100644 --- a/schema/block.go +++ b/schema/block.go @@ -17,6 +17,14 @@ const ( dateFormat = "%04d/%02d/%02d" ) +type Meta struct { + Version int + Name string + Mint, Maxt int64 + Shards int64 + ColumnsForName map[string][]string +} + func SplitBlockPath(name string) (string, string, bool) { var ( year, month, day int @@ -32,6 +40,21 @@ func SplitBlockPath(name string) (string, string, bool) { return filepath.Dir(name), file, true } +func DayFromBlockName(blk string) (time.Time, error) { + var ( + year, month, day int + ) + n, err := fmt.Sscanf(blk, dateFormat, &year, &month, &day) + if err != nil { + return time.Time{}, fmt.Errorf("unable to read timestamp from block name: %w", err) + } + if n != 3 { + return time.Time{}, fmt.Errorf("unexpected number of date atoms parsed: %d != 3", n) + } + + return time.Date(year, time.Month(month), day, 0, 0, 0, 0, time.UTC), nil +} + func BlockNameForDay(t time.Time) (string, error) { if t.Location() != time.UTC { return "", fmt.Errorf("block start time %s must be in UTC", t) diff --git a/schema/schema.go b/schema/schema.go index 2ba16c9..08a5536 100644 --- a/schema/schema.go +++ b/schema/schema.go @@ -16,6 +16,8 @@ import ( const ( LabelColumnPrefix = "___cf_meta_label_" + LabelIndexColumn = "___cf_meta_index" + LabelHashColumn = "___cf_meta_hash" ChunksColumn0 = "___cf_meta_chunk_0" ChunksColumn1 = "___cf_meta_chunk_1" ChunksColumn2 = "___cf_meta_chunk_2" @@ -26,6 +28,32 @@ const ( ChunkColumnsPerDay = 3 ) +const ( + // V0 blocks contain a map[__name__] ~ columns for that series + V0 = 0 + // V1 contains a column in the label parquet file that contains an encoded list of indexes that correspond + // to columns that the row has populated. This will lift the constraint on needing to have + // a matcher on the __name__ label present, as we can compute the necessary column projection + // dynamically from the matching rows. + V1 = 1 + // V2 contains a column in the chunks parquet file for the hash of the labels of the timeseries, this makes + // it possible to project labels and still join series horizontally. We write this into the chunks parquet + // file because its essentially random numbers that would bloat the labels file too much + V2 = 2 +) + +func ChunkColumnName(i int) (string, bool) { + switch i { + case 0: + return ChunksColumn0, true + case 1: + return ChunksColumn1, true + case 2: + return ChunksColumn2, true + } + return "", false +} + func LabelNameToColumn(lbl string) string { return fmt.Sprintf("%s%s", LabelColumnPrefix, lbl) } @@ -34,9 +62,22 @@ func ColumnToLabelName(col string) string { return strings.TrimPrefix(col, LabelColumnPrefix) } +func ChunkColumnIndex(m Meta, t time.Time) (int, bool) { + mints := time.UnixMilli(m.Mint) + + colIdx := 0 + for cur := mints.Add(ChunkColumnLength); !t.Before(cur); cur = cur.Add(ChunkColumnLength) { + colIdx++ + } + return min(colIdx, ChunkColumnsPerDay-1), true +} + func BuildSchemaFromLabels(lbls []string) *parquet.Schema { g := make(parquet.Group) + g[LabelIndexColumn] = parquet.Encoded(parquet.Leaf(parquet.ByteArrayType), &parquet.DeltaLengthByteArray) + g[LabelHashColumn] = parquet.Encoded(parquet.Leaf(parquet.Int64Type), &parquet.Plain) + for _, lbl := range lbls { g[LabelNameToColumn(lbl)] = parquet.Optional(parquet.Encoded(parquet.String(), &parquet.RLEDictionary)) } @@ -53,27 +94,14 @@ func WithCompression(s *parquet.Schema) *parquet.Schema { for _, c := range s.Columns() { lc, _ := s.Lookup(c...) - g[lc.Path[0]] = parquet.Compressed(lc.Node, &zstd.Codec{Level: zstd.SpeedBetterCompression}) + g[lc.Path[0]] = parquet.Compressed(lc.Node, &zstd.Codec{Level: zstd.SpeedBetterCompression, Concurrency: 4}) } - return parquet.NewSchema("uncompressed", g) -} - -func Projection(schema *parquet.Schema, projections []string) *parquet.Schema { - g := make(parquet.Group) - - for i := range projections { - lc, ok := schema.Lookup(projections[i]) - if !ok { - continue - } - g[projections[i]] = lc.Node - } - return parquet.NewSchema("projection", g) + return parquet.NewSchema("compressed", g) } var ( - ChunkColumns = []string{ChunksColumn0, ChunksColumn1, ChunksColumn2} + ChunkColumns = []string{LabelHashColumn, ChunksColumn0, ChunksColumn1, ChunksColumn2} ) func ChunkProjection(s *parquet.Schema) *parquet.Schema { @@ -105,22 +133,28 @@ func LabelsProjection(s *parquet.Schema) *parquet.Schema { return parquet.NewSchema("labels-projection", g) } -func Joined(l, r *parquet.Schema) *parquet.Schema { +func RemoveNullColumns(p *parquet.File) *parquet.Schema { g := make(parquet.Group) - for _, c := range l.Columns() { - lc, ok := l.Lookup(c...) - if !ok { + s := p.Schema() + cidxs := p.ColumnIndexes() + nrg := len(p.RowGroups()) + + for i, c := range s.Columns() { + nps := make([]bool, 0) + for j := range nrg * len(s.Columns()) { + if j%len(s.Columns()) == i { + nps = append(nps, cidxs[j].NullPages...) + } + } + if !slices.ContainsFunc(nps, func(np bool) bool { return np == false }) { continue } - g[c[0]] = lc.Node - } - for _, c := range r.Columns() { - lc, ok := r.Lookup(c...) + lc, ok := s.Lookup(c...) if !ok { continue } g[c[0]] = lc.Node } - return parquet.NewSchema("joined", g) + return parquet.NewSchema("remove-nulls-projection", g) } diff --git a/schema/schema_test.go b/schema/schema_test.go new file mode 100644 index 0000000..d5b07a4 --- /dev/null +++ b/schema/schema_test.go @@ -0,0 +1,30 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package schema + +import ( + "testing" + "time" +) + +func TestDataColumnIndex(t *testing.T) { + m := Meta{Mint: 0} + for _, tt := range []struct { + d time.Duration + expect int + }{ + {d: time.Hour, expect: 0}, + {d: 8 * time.Hour, expect: 1}, + {d: 11 * time.Hour, expect: 1}, + {d: 18 * time.Hour, expect: 2}, + {d: 24 * time.Hour, expect: 2}, + } { + t.Run("", func(ttt *testing.T) { + if got, _ := ChunkColumnIndex(m, time.UnixMilli(m.Mint).Add(tt.d)); got != tt.expect { + ttt.Fatalf("unexpected chunk column index %d, expected %d", got, tt.expect) + } + }) + } +} diff --git a/search/constraint.go b/search/constraint.go index 43c00f3..1d5cff2 100644 --- a/search/constraint.go +++ b/search/constraint.go @@ -5,41 +5,135 @@ package search import ( + "bytes" + "context" "fmt" - "io" - "slices" "sort" + "unsafe" "github.com/parquet-go/parquet-go" "github.com/prometheus/prometheus/model/labels" + + "github.com/cloudflare/parquet-tsdb-poc/schema" ) +type Constraint interface { + fmt.Stringer + + // filter returns a set of non-overlapping increasing row indexes that may satisfy the constraint. + filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []rowRange) ([]rowRange, error) + // init initializes the constraint with respect to the file schema and projections. + init(s *parquet.Schema) error + // path is the path for the column that is constrained + path() string +} + +func matchersToConstraint(matchers ...*labels.Matcher) ([]Constraint, error) { + r := make([]Constraint, 0, len(matchers)) + for _, matcher := range matchers { + switch matcher.Type { + case labels.MatchEqual: + r = append(r, Equal(schema.LabelNameToColumn(matcher.Name), parquet.ValueOf(matcher.Value))) + case labels.MatchNotEqual: + r = append(r, Not(Equal(schema.LabelNameToColumn(matcher.Name), parquet.ValueOf(matcher.Value)))) + case labels.MatchRegexp: + res, err := labels.NewFastRegexMatcher(matcher.Value) + if err != nil { + return nil, err + } + set := res.SetMatches() + if len(set) == 1 { + r = append(r, Equal(schema.LabelNameToColumn(matcher.Name), parquet.ValueOf(set[0]))) + } else { + r = append(r, Regex(schema.LabelNameToColumn(matcher.Name), res)) + } + case labels.MatchNotRegexp: + res, err := labels.NewFastRegexMatcher(matcher.Value) + if err != nil { + return nil, err + } + set := res.SetMatches() + if len(set) == 1 { + r = append(r, Not(Equal(schema.LabelNameToColumn(matcher.Name), parquet.ValueOf(set[0])))) + } else { + r = append(r, Not(Regex(schema.LabelNameToColumn(matcher.Name), res))) + } + default: + return nil, fmt.Errorf("unsupported matcher type %s", matcher.Type) + } + } + return r, nil +} + +func initialize(s *parquet.Schema, cs ...Constraint) error { + for i := range cs { + if err := cs[i].init(s); err != nil { + return fmt.Errorf("unable to initialize constraint %d: %w", i, err) + } + } + return nil +} + +func filter(ctx context.Context, rg parquet.RowGroup, cs ...Constraint) ([]rowRange, error) { + // Constraints for sorting columns are cheaper to evaluate, so we sort them first. + sc := rg.SortingColumns() + + var n int + for i := range sc { + if n == len(cs) { + break + } + for j := range cs { + if cs[j].path() == sc[i].Path()[0] { + cs[n], cs[j] = cs[j], cs[n] + n++ + } + } + } + var err error + rr := []rowRange{{from: int64(0), count: rg.NumRows()}} + for i := range cs { + isPrimary := len(sc) > 0 && cs[i].path() == sc[0].Path()[0] + rr, err = cs[i].filter(ctx, rg, isPrimary, rr) + if err != nil { + return nil, fmt.Errorf("unable to filter with constraint %d: %w", i, err) + } + } + return rr, nil +} + type equalConstraint struct { - path string + pth string val parquet.Value - // set during init - col int comp func(l, r parquet.Value) int - - inspectPages bool } -var _ Constraint = &equalConstraint{} +func (ec *equalConstraint) String() string { + return fmt.Sprintf("equal(%q,%q)", ec.pth, ec.val) +} func Equal(path string, value parquet.Value) Constraint { - return &equalConstraint{path: path, val: value} + return &equalConstraint{pth: path, val: value} } -func EqualWithPageCheck(path string, value parquet.Value) Constraint { - return &equalConstraint{path: path, val: value, inspectPages: true} -} +func (ec *equalConstraint) filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []rowRange) ([]rowRange, error) { + if len(rr) == 0 { + return nil, nil + } + method := methodFromContext(ctx) -func (ec *equalConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { - col, ok := rg.Schema().Lookup(ec.path) + from, to := rr[0].from, rr[len(rr)-1].from+rr[len(rr)-1].count + + col, ok := rg.Schema().Lookup(ec.path()) if !ok { - return nil, nil + // If match empty, return rr (filter nothing) + // otherwise return empty + if ec.matches(parquet.ValueOf("")) { + return rr, nil + } + return []rowRange{}, nil } cc := rg.ColumnChunks()[col.ColumnIndex] @@ -49,11 +143,8 @@ func (ec *equalConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { return nil, nil } - var pgs parquet.Pages - if ec.inspectPages { - pgs = cc.Pages() - defer pgs.Close() - } + pgs := cc.Pages() + defer func() { _ = pgs.Close() }() oidx, err := cc.OffsetIndex() if err != nil { @@ -63,85 +154,123 @@ func (ec *equalConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { if err != nil { return nil, fmt.Errorf("unable to read column index: %w", err) } - var buf []parquet.Value - - res := make([]rowRange, 0) - for i := 0; i < cidx.NumPages(); i++ { + var ( + symbols = new(symbolTable) + res = make([]rowRange, 0) + ) + for i := range cidx.NumPages() { + // If page does not intersect from, to; we can immediately discard it + pfrom := oidx.FirstRowIndex(i) + pcount := rg.NumRows() - pfrom + if i < oidx.NumPages()-1 { + pcount = oidx.FirstRowIndex(i+1) - pfrom + } + pto := pfrom + pcount + if pfrom > to { + break + } + if pto < from { + continue + } + // Page intersects [from, to] but we might be able to discard it with statistics if cidx.NullPage(i) { - pagesDiscarded.WithLabelValues(ec.path).Inc() + if ec.matches(parquet.ValueOf("")) { + res = append(res, rowRange{pfrom, pcount}) + } continue } + + // If we are not matching the empty string ( which would be satisfied by Null too ), we can + // use page statistics to skip rows minv, maxv := cidx.MinValue(i), cidx.MaxValue(i) - if !ec.val.IsNull() && !maxv.IsNull() && ec.comp(ec.val, maxv) > 0 { - pagesDiscarded.WithLabelValues(ec.path).Inc() + if !ec.matches(parquet.ValueOf("")) && !maxv.IsNull() && ec.comp(ec.val, maxv) > 0 { if cidx.IsDescending() { break } continue } - if !ec.val.IsNull() && !minv.IsNull() && ec.comp(ec.val, minv) < 0 { - pagesDiscarded.WithLabelValues(ec.path).Inc() + if !ec.matches(parquet.ValueOf("")) && !minv.IsNull() && ec.comp(ec.val, minv) < 0 { if cidx.IsAscending() { break } continue } - from := oidx.FirstRowIndex(i) - - // TODO: this would also work for descending columns, but for now - // this is only used on __name__ which is ascending anyway so we dont - // bother implementing it for descending or unordered columns. - // TODO: for unordered columns we could inspect the dictionary here - // and for descending columns we just have to flip the inequalities - if ec.inspectPages && cidx.IsAscending() { - if err := pgs.SeekToRow(from); err != nil { - return nil, fmt.Errorf("unable to seek to row: %w", err) - } - pg, err := pgs.ReadPage() - if err != nil { - return nil, fmt.Errorf("unable to read page: %w", err) + // We cannot discard the page through statistics but we might need to read it to see if it has the value + if err := pgs.SeekToRow(pfrom); err != nil { + return nil, fmt.Errorf("unable to seek to row: %w", err) + } + pg, err := pgs.ReadPage() + if err != nil { + return nil, fmt.Errorf("unable to read page: %w", err) + } + + symbols.Reset(pg) + + pagesScanned.WithLabelValues(ec.path(), scanEqual, method).Add(1) + + // The page has the value, we need to find the matching row ranges + n := int(pg.NumRows()) + bl := int(max(pfrom, from) - pfrom) + br := n - int(pto-min(pto, to)) + var l, r int + switch { + case cidx.IsAscending() && primary: + l = sort.Search(n, func(i int) bool { return ec.comp(ec.val, symbols.Get(i)) <= 0 }) + r = sort.Search(n, func(i int) bool { return ec.comp(ec.val, symbols.Get(i)) < 0 }) + + if lv, rv := max(bl, l), min(br, r); rv > lv { + res = append(res, rowRange{pfrom + int64(lv), int64(rv - lv)}) } - pagesRead.WithLabelValues(ec.path).Inc() - vals, rows := pg.Values(), pg.NumRows() - buf = slices.Grow(buf, int(rows))[:rows] - n, err := vals.ReadValues(buf) - if err != nil && err != io.EOF { - return nil, fmt.Errorf("unable to read page values") + default: + off, count := bl, 0 + for j := bl; j < br; j++ { + if !ec.matches(symbols.Get(j)) { + if count != 0 { + res = append(res, rowRange{pfrom + int64(off), int64(count)}) + } + off, count = j, 0 + } else { + if count == 0 { + off = j + } + count++ + } } - l := sort.Search(n, func(i int) bool { return ec.comp(ec.val, buf[i]) <= 0 }) - r := sort.Search(n, func(i int) bool { return ec.comp(ec.val, buf[i]) < 0 }) - res = append(res, rowRange{from: from + int64(l), count: int64(r - l)}) - } else { - count := rg.NumRows() - from - if i < oidx.NumPages()-1 { - count = oidx.FirstRowIndex(i+1) - from + if count != 0 { + res = append(res, rowRange{pfrom + int64(off), int64(count)}) } - res = append(res, rowRange{from: from, count: count}) } } if len(res) == 0 { return nil, nil } - return simplify(res), nil -} - -func (ec *equalConstraint) accept(r parquet.Row) bool { - return ec.comp(r[ec.col], ec.val) == 0 + return intersectRowRanges(simplify(res), rr), nil } func (ec *equalConstraint) init(s *parquet.Schema) error { - c, ok := s.Lookup(ec.path) + c, ok := s.Lookup(ec.path()) if !ok { - return fmt.Errorf("schema: must contain path: %s", ec.path) + return nil } - if c.Node.Type().Kind() != ec.val.Kind() { - return fmt.Errorf("schema: cannot search value of kind %s in column of kind %s", ec.val.Kind(), c.Node.Type().Kind()) + stringKind := parquet.String().Type().Kind() + if ec.val.Kind() != stringKind { + return fmt.Errorf("schema: can only search string kind, got: %s", ec.val.Kind()) + } + if c.Node.Type().Kind() != stringKind { + return fmt.Errorf("schema: cannot search value of kind %s in column of kind %s", stringKind, c.Node.Type().Kind()) } ec.comp = c.Node.Type().Compare - ec.col = c.ColumnIndex return nil } +func (ec *equalConstraint) path() string { + return ec.pth +} + +func (ec *equalConstraint) matches(v parquet.Value) bool { + return bytes.Equal(v.ByteArray(), ec.val.ByteArray()) +} + func (ec *equalConstraint) skipByBloomfilter(cc parquet.ColumnChunk) (bool, error) { bf := cc.BloomFilter() if bf == nil { @@ -154,226 +283,173 @@ func (ec *equalConstraint) skipByBloomfilter(cc parquet.ColumnChunk) (bool, erro return !ok, nil } -type andConstraint struct { - cs []Constraint +func Regex(path string, r *labels.FastRegexMatcher) Constraint { + return ®exConstraint{pth: path, cache: make(map[parquet.Value]bool), r: r} } -var _ Constraint = &andConstraint{} +type regexConstraint struct { + pth string + cache map[parquet.Value]bool -func And(cs ...Constraint) Constraint { - return &andConstraint{cs: cs} + r *labels.FastRegexMatcher } -func (ac *andConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { - var res []rowRange - for i := range ac.cs { - rrs, err := ac.cs[i].rowRanges(rg) - if err != nil { - return nil, fmt.Errorf("unable to get lhs row ranges: %w", err) - } - if i == 0 { - res = rrs - } else { - res = intersectRowRanges(res, rrs) - } - } - return simplify(res), nil +func (rc *regexConstraint) String() string { + return fmt.Sprintf("regex(%v,%v)", rc.pth, rc.r.GetRegexString()) } -func (ac *andConstraint) accept(r parquet.Row) bool { - for i := range ac.cs { - if !ac.cs[i].accept(r) { - return false - } +func (rc *regexConstraint) filter(ctx context.Context, rg parquet.RowGroup, _ bool, rr []rowRange) ([]rowRange, error) { + if len(rr) == 0 { + return nil, nil } - return true -} + method := methodFromContext(ctx) + + from, to := rr[0].from, rr[len(rr)-1].from+rr[len(rr)-1].count -func (ac *andConstraint) init(s *parquet.Schema) error { - for i := range ac.cs { - if err := ac.cs[i].init(s); err != nil { - return fmt.Errorf("unable to init constraint %d: %w", i, err) + col, ok := rg.Schema().Lookup(rc.path()) + if !ok { + // If match empty, return rr (filter nothing) + // otherwise return empty + if rc.matches(parquet.ValueOf("")) { + return rr, nil } + return []rowRange{}, nil } - return nil -} - -type notConstraint struct { - cs Constraint -} - -var _ Constraint = ¬Constraint{} - -func Not(cs Constraint) Constraint { - return ¬Constraint{cs: cs} -} - -func (nc *notConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { - return []rowRange{{from: 0, count: int64(rg.NumRows())}}, nil -} - -func (nc *notConstraint) accept(r parquet.Row) bool { - return !nc.cs.accept(r) -} - -func (nc *notConstraint) init(s *parquet.Schema) error { - return nc.cs.init(s) -} - -type nullConstraint struct { -} - -var _ Constraint = &nullConstraint{} - -func Null() Constraint { - return &nullConstraint{} -} - -func (null *nullConstraint) rowRanges(parquet.RowGroup) ([]rowRange, error) { - return nil, nil -} - -func (null *nullConstraint) accept(parquet.Row) bool { - return false -} - -func (null *nullConstraint) init(_ *parquet.Schema) error { - return nil -} - -type orConstraint struct { - cs []Constraint -} - -var _ Constraint = &orConstraint{} + cc := rg.ColumnChunks()[col.ColumnIndex] -func Or(cs ...Constraint) Constraint { - return &orConstraint{cs: cs} -} + pgs := cc.Pages() + defer func() { _ = pgs.Close() }() -func Set(path string, values []parquet.Value) Constraint { - equals := []Constraint{} - for _, val := range values { - equals = append(equals, Equal(path, val)) + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("unable to read offset index: %w", err) } - return &orConstraint{cs: equals} -} + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("unable to read column index: %w", err) + } + var ( + symbols = new(symbolTable) + res = make([]rowRange, 0) + ) + for i := range cidx.NumPages() { + // If page does not intersect from, to; we can immediately discard it + pfrom := oidx.FirstRowIndex(i) + pcount := rg.NumRows() - pfrom + if i < oidx.NumPages()-1 { + pcount = oidx.FirstRowIndex(i+1) - pfrom + } + pto := pfrom + pcount + if pfrom > to { + break + } + if pto < from { + continue + } + // Page intersects [from, to] but we might be able to discard it with statistics + if cidx.NullPage(i) { + if rc.matches(parquet.ValueOf("")) { + res = append(res, rowRange{pfrom, pcount}) + } + continue + } + // TODO: use setmatches / prefix for statistics -func (oc *orConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { - var res []rowRange - for i := range oc.cs { - rrs, err := oc.cs[i].rowRanges(rg) + // We cannot discard the page through statistics but we might need to read it to see if it has the value + if err := pgs.SeekToRow(pfrom); err != nil { + return nil, fmt.Errorf("unable to seek to row: %w", err) + } + pg, err := pgs.ReadPage() if err != nil { - return nil, fmt.Errorf("unable to get lhs row ranges: %w", err) + return nil, fmt.Errorf("unable to read page: %w", err) } - res = append(res, rrs...) - } - return simplify(res), nil -} -func (oc *orConstraint) accept(r parquet.Row) bool { - for i := range oc.cs { - if oc.cs[i].accept(r) { - return true + symbols.Reset(pg) + + pagesScanned.WithLabelValues(rc.path(), scanRegex, method).Add(1) + + // The page has the value, we need to find the matching row ranges + n := int(pg.NumRows()) + bl := int(max(pfrom, from) - pfrom) + br := n - int(pto-min(pto, to)) + off, count := bl, 0 + for j := bl; j < br; j++ { + if !rc.matches(symbols.Get(j)) { + if count != 0 { + res = append(res, rowRange{pfrom + int64(off), int64(count)}) + } + off, count = j, 0 + } else { + if count == 0 { + off = j + } + count++ + } } + if count != 0 { + res = append(res, rowRange{pfrom + int64(off), int64(count)}) + } + } + if len(res) == 0 { + return nil, nil } - return false + return intersectRowRanges(simplify(res), rr), nil } -func (oc *orConstraint) init(s *parquet.Schema) error { - for i := range oc.cs { - if err := oc.cs[i].init(s); err != nil { - return fmt.Errorf("unable to init constraint %d: %w", i, err) - } +func (rc *regexConstraint) init(s *parquet.Schema) error { + c, ok := s.Lookup(rc.path()) + if !ok { + return nil + } + if stringKind := parquet.String().Type().Kind(); c.Node.Type().Kind() != stringKind { + return fmt.Errorf("schema: cannot search value of kind %s in column of kind %s", stringKind, c.Node.Type().Kind()) } + rc.cache = make(map[parquet.Value]bool) return nil } -type regexConstraint struct { - path string - matcher *labels.FastRegexMatcher - - // set during init - col int - cache map[string]bool +func (rc *regexConstraint) path() string { + return rc.pth } -var _ Constraint = ®exConstraint{} - -func Regex(path string, regex string) (Constraint, error) { - matcher, err := labels.NewFastRegexMatcher(regex) - if err != nil { - return nil, err - } - - if len(matcher.SetMatches()) > 0 { - vals := []parquet.Value{} - for _, match := range matcher.SetMatches() { - vals = append(vals, parquet.ValueOf(match)) - } - return Set(path, vals), nil +func (rc *regexConstraint) matches(v parquet.Value) bool { + accept, seen := rc.cache[v] + if !seen { + accept = rc.r.MatchString(yoloString(v.ByteArray())) + rc.cache[v] = accept } - - return ®exConstraint{ - matcher: matcher, - path: path, - }, nil + return accept } -func (ec *regexConstraint) rowRanges(rg parquet.RowGroup) ([]rowRange, error) { - col, ok := rg.Schema().Lookup(ec.path) - if !ok { - return nil, nil - } - cc := rg.ColumnChunks()[col.ColumnIndex] - - oidx, err := cc.OffsetIndex() - if err != nil { - return nil, fmt.Errorf("unable to read offset index: %w", err) - } - cidx, err := cc.ColumnIndex() - if err != nil { - return nil, fmt.Errorf("unable to read column index: %w", err) - } +func yoloString(buf []byte) string { + return *((*string)(unsafe.Pointer(&buf))) +} - res := []rowRange{} - for i := 0; i < cidx.NumPages(); i++ { - if cidx.NullPage(i) { - pagesDiscarded.WithLabelValues(ec.path).Inc() - continue - } - from := oidx.FirstRowIndex(i) - count := rg.NumRows() - from - if i < oidx.NumPages()-1 { - count = oidx.FirstRowIndex(i+1) - from - } - res = append(res, rowRange{from: from, count: count}) - } - return simplify(res), nil +func Not(c Constraint) Constraint { + return ¬Constraint{c: c} } -func (ec *regexConstraint) accept(r parquet.Row) bool { - val := r[ec.col].String() - accept, seen := ec.cache[val] - if !seen { - accept = ec.matcher.MatchString(val) - ec.cache[val] = accept - } - return accept +type notConstraint struct { + c Constraint } -func (ec *regexConstraint) init(s *parquet.Schema) error { - c, ok := s.Lookup(ec.path) - if !ok { - return fmt.Errorf("schema: must contain path: %s", ec.path) - } +func (nc *notConstraint) String() string { + return fmt.Sprintf("not(%v)", nc.c.String()) +} - if c.Node.Type().Kind() != parquet.ByteArray { - return fmt.Errorf("schema: expected string, cannot assert regex of type %s", c.Node.Type().String()) +func (nc *notConstraint) filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []rowRange) ([]rowRange, error) { + base, err := nc.c.filter(ctx, rg, primary, rr) + if err != nil { + return nil, fmt.Errorf("unable to compute child constraint: %w", err) } + // no need to intersect since its already subset of rr + return complementRowRanges(base, rr), nil +} - ec.cache = map[string]bool{} - ec.col = c.ColumnIndex +func (nc *notConstraint) init(s *parquet.Schema) error { + return nc.c.init(s) +} - return nil +func (nc *notConstraint) path() string { + return nc.c.path() } diff --git a/search/constraint_test.go b/search/constraint_test.go new file mode 100644 index 0000000..2c29212 --- /dev/null +++ b/search/constraint_test.go @@ -0,0 +1,293 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package search + +import ( + "bytes" + "slices" + "testing" + + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/model/labels" +) + +func buildFile[T any](t testing.TB, rows []T) *parquet.File { + buf := bytes.NewBuffer(nil) + w := parquet.NewGenericWriter[T](buf, parquet.PageBufferSize(12), parquet.WriteBufferSize(0)) + for _, row := range rows { + if _, err := w.Write([]T{row}); err != nil { + t.Fatal(err) + } + } + if err := w.Close(); err != nil { + t.Fatal(err) + } + reader := bytes.NewReader(buf.Bytes()) + file, err := parquet.OpenFile(reader, reader.Size()) + if err != nil { + t.Fatal(err) + } + return file +} + +func mustNewFastRegexMatcher(t *testing.T, s string) *labels.FastRegexMatcher { + res, err := labels.NewFastRegexMatcher(s) + if err != nil { + t.Fatalf("unable to build fast regex matcher: %s", err) + } + return res +} + +func TestFilter(t *testing.T) { + type expectation struct { + constraints []Constraint + expect []rowRange + } + type testcase[T any] struct { + rows []T + expectations []expectation + } + + t.Run("", func(t *testing.T) { + type s struct { + A string `parquet:",optional,dict"` + B string `parquet:",optional,dict"` + C string `parquet:",optional,dict"` + } + for _, tc := range []testcase[s]{ + { + rows: []s{ + {A: "1", B: "2", C: "a"}, + {A: "3", B: "4", C: "b"}, + {A: "7", B: "12", C: "c"}, + {A: "9", B: "22", C: "d"}, + {A: "0", B: "1", C: "e"}, + {A: "7", B: "1", C: "f"}, + {A: "7", B: "1", C: "g"}, + {A: "0", B: "1", C: "h"}, + }, + expectations: []expectation{ + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("7")), + Equal("C", parquet.ValueOf("g")), + }, + expect: []rowRange{ + {from: 6, count: 1}, + }, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("7")), + }, + expect: []rowRange{ + {from: 2, count: 1}, + {from: 5, count: 2}, + }, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("7")), Not(Equal("B", parquet.ValueOf("1"))), + }, + expect: []rowRange{ + {from: 2, count: 1}, + }, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("7")), Not(Equal("C", parquet.ValueOf("c"))), + }, + expect: []rowRange{ + {from: 5, count: 2}, + }, + }, + { + constraints: []Constraint{ + Not(Equal("A", parquet.ValueOf("227"))), + }, + expect: []rowRange{ + {from: 0, count: 8}, + }, + }, + { + constraints: []Constraint{ + Regex("C", mustNewFastRegexMatcher(t, "a|c|d")), + }, + expect: []rowRange{ + {from: 0, count: 1}, + {from: 2, count: 2}, + }, + }, + }, + }, + { + rows: []s{ + {A: "1", B: "2"}, + {A: "1", B: "3"}, + {A: "1", B: "4"}, + {A: "1", B: "4"}, + {A: "1", B: "5"}, + {A: "1", B: "5"}, + {A: "2", B: "5"}, + {A: "2", B: "5"}, + {A: "2", B: "5"}, + {A: "3", B: "5"}, + {A: "3", B: "6"}, + {A: "3", B: "2"}, + {A: "4", B: "8", C: "foo"}, + }, + expectations: []expectation{ + { + constraints: []Constraint{ + Regex("C", mustNewFastRegexMatcher(t, "")), + }, + expect: []rowRange{ + {from: 0, count: 12}, + }, + }, + { + constraints: []Constraint{ + Equal("C", parquet.ValueOf("")), + }, + expect: []rowRange{ + {from: 0, count: 12}, + }, + }, + { + constraints: []Constraint{ + Not(Equal("A", parquet.ValueOf("3"))), + }, + expect: []rowRange{ + {from: 0, count: 9}, + {from: 12, count: 1}, + }, + }, + { + constraints: []Constraint{ + Not(Equal("A", parquet.ValueOf("3"))), + Equal("B", parquet.ValueOf("5")), + }, + expect: []rowRange{ + {from: 4, count: 5}, + }, + }, + { + constraints: []Constraint{ + Not(Equal("A", parquet.ValueOf("3"))), + Not(Equal("A", parquet.ValueOf("1"))), + }, + expect: []rowRange{ + {from: 6, count: 3}, + {from: 12, count: 1}, + }, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("2")), + Not(Equal("B", parquet.ValueOf("5"))), + }, + expect: []rowRange{}, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("2")), + Not(Equal("B", parquet.ValueOf("5"))), + }, + expect: []rowRange{}, + }, + { + constraints: []Constraint{ + Equal("A", parquet.ValueOf("3")), + Not(Equal("B", parquet.ValueOf("2"))), + }, + expect: []rowRange{ + {from: 9, count: 2}, + }, + }, + }, + }, + { + rows: []s{ + {A: "1", B: "1"}, + {A: "1", B: "2"}, + {A: "2", B: "1"}, + {A: "2", B: "2"}, + {A: "1", B: "1"}, + {A: "1", B: "2"}, + {A: "2", B: "1"}, + {A: "2", B: "2"}, + }, + expectations: []expectation{ + { + constraints: []Constraint{ + Not(Equal("A", parquet.ValueOf("1"))), + Not(Equal("B", parquet.ValueOf("2"))), + }, + expect: []rowRange{ + {from: 2, count: 1}, + {from: 6, count: 1}, + }, + }, + }, + }, + { + rows: []s{ + {C: "foo"}, + {C: "bar"}, + {C: "foo"}, + {C: "buz"}, + }, + expectations: []expectation{ + { + constraints: []Constraint{ + Regex("C", mustNewFastRegexMatcher(t, "f.*")), + }, + expect: []rowRange{ + {from: 0, count: 1}, + {from: 2, count: 1}, + }, + }, + { + constraints: []Constraint{ + Regex("C", mustNewFastRegexMatcher(t, "b.*")), + }, + expect: []rowRange{ + {from: 1, count: 1}, + {from: 3, count: 1}, + }, + }, + { + constraints: []Constraint{ + Regex("C", mustNewFastRegexMatcher(t, "f.*|b.*")), + }, + expect: []rowRange{ + {from: 0, count: 4}, + }, + }, + }, + }, + } { + + ctx := contextWithMethod(t.Context(), "test") + sfile := buildFile(t, tc.rows) + for _, expectation := range tc.expectations { + t.Run("", func(t *testing.T) { + if err := initialize(sfile.Schema(), expectation.constraints...); err != nil { + t.Fatal(err) + } + for _, rg := range sfile.RowGroups() { + rr, err := filter(ctx, rg, expectation.constraints...) + if err != nil { + t.Fatal(err) + } + if !slices.Equal(rr, expectation.expect) { + t.Fatalf("expected %+v, got %+v", expectation.expect, rr) + } + } + }) + } + } + }) +} diff --git a/search/materialize.go b/search/materialize.go new file mode 100644 index 0000000..97b8f48 --- /dev/null +++ b/search/materialize.go @@ -0,0 +1,1017 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package search + +import ( + "context" + "encoding/binary" + "errors" + "fmt" + "io" + "iter" + "maps" + "math" + + "slices" + "sync" + "time" + + "github.com/parquet-go/parquet-go" + "go.opentelemetry.io/otel/attribute" + "golang.org/x/sync/errgroup" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/util/annotations" + + "github.com/cloudflare/parquet-tsdb-poc/internal/encoding" + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" + "github.com/cloudflare/parquet-tsdb-poc/schema" +) + +// materializeSeries reconstructs the ChunkSeries that belong to the specified row ranges (rr). +// It uses the row group index (rgi) and time bounds (mint, maxt) to filter and decode the series. +func materializeSeries( + ctx context.Context, + m SelectReadMeta, + rgi int, + mint, + maxt int64, + hints *storage.SelectHints, + rr []rowRange, +) ([]SeriesChunks, annotations.Annotations, error) { + var annos annotations.Annotations + + if limit := hints.Limit; limit > 0 { + annos.Add(warnings.ErrorTruncatedResponse) + // Series are all different so we can actually limit the rowranges themselves + // This would not work for Label APIs but for series its ok. + rr = limitRowRanges(int64(limit), rr) + } + + if err := checkRowQuota(m.RowCountQuota, rr); err != nil { + return nil, annos, err + } + res := make([]SeriesChunks, totalRows(rr)) + + g, ctx := errgroup.WithContext(ctx) + + g.Go(func() error { + sLbls, err := materializeLabels(ctx, m, rgi, rr) + if err != nil { + return fmt.Errorf("error materializing labels: %w", err) + } + + for i, s := range sLbls { + m.ExternalLabels.Range(func(lbl labels.Label) { s.Set(lbl.Name, lbl.Value) }) + s.Del(m.ReplicaLabelNames...) + + lbls := s.Labels() + h := lbls.Hash() + + res[i].Lset = lbls + res[i].LsetHash = h + } + return nil + }) + + g.Go(func() error { + if hints.Func == "series" { + return nil + } + chks, err := materializeChunks(ctx, m, rgi, mint, maxt, rr) + if err != nil { + return fmt.Errorf("unable to materialize chunks: %w", err) + } + + for i, c := range chks { + for j := range c { + res[i].Chunks = append(res[i].Chunks, c[j]) + } + } + return nil + }) + + if err := g.Wait(); err != nil { + return res, annos, fmt.Errorf("unable to materialize series: %w", err) + } + return res, annos, nil +} + +func materializeLabels(ctx context.Context, m SelectReadMeta, rgi int, rr []rowRange) ([]labels.Builder, error) { + switch v := m.Meta.Version; v { + case schema.V0: + return materializeLabelsV0(ctx, m, rgi, rr) + case schema.V1, schema.V2: + return materializeLabelsV1(ctx, m, rgi, rr) + default: + return nil, fmt.Errorf("unable to materialize labels for block of version %q", v) + } +} + +// v0 blocks have a map to resolve column names for a metric +func materializeLabelsV0(ctx context.Context, m SelectReadMeta, rgi int, rr []rowRange) ([]labels.Builder, error) { + rowCount := totalRows(rr) + + metricNameColumn := schema.LabelNameToColumn(labels.MetricName) + + lc, ok := m.LabelPfile.Schema().Lookup(metricNameColumn) + if !ok { + return nil, fmt.Errorf("unable to to find column %q", metricNameColumn) + } + + rg := m.LabelPfile.RowGroups()[rgi] + + cc := rg.ColumnChunks()[lc.ColumnIndex] + metricNames, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return nil, fmt.Errorf("unable to materialize metric name column: %w", err) + } + + seen := make(map[string]struct{}) + colsMap := make(map[int]*[]parquet.Value, 10) + + v := make([]parquet.Value, 0, rowCount) + colsMap[lc.ColumnIndex] = &v + for _, nm := range metricNames { + key := yoloString(nm.ByteArray()) + if _, ok := seen[key]; !ok { + cols := m.Meta.ColumnsForName[key] + for _, c := range cols { + lc, ok := m.LabelPfile.Schema().Lookup(c) + if !ok { + continue + } + colsMap[lc.ColumnIndex] = &[]parquet.Value{} + } + seen[key] = struct{}{} + } + } + + g, ctx := errgroup.WithContext(ctx) + for cIdx, v := range colsMap { + g.Go(func() error { + cc := rg.ColumnChunks()[cIdx] + values, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return fmt.Errorf("unable to materialize labels values: %w", err) + } + *v = values + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, err + } + + builders := make([]labels.Builder, rowCount) + for cIdx, values := range colsMap { + colName := m.LabelPfile.Schema().Columns()[cIdx][0] + labelName := schema.ColumnToLabelName(colName) + + for i, value := range *values { + if value.IsNull() { + continue + } + builders[i].Set(labelName, yoloString(value.ByteArray())) + } + } + return builders, nil +} + +// v1 blocks have a cf_meta_index column that contains an index which columns to resolve for a metric +func materializeLabelsV1(ctx context.Context, m SelectReadMeta, rgi int, rr []rowRange) ([]labels.Builder, error) { + rowCount := totalRows(rr) + + lc, ok := m.LabelPfile.Schema().Lookup(schema.LabelIndexColumn) + if !ok { + return nil, fmt.Errorf("unable to to find label index column %q", schema.LabelIndexColumn) + } + + rg := m.LabelPfile.RowGroups()[rgi] + + cc := rg.ColumnChunks()[lc.ColumnIndex] + colsIdxs, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return nil, fmt.Errorf("unable to materialize label index column: %w", err) + } + + seen := make(map[string]struct{}) + colsMap := make(map[int]*[]parquet.Value, 10) + for _, colsIdx := range colsIdxs { + key := yoloString(colsIdx.ByteArray()) + if _, ok := seen[key]; !ok { + idxs, err := encoding.DecodeLabelColumnIndex(colsIdx.ByteArray()) + if err != nil { + return nil, fmt.Errorf("unable to decode column index: %w", err) + } + for _, idx := range idxs { + colsMap[idx] = &[]parquet.Value{} + } + seen[key] = struct{}{} + } + } + + g, ctx := errgroup.WithContext(ctx) + for cIdx, v := range colsMap { + g.Go(func() error { + cc := rg.ColumnChunks()[cIdx] + values, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return fmt.Errorf("unable to materialize labels values: %w", err) + } + *v = values + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, err + } + + builders := make([]labels.Builder, rowCount) + for cIdx, values := range colsMap { + colName := m.LabelPfile.Schema().Columns()[cIdx][0] + labelName := schema.ColumnToLabelName(colName) + + for i, value := range *values { + if value.IsNull() { + continue + } + builders[i].Set(labelName, yoloString(value.ByteArray())) + } + } + return builders, nil +} + +func materializeChunks( + ctx context.Context, + m SelectReadMeta, + rgi int, + mint int64, + maxt int64, + rr []rowRange) ([][]chunks.Meta, error) { + rowCount := totalRows(rr) + + minChunkCol, ok := schema.ChunkColumnIndex(m.Meta, time.UnixMilli(mint)) + if !ok { + return nil, errors.New("unable to find min chunk column") + } + maxChunkCol, ok := schema.ChunkColumnIndex(m.Meta, time.UnixMilli(maxt)) + if !ok { + return nil, errors.New("unable to find max chunk column") + } + rg := m.ChunkPfile.RowGroups()[rgi] + + r := make([][]chunks.Meta, rowCount) + + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + for i := minChunkCol; i <= maxChunkCol; i++ { + colName, ok := schema.ChunkColumnName(i) + if !ok { + return nil, fmt.Errorf("unable to find chunk column for column index %d", i) + } + col, ok := rg.Schema().Lookup(colName) + if !ok { + return nil, fmt.Errorf("unable to find chunk column for column name %q", colName) + } + + cc := rg.ColumnChunks()[col.ColumnIndex] + g.Go(func() error { + values, err := materializeChunkColumn( + ctx, + rg, + cc, + rr, + withByteQuota(m.ChunkBytesQuota), + withReaderFromContext(m.ChunkFileReaderFromContext), + withPartitionMaxRangeSize(m.ChunkPagePartitionMaxRange), + withPartitionMaxGapSize(m.ChunkPagePartitionMaxGap), + withPartitionMaxConcurrency(m.ChunkPagePartitionMaxConcurrency), + ) + if err != nil { + return fmt.Errorf("unable to materialize column: %w", err) + } + + for j, chkVal := range values { + chks := make([]chunks.Meta, 0, 12) + bs := chkVal.ByteArray() + for len(bs) != 0 { + enc := chunkenc.Encoding(binary.BigEndian.Uint32(bs[:4])) + bs = bs[4:] + cmint := encoding.ZigZagDecode(binary.BigEndian.Uint64(bs[:8])) + bs = bs[8:] + cmaxt := encoding.ZigZagDecode(binary.BigEndian.Uint64(bs[:8])) + bs = bs[8:] + l := binary.BigEndian.Uint32(bs[:4]) + bs = bs[4:] + chk, err := chunkenc.FromData(enc, bs[:l]) + if err != nil { + return fmt.Errorf("unable to create chunk from data: %w", err) + } + chks = append(chks, chunks.Meta{MinTime: cmint, MaxTime: cmaxt, Chunk: chk}) + bs = bs[l:] + } + + mu.Lock() + r[j] = append(r[j], chks...) + mu.Unlock() + } + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, fmt.Errorf("unable to process chunks: %w", err) + } + + for i := range r { + slices.SortFunc(r[i], func(a, b chunks.Meta) int { return int(a.MinTime - b.MinTime) }) + } + + return r, nil +} + +func materializeLabelNames(ctx context.Context, meta LabelNamesReadMeta, rgi int, rr []rowRange) ([]string, annotations.Annotations, error) { + switch v := meta.Meta.Version; v { + case schema.V0: + return materializeLabelNamesV0(ctx, meta, rgi, rr) + case schema.V1, schema.V2: + return materializeLabelNamesV1(ctx, meta, rgi, rr) + default: + return nil, nil, fmt.Errorf("unable to materialize labels names for block of version %q", v) + } +} + +func materializeLabelNamesV0(ctx context.Context, meta LabelNamesReadMeta, rgi int, rr []rowRange) ([]string, annotations.Annotations, error) { + var annos annotations.Annotations + + metricNameColumn := schema.LabelNameToColumn(labels.MetricName) + + lc, ok := meta.LabelPfile.Schema().Lookup(metricNameColumn) + if !ok { + return nil, annos, fmt.Errorf("unable to to find column %q", metricNameColumn) + } + + rg := meta.LabelPfile.RowGroups()[rgi] + + cc := rg.ColumnChunks()[lc.ColumnIndex] + metricNames, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return nil, annos, fmt.Errorf("unable to materialize metric name column: %w", err) + } + + seen := make(map[string]struct{}) + colIdxs := make(map[int]struct{}) + for _, mn := range metricNames { + key := yoloString(mn.ByteArray()) + if _, ok := seen[key]; !ok { + cols := meta.Meta.ColumnsForName[key] + for _, c := range cols { + lc, ok := meta.LabelPfile.Schema().Lookup(c) + if !ok { + continue + } + colIdxs[lc.ColumnIndex] = struct{}{} + } + } + seen[key] = struct{}{} + } + + cols := meta.LabelPfile.Schema().Columns() + + res := make([]string, 0, len(colIdxs)) + for k := range colIdxs { + res = append(res, schema.ColumnToLabelName(cols[k][0])) + } + return res, annos, nil + +} + +func materializeLabelNamesV1(ctx context.Context, meta LabelNamesReadMeta, rgi int, rr []rowRange) ([]string, annotations.Annotations, error) { + var annos annotations.Annotations + + lc, ok := meta.LabelPfile.Schema().Lookup(schema.LabelIndexColumn) + if !ok { + return nil, annos, fmt.Errorf("unable to to find label index column %q", schema.LabelIndexColumn) + } + + rg := meta.LabelPfile.RowGroups()[rgi] + + cc := rg.ColumnChunks()[lc.ColumnIndex] + colsIdxs, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return nil, annos, fmt.Errorf("unable to materialize label index column: %w", err) + } + + seen := make(map[string]struct{}) + colIdxs := make(map[int]struct{}) + for _, colsIdx := range colsIdxs { + key := yoloString(colsIdx.ByteArray()) + if _, ok := seen[key]; !ok { + idxs, err := encoding.DecodeLabelColumnIndex(colsIdx.ByteArray()) + if err != nil { + return nil, annos, fmt.Errorf("materializer failed to decode column index: %w", err) + } + for _, idx := range idxs { + colIdxs[idx] = struct{}{} + } + seen[key] = struct{}{} + } + } + + cols := meta.LabelPfile.Schema().Columns() + + res := make([]string, 0, len(colIdxs)) + for k := range colIdxs { + res = append(res, schema.ColumnToLabelName(cols[k][0])) + } + return res, annos, nil +} + +func materializeLabelValues(ctx context.Context, meta LabelValuesReadMeta, name string, rgi int, rr []rowRange) ([]string, annotations.Annotations, error) { + switch v := meta.Meta.Version; v { + case schema.V0, schema.V1, schema.V2: + return materializeLabelValuesV0V1(ctx, meta, name, rgi, rr) + default: + return nil, nil, fmt.Errorf("unable to materialize labels values for block of version %q", v) + } +} + +func materializeLabelValuesV0V1(ctx context.Context, meta LabelValuesReadMeta, name string, rgi int, rr []rowRange) ([]string, annotations.Annotations, error) { + var annos annotations.Annotations + + lc, ok := meta.LabelPfile.Schema().Lookup(schema.LabelNameToColumn(name)) + if !ok { + return nil, annos, nil + } + + rg := meta.LabelPfile.RowGroups()[rgi] + + cc := rg.ColumnChunks()[lc.ColumnIndex] + vals, err := materializeLabelColumn(ctx, rg, cc, rr) + if err != nil { + return nil, nil, fmt.Errorf("unable to materialize label %q column: %w", name, err) + } + + seen := make(map[string]struct{}) + res := make([]string, 0) + for _, v := range vals { + if v.IsNull() { + continue + } + key := yoloString(v.ByteArray()) + if _, ok := seen[key]; !ok { + res = append(res, v.Clone().String()) + seen[key] = struct{}{} + } + } + return res, annos, nil +} + +type chunkMaterializeConfig struct { + bytesQuota *limits.Quota + + readerFromContext readerFromContext + + partitionMaxRangeSize uint64 + partitionMaxGapSize uint64 + partitionMaxConcurrency int +} + +type chunkMaterializeOption func(*chunkMaterializeConfig) + +func withByteQuota(q *limits.Quota) chunkMaterializeOption { + return func(cfg *chunkMaterializeConfig) { + cfg.bytesQuota = q + } +} + +func withPartitionMaxRangeSize(maxRangeSize uint64) chunkMaterializeOption { + return func(cfg *chunkMaterializeConfig) { + cfg.partitionMaxRangeSize = maxRangeSize + } +} + +func withPartitionMaxGapSize(maxGapSize uint64) chunkMaterializeOption { + return func(cfg *chunkMaterializeConfig) { + cfg.partitionMaxGapSize = maxGapSize + } +} + +func withPartitionMaxConcurrency(maxConcurrency int) chunkMaterializeOption { + return func(cfg *chunkMaterializeConfig) { + cfg.partitionMaxConcurrency = maxConcurrency + } +} + +type readerFromContext func(ctx context.Context) io.ReaderAt + +func withReaderFromContext(rx readerFromContext) chunkMaterializeOption { + return func(cfg *chunkMaterializeConfig) { + cfg.readerFromContext = rx + } +} + +func materializeChunkColumn(ctx context.Context, rg parquet.RowGroup, cc parquet.ColumnChunk, rr []rowRange, opts ...chunkMaterializeOption) ([]parquet.Value, error) { + if len(rr) == 0 { + return nil, nil + } + rowCount := totalRows(rr) + column := rg.Schema().Columns()[cc.Column()][0] + + cfg := chunkMaterializeConfig{ + bytesQuota: limits.UnlimitedQuota(), + partitionMaxRangeSize: math.MaxUint64, + partitionMaxGapSize: math.MaxUint64, + } + for i := range opts { + opts[i](&cfg) + } + ctx, span := tracing.Tracer().Start(ctx, "Materialize Column") + defer span.End() + + span.SetAttributes(attribute.String("column", rg.Schema().Columns()[cc.Column()][0])) + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("could not get offset index: %w", err) + } + + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("could not get column index: %w", err) + } + + pagesToRowsMap := make(map[int][]rowRange, len(rr)) + for i := range cidx.NumPages() { + pr := rowRange{ + from: oidx.FirstRowIndex(i), + } + pr.count = rg.NumRows() + + if i < oidx.NumPages()-1 { + pr.count = oidx.FirstRowIndex(i+1) - pr.from + } + + for _, r := range rr { + if intersect(pr, r) { + pagesToRowsMap[i] = append(pagesToRowsMap[i], intersection(r, pr)) + } + } + } + + if err := checkByteQuota(cfg.bytesQuota, maps.Keys(pagesToRowsMap), oidx); err != nil { + return nil, err + } + + pageRanges := partitionPageRanges( + cfg.partitionMaxRangeSize, + cfg.partitionMaxGapSize, + pagesToRowsMap, + oidx, + ) + + r := make(map[rowRange][]parquet.Value, len(pageRanges)) + for _, v := range pageRanges { + for _, rs := range v.rows { + r[rs] = make([]parquet.Value, 0, rs.count) + } + } + + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + if limit := cfg.partitionMaxConcurrency; limit != 0 { + g.SetLimit(limit) + } + + method := methodFromContext(ctx) + columnMaterialized.WithLabelValues(column, method).Add(1) + rowsMaterialized.WithLabelValues(column, method).Add(float64(rowCount)) + + for _, p := range pageRanges { + g.Go(func() error { + ctx, span := tracing.Tracer().Start(ctx, "Materialize Page Range") + defer span.End() + + span.SetAttributes(attribute.IntSlice("pages", p.pages)) + + rdrAt := cfg.readerFromContext(ctx) + + // TODO: read pages in one big read here - then use "PagesFrom" with a bytes.NewReader + // that reads that byte slice - this prevents AsyncPages from overfetching and coalesces + // small reads into one big read + + minOffset := oidx.Offset(p.pages[0]) + maxOffset := oidx.Offset(p.pages[len(p.pages)-1]) + oidx.CompressedPageSize(p.pages[len(p.pages)-1]) + + bufRdrAt := newBufferedReaderAt(rdrAt, minOffset, maxOffset) + + pagesRead.WithLabelValues(column, method).Add(float64(len(p.pages))) + pagesReadSize.WithLabelValues(column, method).Add(float64(maxOffset - minOffset)) + + pgs := cc.(*parquet.FileColumnChunk).PagesFrom(bufRdrAt) + defer func() { _ = pgs.Close() }() + + if err := pgs.SeekToRow(p.rows[0].from); err != nil { + return fmt.Errorf("could not seek to row: %w", err) + } + + vi := &chunkValuesIterator{} + remainingRr := p.rows + currentRr := remainingRr[0] + next := currentRr.from + remaining := currentRr.count + currentRow := currentRr.from + + remainingRr = remainingRr[1:] + for len(remainingRr) > 0 || remaining > 0 { + page, err := pgs.ReadPage() + if err != nil { + return fmt.Errorf("unable to read page: %w", err) + } + + vi.Reset(page) + for vi.Next() { + if currentRow == next { + mu.Lock() + r[currentRr] = append(r[currentRr], vi.At()) + mu.Unlock() + remaining-- + if remaining > 0 { + next = next + 1 + } else if len(remainingRr) > 0 { + currentRr = remainingRr[0] + next = currentRr.from + remaining = currentRr.count + remainingRr = remainingRr[1:] + } + } + currentRow++ + } + parquet.Release(page) + + if err := vi.Error(); err != nil { + return fmt.Errorf("error during page iteration: %w", err) + } + } + return nil + }) + } + if err = g.Wait(); err != nil { + return nil, fmt.Errorf("unable to materialize columns: %w", err) + } + + ranges := slices.Collect(maps.Keys(r)) + slices.SortFunc(ranges, func(a, b rowRange) int { + return int(a.from - b.from) + }) + + res := make([]parquet.Value, 0, totalRows(rr)) + for _, v := range ranges { + res = append(res, r[v]...) + } + return res, nil +} + +func materializeLabelColumn(ctx context.Context, rg parquet.RowGroup, cc parquet.ColumnChunk, rr []rowRange) ([]parquet.Value, error) { + if len(rr) == 0 { + return nil, nil + } + rowCount := totalRows(rr) + column := rg.Schema().Columns()[cc.Column()][0] + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("could not get offset index: %w", err) + } + + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("could not get column index: %w", err) + } + + pagesToRowsMap := make(map[int][]rowRange, len(rr)) + for i := range cidx.NumPages() { + pr := rowRange{ + from: oidx.FirstRowIndex(i), + } + pr.count = rg.NumRows() + + if i < oidx.NumPages()-1 { + pr.count = oidx.FirstRowIndex(i+1) - pr.from + } + + for _, r := range rr { + if intersect(pr, r) { + pagesToRowsMap[i] = append(pagesToRowsMap[i], intersection(r, pr)) + } + } + } + + pageRanges := partitionPageRanges( + math.MaxUint64, + math.MaxUint64, + pagesToRowsMap, + oidx, + ) + + r := make(map[rowRange][]parquet.Value, len(pageRanges)) + for _, v := range pageRanges { + for _, rs := range v.rows { + r[rs] = make([]parquet.Value, 0, rs.count) + } + } + + var mu sync.Mutex + g, ctx := errgroup.WithContext(ctx) + + method := methodFromContext(ctx) + columnMaterialized.WithLabelValues(column, method).Add(1) + rowsMaterialized.WithLabelValues(column, method).Add(float64(rowCount)) + + for _, p := range pageRanges { + g.Go(func() error { + minOffset := oidx.Offset(p.pages[0]) + maxOffset := oidx.Offset(p.pages[len(p.pages)-1]) + oidx.CompressedPageSize(p.pages[len(p.pages)-1]) + + pagesRead.WithLabelValues(column, method).Add(float64(len(p.pages))) + pagesReadSize.WithLabelValues(column, method).Add(float64(maxOffset - minOffset)) + + pgs := cc.Pages() + defer func() { _ = pgs.Close() }() + + if err := pgs.SeekToRow(p.rows[0].from); err != nil { + return fmt.Errorf("could not seek to row: %w", err) + } + + vi := &chunkValuesIterator{} + remainingRr := p.rows + currentRr := remainingRr[0] + next := currentRr.from + remaining := currentRr.count + currentRow := currentRr.from + + remainingRr = remainingRr[1:] + for len(remainingRr) > 0 || remaining > 0 { + page, err := pgs.ReadPage() + if err != nil { + return fmt.Errorf("unable to read page: %w", err) + } + + vi.Reset(page) + for vi.Next() { + if currentRow == next { + mu.Lock() + r[currentRr] = append(r[currentRr], vi.At()) + mu.Unlock() + remaining-- + if remaining > 0 { + next = next + 1 + } else if len(remainingRr) > 0 { + currentRr = remainingRr[0] + next = currentRr.from + remaining = currentRr.count + remainingRr = remainingRr[1:] + } + } + currentRow++ + } + parquet.Release(page) + + if err := vi.Error(); err != nil { + return fmt.Errorf("error during page iteration: %w", err) + } + } + return nil + }) + } + if err = g.Wait(); err != nil { + return nil, fmt.Errorf("unable to materialize columns: %w", err) + } + + ranges := slices.Collect(maps.Keys(r)) + slices.SortFunc(ranges, func(a, b rowRange) int { + return int(a.from - b.from) + }) + + res := make([]parquet.Value, 0, totalRows(rr)) + for _, v := range ranges { + res = append(res, r[v]...) + } + return res, nil +} + +func totalRows(rr []rowRange) int64 { + res := int64(0) + for _, r := range rr { + res += r.count + } + return res +} + +func totalBytes(pages iter.Seq[int], oidx parquet.OffsetIndex) int64 { + res := int64(0) + for i := range pages { + res += oidx.CompressedPageSize(i) + } + return res +} + +func checkRowQuota(rowQuota *limits.Quota, rr []rowRange) error { + if err := rowQuota.Reserve(totalRows(rr)); err != nil { + return fmt.Errorf("would use too many rows: %w", err) + } + return nil +} + +func checkByteQuota(byteQuota *limits.Quota, pages iter.Seq[int], oidx parquet.OffsetIndex) error { + if err := byteQuota.Reserve(totalBytes(pages, oidx)); err != nil { + return fmt.Errorf("would use too many bytes: %w", err) + } + return nil +} + +type pageEntryRead struct { + pages []int + rows []rowRange +} + +func partitionPageRanges( + maxRangeSize uint64, + maxGapSize uint64, + pageIdx map[int][]rowRange, + offset parquet.OffsetIndex, +) []pageEntryRead { + partitioner := newGapBasedPartitioner(maxRangeSize, maxGapSize) + if len(pageIdx) == 0 { + return []pageEntryRead{} + } + idxs := make([]int, 0, len(pageIdx)) + for idx := range pageIdx { + idxs = append(idxs, idx) + } + + slices.Sort(idxs) + + parts := partitioner.partition(len(idxs), func(i int) (uint64, uint64) { + return uint64(offset.Offset(idxs[i])), uint64(offset.Offset(idxs[i]) + offset.CompressedPageSize(idxs[i])) + }) + + r := make([]pageEntryRead, 0, len(parts)) + for _, part := range parts { + pagesToRead := pageEntryRead{} + for i := part.elemRng[0]; i < part.elemRng[1]; i++ { + pagesToRead.pages = append(pagesToRead.pages, idxs[i]) + pagesToRead.rows = append(pagesToRead.rows, pageIdx[idxs[i]]...) + } + pagesToRead.rows = simplify(pagesToRead.rows) + r = append(r, pagesToRead) + } + return r +} + +type labelValuesIterator struct { + p parquet.Page + + cachedSymbols map[int32]parquet.Value + st symbolTable + + vr parquet.ValueReader + + current int + buffer []parquet.Value + currentBufferIndex int + err error +} + +func (vi *labelValuesIterator) Reset(p parquet.Page) { + vi.p = p + vi.vr = p.Values() + vi.st.Reset(p) + vi.cachedSymbols = make(map[int32]parquet.Value, p.Dictionary().Len()) + vi.current = -1 +} + +func (vi *labelValuesIterator) Next() bool { + if vi.err != nil { + return false + } + + vi.current++ + if vi.current >= int(vi.p.NumRows()) { + return false + } + + vi.currentBufferIndex++ + + if vi.currentBufferIndex == len(vi.buffer) { + n, err := vi.vr.ReadValues(vi.buffer[:cap(vi.buffer)]) + if err != nil && err != io.EOF { + vi.err = err + return false + } + vi.buffer = vi.buffer[:n] + vi.currentBufferIndex = 0 + } + return true +} + +func (vi *labelValuesIterator) Error() error { + return vi.err +} + +func (vi *labelValuesIterator) At() parquet.Value { + sym := vi.st.GetIndex(vi.current) + // Cache a clone of the current symbol table entry. + // This allows us to release the original page while avoiding unnecessary future clones. + if _, ok := vi.cachedSymbols[sym]; !ok { + vi.cachedSymbols[sym] = vi.st.Get(vi.current).Clone() + } + return vi.cachedSymbols[sym] +} + +type chunkValuesIterator struct { + p parquet.Page + + vr parquet.ValueReader + + current int + buffer []parquet.Value + currentBufferIndex int + err error +} + +func (vi *chunkValuesIterator) Reset(p parquet.Page) { + vi.p = p + vi.vr = p.Values() + vi.buffer = make([]parquet.Value, 0, 128) + vi.currentBufferIndex = -1 + vi.current = -1 +} + +func (vi *chunkValuesIterator) Next() bool { + if vi.err != nil { + return false + } + + vi.current++ + if vi.current >= int(vi.p.NumRows()) { + return false + } + + vi.currentBufferIndex++ + + if vi.currentBufferIndex == len(vi.buffer) { + n, err := vi.vr.ReadValues(vi.buffer[:cap(vi.buffer)]) + if err != nil && err != io.EOF { + vi.err = err + return false + } + vi.buffer = vi.buffer[:n] + vi.currentBufferIndex = 0 + } + return true +} + +func (vi *chunkValuesIterator) Error() error { + return vi.err +} + +func (vi *chunkValuesIterator) At() parquet.Value { + return vi.buffer[vi.currentBufferIndex].Clone() +} + +type bufferedReaderAt struct { + r io.ReaderAt + b []byte + offset int64 +} + +func (b bufferedReaderAt) ReadAt(p []byte, off int64) (n int, err error) { + if off >= b.offset && off < b.offset+int64(len(b.b)) { + diff := off - b.offset + n := copy(p, b.b[diff:]) + return n, nil + } + return b.r.ReadAt(p, off) +} + +func newBufferedReaderAt(r io.ReaderAt, minOffset, maxOffset int64) io.ReaderAt { + if minOffset < maxOffset { + b := make([]byte, maxOffset-minOffset) + n, err := r.ReadAt(b, minOffset) + if err == nil { + return &bufferedReaderAt{r: r, b: b[:n], offset: minOffset} + } + } + return r +} diff --git a/search/metrics.go b/search/metrics.go index f212737..65d4b2d 100644 --- a/search/metrics.go +++ b/search/metrics.go @@ -5,26 +5,67 @@ package search import ( - "github.com/hashicorp/go-multierror" + "context" + "errors" + "github.com/prometheus/client_golang/prometheus" ) +const ( + scanRegex = "regex" + scanEqual = "equal" + + methodSelect = "select" + methodLabelNames = "label_names" + methodLabelValues = "label_values" +) + +type ctxMethodKey struct{} + +var ctxMethodKeyVal = ctxMethodKey{} + +func contextWithMethod(ctx context.Context, method string) context.Context { + return context.WithValue(ctx, ctxMethodKeyVal, method) +} + +func methodFromContext(ctx context.Context) string { + return ctx.Value(ctxMethodKeyVal).(string) +} + var ( - pagesDiscarded = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "pages_discarded_total", - Help: "Pages discarded during parquet index scans", - }, []string{"column"}, + pagesScanned = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "pages_scanned_total", + Help: "Pages read during scans", + }, []string{"column", "scan", "method"}, ) pagesRead = prometheus.NewCounterVec(prometheus.CounterOpts{ Name: "pages_read_total", - Help: "Pages read during parquet page scans", - }, []string{"column"}, + Help: "Pages read during parquet operations", + }, []string{"column", "method"}, + ) + pagesReadSize = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "pages_read_size_bytes_total", + Help: "Cummulative size of pages in bytes that were read during parquet operations", + }, []string{"column", "method"}, + ) + columnMaterialized = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "column_materialized_total", + Help: "How often we had to materialize a column during queries", + }, []string{"column", "method"}, + ) + rowsMaterialized = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "rows_materialized_total", + Help: "How many rows we had to materialize for queries", + }, []string{"column", "method"}, ) ) func RegisterMetrics(reg prometheus.Registerer) error { - return multierror.Append(nil, - reg.Register(pagesDiscarded), + return errors.Join( + reg.Register(pagesScanned), reg.Register(pagesRead), - ).ErrorOrNil() + reg.Register(pagesReadSize), + reg.Register(columnMaterialized), + reg.Register(rowsMaterialized), + ) } diff --git a/search/partitioner.go b/search/partitioner.go new file mode 100644 index 0000000..44f09d7 --- /dev/null +++ b/search/partitioner.go @@ -0,0 +1,66 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package search + +import "math" + +type part struct { + start uint64 + end uint64 + + elemRng [2]int +} + +type partitioner interface { + // Partition partitions length entries into n <= length ranges that cover all input ranges. + // It supports overlapping ranges. + // It expects range to be sorted by lower bound. + partition(length int, rng func(int) (uint64, uint64)) []part +} + +type gapBasedPartitioner struct { + maxGapSize uint64 + maxRangeSize uint64 +} + +func newGapBasedPartitioner(maxRangeSize, maxGapSize uint64) partitioner { + return gapBasedPartitioner{ + maxGapSize: maxGapSize, + maxRangeSize: maxRangeSize, + } +} + +// partition partitions length entries into n <= length ranges that cover all +// input ranges by combining entries that are separated by reasonably small gaps. +// It is used to combine multiple small ranges from object storage into bigger, more efficient/cheaper ones. +func (g gapBasedPartitioner) partition(length int, rng func(int) (uint64, uint64)) (parts []part) { + for k := 0; k < length; { + j := k + k++ + + p := part{} + p.start, p.end = rng(j) + + // Keep growing the range until the end or we encounter a large gap. + for ; k < length; k++ { + s, e := rng(k) + + if e-p.start > g.maxRangeSize { + break + } + + if g.maxGapSize != math.MaxUint64 && p.end+g.maxGapSize < s { + break + } + + if p.end < e { + p.end = e + } + } + p.elemRng = [2]int{j, k} + parts = append(parts, p) + } + return parts +} diff --git a/search/partitioner_test.go b/search/partitioner_test.go new file mode 100644 index 0000000..a0bab79 --- /dev/null +++ b/search/partitioner_test.go @@ -0,0 +1,158 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package search + +import ( + "math" + "slices" + "testing" +) + +// TODO: add more tests and maybe even a sharper way to partition +func TestPartitioner(t *testing.T) { + type page struct { + fromBytes uint64 + toBytes uint64 + } + for _, tc := range []struct { + maxRangeSize uint64 + maxGapSize uint64 + pages []page + expect []part + }{ + { + maxRangeSize: 15, + maxGapSize: 2, + pages: []page{ + {fromBytes: 0, toBytes: 3}, + {fromBytes: 4, toBytes: 6}, + {fromBytes: 6, toBytes: 15}, + {fromBytes: 10, toBytes: 40}, + {fromBytes: 43, toBytes: 44}, + {fromBytes: 46, toBytes: 58}, + {fromBytes: 58, toBytes: 59}, + }, + expect: []part{ + {start: 0, end: 15, elemRng: [2]int{0, 3}}, + {start: 10, end: 40, elemRng: [2]int{3, 4}}, + {start: 43, end: 58, elemRng: [2]int{4, 6}}, + {start: 58, end: 59, elemRng: [2]int{6, 7}}, + }, + }, + { + maxRangeSize: 10, + maxGapSize: 10, + pages: []page{ + {fromBytes: 0, toBytes: 10}, + {fromBytes: 11, toBytes: 20}, + {fromBytes: 21, toBytes: 30}, + {fromBytes: 31, toBytes: 40}, + }, + expect: []part{ + {start: 0, end: 10, elemRng: [2]int{0, 1}}, + {start: 11, end: 20, elemRng: [2]int{1, 2}}, + {start: 21, end: 30, elemRng: [2]int{2, 3}}, + {start: 31, end: 40, elemRng: [2]int{3, 4}}, + }, + }, + { + maxRangeSize: 20, + maxGapSize: 100, + pages: []page{ + {fromBytes: 0, toBytes: 10}, + {fromBytes: 21, toBytes: 30}, + {fromBytes: 31, toBytes: 40}, + }, + expect: []part{ + {start: 0, end: 10, elemRng: [2]int{0, 1}}, + {start: 21, end: 40, elemRng: [2]int{1, 3}}, + }, + }, + { + maxRangeSize: 100, + maxGapSize: 5, + pages: []page{ + {fromBytes: 10, toBytes: 20}, + {fromBytes: 26, toBytes: 40}, + {fromBytes: 42, toBytes: 55}, + {fromBytes: 60, toBytes: 75}, + {fromBytes: 81, toBytes: 90}, + }, + expect: []part{ + {start: 10, end: 20, elemRng: [2]int{0, 1}}, + {start: 26, end: 75, elemRng: [2]int{1, 4}}, + {start: 81, end: 90, elemRng: [2]int{4, 5}}, + }, + }, + { + maxRangeSize: 30, + maxGapSize: 10, + pages: []page{ + {fromBytes: 5, toBytes: 15}, + {fromBytes: 18, toBytes: 25}, + {fromBytes: 26, toBytes: 35}, + {fromBytes: 36, toBytes: 45}, + {fromBytes: 50, toBytes: 60}, + }, + expect: []part{ + {start: 5, end: 35, elemRng: [2]int{0, 3}}, + {start: 36, end: 60, elemRng: [2]int{3, 5}}, + }, + }, + { + maxRangeSize: 100, + maxGapSize: 20, + pages: []page{ + {fromBytes: 10, toBytes: 30}, + {fromBytes: 50, toBytes: 70}, + {fromBytes: 91, toBytes: 100}, + }, + expect: []part{ + {start: 10, end: 70, elemRng: [2]int{0, 2}}, + {start: 91, end: 100, elemRng: [2]int{2, 3}}, + }, + }, + { + maxRangeSize: 50, + maxGapSize: 10, + pages: []page{ + {fromBytes: 10, toBytes: 10}, + {fromBytes: 15, toBytes: 25}, + {fromBytes: 20, toBytes: 30}, + {fromBytes: 42, toBytes: 42}, + {fromBytes: 45, toBytes: 60}, + }, + expect: []part{ + {start: 10, end: 30, elemRng: [2]int{0, 3}}, + {start: 42, end: 60, elemRng: [2]int{3, 5}}, + }, + }, + { + maxRangeSize: math.MaxUint64, + maxGapSize: math.MaxUint64, + pages: []page{ + {fromBytes: 10, toBytes: 20}, + {fromBytes: 100, toBytes: 200}, + {fromBytes: 1000, toBytes: 2000}, + {fromBytes: 10000, toBytes: 20000}, + }, + expect: []part{ + {start: 10, end: 20000, elemRng: [2]int{0, 4}}, + }, + }, + } { + t.Run("", func(tt *testing.T) { + gen := func(i int) (uint64, uint64) { + p := tc.pages[i] + return p.fromBytes, p.toBytes + } + part := newGapBasedPartitioner(tc.maxRangeSize, tc.maxGapSize) + + if got := part.partition(len(tc.pages), gen); !slices.Equal(got, tc.expect) { + tt.Fatalf("expected %+v, got %+v", tc.expect, got) + } + }) + } +} diff --git a/search/rowrange.go b/search/rowrange.go index 335c733..6d130e8 100644 --- a/search/rowrange.go +++ b/search/rowrange.go @@ -13,10 +13,35 @@ type rowRange struct { count int64 } -// intersect intserects the row ranges from left hand sight with the row ranges from rhs +func intersect(a, b rowRange) bool { + endA := a.from + a.count + endB := b.from + b.count + return a.from < endB && b.from < endA +} + +func intersection(a, b rowRange) rowRange { + os, oe := max(a.from, b.from), min(a.from+a.count, b.from+b.count) + return rowRange{from: os, count: oe - os} +} + +func limitRowRanges(limit int64, rr []rowRange) []rowRange { + res := make([]rowRange, 0, len(rr)) + cur := int64(0) + for i := range rr { + if cur+rr[i].count > limit { + res = append(res, rowRange{from: rr[i].from, count: rr[i].count - (rr[i].count - limit + cur)}) + break + } + res = append(res, rr[i]) + cur += rr[i].count + } + return simplify(res) +} + +// intersect intersects the row ranges from left hand sight with the row ranges from rhs // it assumes that lhs and rhs are simplified and returns a simplified result. // it operates in o(l+r) time by cursoring through ranges with a two pointer approach. -func intersectRowRanges(lhs []rowRange, rhs []rowRange) []rowRange { +func intersectRowRanges(lhs, rhs []rowRange) []rowRange { res := make([]rowRange, 0) for l, r := 0, 0; l < len(lhs) && r < len(rhs); { al, bl := lhs[l].from, lhs[l].from+lhs[l].count @@ -35,6 +60,64 @@ func intersectRowRanges(lhs []rowRange, rhs []rowRange) []rowRange { r++ } } + return simplify(res) +} + +// complementrowRanges returns the ranges that are in rhs but not in lhs. +// For example, if you have: +// lhs: [{from: 1, count: 3}] // represents rows 1,2,3 +// rhs: [{from: 0, count: 5}] // represents rows 0,1,2,3,4 +// The complement would be [{from: 0, count: 1}, {from: 4, count: 1}] // represents rows 0,4 +// because these are the rows in rhs that are not in lhs. +// +// The function assumes that lhs and rhs are simplified (no overlapping ranges) +// and returns a simplified result. It operates in O(l+r) time by using a two-pointer approach +// to efficiently process both ranges. +func complementRowRanges(lhs, rhs []rowRange) []rowRange { + res := make([]rowRange, 0) + + l, r := 0, 0 + for l < len(lhs) && r < len(rhs) { + al, bl := lhs[l].from, lhs[l].from+lhs[l].count + ar, br := rhs[r].from, rhs[r].from+rhs[r].count + + // check if rows intersect + switch { + case al > br || ar > bl: + // no intersection, advance cursor that ends first + if bl <= br { + l++ + } else { + res = append(res, rowRange{from: ar, count: br - ar}) + r++ + } + case al < ar && bl > br: + // l contains r, complement of l in r is empty, advance r + r++ + case al < ar && bl <= br: + // l covers r from left but has room on top + oe := min(bl, br) + rhs[r].from += oe - ar + rhs[r].count -= oe - ar + l++ + case al >= ar && bl > br: + // l covers r from right but has room on bottom + os := max(al, ar) + res = append(res, rowRange{from: ar, count: os - ar}) + r++ + case al >= ar && bl <= br: + // l is included r + os, oe := max(al, ar), min(bl, br) + res = append(res, rowRange{from: rhs[r].from, count: os - rhs[r].from}) + rhs[r].from = oe + rhs[r].count = br - oe + l++ + } + } + + for ; r < len(rhs); r++ { + res = append(res, rhs[r]) + } return simplify(res) } diff --git a/search/rowrange_test.go b/search/rowrange_test.go index 8fcc32a..86a9464 100644 --- a/search/rowrange_test.go +++ b/search/rowrange_test.go @@ -9,6 +9,36 @@ import ( "testing" ) +func TestLimit(t *testing.T) { + for _, tt := range []struct { + rr []rowRange + limit int64 + expect []rowRange + }{ + { + rr: []rowRange{{from: 0, count: 4}}, + limit: 3, + expect: []rowRange{{from: 0, count: 3}}, + }, + { + rr: []rowRange{{from: 0, count: 4}, {from: 5, count: 10}}, + limit: 5, + expect: []rowRange{{from: 0, count: 4}, {from: 5, count: 1}}, + }, + { + rr: []rowRange{{from: 0, count: 1e6}}, + limit: 5, + expect: []rowRange{{from: 0, count: 5}}, + }, + } { + t.Run("", func(t *testing.T) { + if res := limitRowRanges(tt.limit, tt.rr); !slices.Equal(res, tt.expect) { + t.Fatalf("Expected %v to match %v", res, tt.expect) + } + }) + } +} + func TestIntersect(t *testing.T) { for _, tt := range []struct{ lhs, rhs, expect []rowRange }{ { @@ -74,7 +104,94 @@ func TestIntersect(t *testing.T) { } { t.Run("", func(t *testing.T) { if res := intersectRowRanges(tt.lhs, tt.rhs); !slices.Equal(res, tt.expect) { - t.Fatalf("Expected %q to match %q", res, tt.expect) + t.Fatalf("Expected %v to match %v", res, tt.expect) + } + }) + } +} + +func TestComplement(t *testing.T) { + for _, tt := range []struct{ lhs, rhs, expect []rowRange }{ + { + lhs: []rowRange{{from: 4, count: 3}}, + rhs: []rowRange{{from: 2, count: 1}, {from: 5, count: 2}}, + expect: []rowRange{{from: 2, count: 1}}, + }, + { + lhs: []rowRange{{from: 2, count: 4}}, + rhs: []rowRange{{from: 0, count: 7}}, + expect: []rowRange{{from: 0, count: 2}, {from: 6, count: 1}}, + }, + { + lhs: []rowRange{{from: 2, count: 4}}, + rhs: []rowRange{{from: 3, count: 7}}, + expect: []rowRange{{from: 6, count: 4}}, + }, + { + lhs: []rowRange{{from: 8, count: 10}}, + rhs: []rowRange{{from: 3, count: 7}}, + expect: []rowRange{{from: 3, count: 5}}, + }, + { + lhs: []rowRange{{from: 16, count: 10}}, + rhs: []rowRange{{from: 3, count: 7}}, + expect: []rowRange{{from: 3, count: 7}}, + }, + { + lhs: []rowRange{{from: 1, count: 2}, {from: 4, count: 2}}, + rhs: []rowRange{{from: 2, count: 2}, {from: 5, count: 8}}, + expect: []rowRange{{from: 3, count: 1}, {from: 6, count: 7}}, + }, + // Empty input cases + { + lhs: []rowRange{}, + rhs: []rowRange{{from: 1, count: 5}}, + expect: []rowRange{{from: 1, count: 5}}, + }, + { + lhs: []rowRange{{from: 1, count: 5}}, + rhs: []rowRange{}, + expect: []rowRange{}, + }, + { + lhs: []rowRange{}, + rhs: []rowRange{}, + expect: []rowRange{}, + }, + // Adjacent ranges + { + lhs: []rowRange{{from: 1, count: 3}}, + rhs: []rowRange{{from: 1, count: 3}, {from: 4, count: 2}}, + expect: []rowRange{{from: 4, count: 2}}, + }, + // Ranges with gaps + { + lhs: []rowRange{{from: 1, count: 2}, {from: 5, count: 2}}, + rhs: []rowRange{{from: 0, count: 8}}, + expect: []rowRange{{from: 0, count: 1}, {from: 3, count: 2}, {from: 7, count: 1}}, + }, + // Zero-count ranges + { + lhs: []rowRange{{from: 1, count: 0}}, + rhs: []rowRange{{from: 1, count: 5}}, + expect: []rowRange{{from: 1, count: 5}}, + }, + // Completely disjoint ranges + { + lhs: []rowRange{{from: 1, count: 2}}, + rhs: []rowRange{{from: 5, count: 2}}, + expect: []rowRange{{from: 5, count: 2}}, + }, + // Multiple overlapping ranges + { + lhs: []rowRange{{from: 1, count: 3}, {from: 4, count: 3}, {from: 8, count: 2}}, + rhs: []rowRange{{from: 0, count: 11}}, + expect: []rowRange{{from: 0, count: 1}, {from: 7, count: 1}, {from: 10, count: 1}}, + }, + } { + t.Run("", func(t *testing.T) { + if res := complementRowRanges(tt.lhs, tt.rhs); !slices.Equal(res, tt.expect) { + t.Fatalf("Expected %v to match %v", res, tt.expect) } }) } @@ -119,3 +236,74 @@ func TestSimplify(t *testing.T) { }) } } + +func TestIntersects(t *testing.T) { + for _, tt := range []struct { + a, b rowRange + expect bool + }{ + // Identical ranges + { + a: rowRange{from: 0, count: 5}, + b: rowRange{from: 0, count: 5}, + expect: true, + }, + // Completely disjoint ranges + { + a: rowRange{from: 0, count: 3}, + b: rowRange{from: 5, count: 3}, + expect: false, + }, + // Adjacent ranges (should not overlap as ranges are half-open) + { + a: rowRange{from: 0, count: 3}, + b: rowRange{from: 3, count: 3}, + expect: false, + }, + // One range completely contains the other + { + a: rowRange{from: 0, count: 10}, + b: rowRange{from: 2, count: 5}, + expect: true, + }, + // Partial overlap from left + { + a: rowRange{from: 0, count: 5}, + b: rowRange{from: 3, count: 5}, + expect: true, + }, + // Partial overlap from right + { + a: rowRange{from: 3, count: 5}, + b: rowRange{from: 0, count: 5}, + expect: true, + }, + // Zero-count ranges + { + a: rowRange{from: 0, count: 0}, + b: rowRange{from: 0, count: 5}, + expect: false, + }, + { + a: rowRange{from: 0, count: 5}, + b: rowRange{from: 0, count: 0}, + expect: false, + }, + // Negative ranges (edge case) + { + a: rowRange{from: -5, count: 5}, + b: rowRange{from: -3, count: 5}, + expect: true, + }, + } { + t.Run("", func(t *testing.T) { + if res := intersect(tt.a, tt.b); res != tt.expect { + t.Fatalf("Expected %v.Overlaps(%v) to be %v, got %v", tt.a, tt.b, tt.expect, res) + } + // Test symmetry + if res := intersect(tt.b, tt.a); res != tt.expect { + t.Fatalf("Expected %v.Overlaps(%v) to be %v, got %v", tt.b, tt.a, tt.expect, res) + } + }) + } +} diff --git a/search/search.go b/search/search.go index eca7ce7..6c808c8 100644 --- a/search/search.go +++ b/search/search.go @@ -6,431 +6,385 @@ package search import ( "context" - "errors" "fmt" "io" + "slices" + "strings" + "sync" - "github.com/hashicorp/go-multierror" - "github.com/parquet-go/parquet-go" + "golang.org/x/sync/errgroup" - "github.com/cloudflare/parquet-tsdb-poc/internal/tracing" + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/util/annotations" + + "github.com/cloudflare/parquet-tsdb-poc/internal/limits" + "github.com/cloudflare/parquet-tsdb-poc/internal/util" + "github.com/cloudflare/parquet-tsdb-poc/internal/warnings" "github.com/cloudflare/parquet-tsdb-poc/schema" ) -type Constraint interface { - // rowRanges returns a set of non-overlapping increasing row indexes that may satisfy the constraint. - rowRanges(rg parquet.RowGroup) ([]rowRange, error) - // accept returns if this constraint is satisfied by the row. - accept(r parquet.Row) bool - // init initializes the constraint with respect to the file schema and projections. - init(s *parquet.Schema) error +type SeriesChunks struct { + LsetHash uint64 + Lset labels.Labels + Chunks []chunks.Meta } -type RowReaderCloser interface { - parquet.RowReader - io.Closer -} +type SelectReadMeta struct { + // The actual data for this Select call and metadata for how to read it + Meta schema.Meta + LabelPfile *parquet.File + ChunkPfile *parquet.File -func Match( - ctx context.Context, - c Constraint, - labelPfile *parquet.File, - labelSchema *parquet.Schema, - chunkPfile *parquet.File, - chunkSchema *parquet.Schema, -) (RowReaderCloser, error) { - ctx, span := tracing.Tracer().Start(ctx, "Match") - defer span.End() - - labelRowGroups := labelPfile.RowGroups() - chunkRowGroups := chunkPfile.RowGroups() - - joinedSchema := schema.Joined(labelSchema, chunkSchema) - if err := c.init(joinedSchema); err != nil { - return nil, fmt.Errorf("unable to initialize constraints: %w", err) - } + // We smuggle a bucket reader here so we can create a prepared ReaderAt + // for chunks that is scoped to this query and can be traced. + ChunkFileReaderFromContext func(ctx context.Context) io.ReaderAt - // label and chunk files have same number of rows and rowgroups, just pick either - numRowGroups := len(labelRowGroups) + // Quotas for the query that issued this Select call + ChunkBytesQuota *limits.Quota + RowCountQuota *limits.Quota - rrs := make([]RowReaderCloser, 0, numRowGroups) - for i := 0; i != numRowGroups; i++ { - ranges, err := c.rowRanges(labelRowGroups[i]) - if err != nil { - return nil, fmt.Errorf("unable to compute ranges for row group: %w", err) - } - if len(ranges) == 0 { - continue - } + // Hints about how we should partition page ranges for chunks. + // Page ranges are coalesced into a partition that is more efficient + // to read in object storage. I.e we will merge adjacent (up to a max gap) Pages + // into bigger ranges (up to a max size). These ranges are scheduled to be read + // concurrently. + ChunkPagePartitionMaxRange uint64 + ChunkPagePartitionMaxGap uint64 + ChunkPagePartitionMaxConcurrency int - columnChunks := make([]parquet.ColumnChunk, 0, len(joinedSchema.Columns())) - for _, p := range joinedSchema.Columns() { - if col, ok := labelRowGroups[i].Schema().Lookup(p...); ok { - columnChunks = append(columnChunks, labelRowGroups[i].ColumnChunks()[col.ColumnIndex]) - } else if col, ok := chunkRowGroups[i].Schema().Lookup(p...); ok { - columnChunks = append(columnChunks, chunkRowGroups[i].ColumnChunks()[col.ColumnIndex]) - } else { - // nothing to read here really - continue - } - } - rrs = append(rrs, newRangesRowReader(ranges, newRowGroupRows(joinedSchema, columnChunks))) - } - return newFilterRowReader(newConcatRowReader(rrs), c.accept), nil + // Thanos labels processing hints + ExternalLabels labels.Labels + ReplicaLabelNames []string } -type rangesRowReader struct { - ranges []rowRange - rows parquet.Rows - - n int - rMaxRow int - rCurRow int -} - -func newRangesRowReader(ranges []rowRange, rows parquet.Rows) *rangesRowReader { - return &rangesRowReader{ranges: ranges, rows: rows, n: -1} -} - -func (r *rangesRowReader) next() error { - if r.n == len(r.ranges)-1 { - return io.EOF - } - r.n++ - r.rMaxRow = int(r.ranges[r.n].count) - r.rCurRow = 0 - return r.rows.SeekToRow(r.ranges[r.n].from) -} - -func (r *rangesRowReader) ReadRows(buf []parquet.Row) (int, error) { - canRead := r.rMaxRow - r.rCurRow - if canRead == 0 { - if err := r.next(); err != nil { - return 0, err - } - canRead = r.rMaxRow - r.rCurRow - } - buf = buf[:min(len(buf), canRead)] - - n, err := r.rows.ReadRows(buf) - if err != nil { - return n, err +func Select( + ctx context.Context, + meta SelectReadMeta, + mint int64, + maxt int64, + hints *storage.SelectHints, + ms ...*labels.Matcher, +) ([]SeriesChunks, annotations.Annotations, error) { + ctx = contextWithMethod(ctx, methodSelect) + + ms, ok := matchExternalLabels(meta.ExternalLabels, ms) + if !ok { + return nil, nil, nil } - r.rCurRow += n - return n, err -} - -func (r *rangesRowReader) Close() error { - return r.rows.Close() -} + // label and chunk files have same number of rows and rowgroups, just pick either + labelRowGroups := meta.LabelPfile.RowGroups() + numRowGroups := len(labelRowGroups) -type concatRowReader struct { - idx int - rrs []RowReaderCloser -} + var ( + mu sync.Mutex + annos annotations.Annotations + ) -func newConcatRowReader(rrs []RowReaderCloser) *concatRowReader { - return &concatRowReader{rrs: rrs} -} + res := make([]SeriesChunks, 0) -func (f *concatRowReader) ReadRows(r []parquet.Row) (int, error) { - if f.idx >= len(f.rrs) { - return 0, io.EOF - } - n := 0 - for n != len(r) && f.idx != len(f.rrs) { - m, err := f.rrs[f.idx].ReadRows(r[n:]) - n += m - if err != nil { - if err == io.EOF { - f.idx++ - } else { - return n, err + g, ctx := errgroup.WithContext(ctx) + for i := range numRowGroups { + g.Go(func() error { + cs, err := matchersToConstraint(ms...) + if err != nil { + return fmt.Errorf("unable to convert matchers to constraints: %w", err) + } + if err := initialize(meta.LabelPfile.Schema(), cs...); err != nil { + return fmt.Errorf("unable to initialize constraints: %w", err) } - } - } - if n != len(r) { - return n, io.EOF - } - return n, nil -} - -func (f *concatRowReader) Close() error { - var err *multierror.Error - for i := range f.rrs { - err = multierror.Append(err, f.rrs[i].Close()) - } - return err.ErrorOrNil() -} - -type filterRowReader struct { - rr parquet.RowReader - closer io.Closer -} - -func newFilterRowReader(rr RowReaderCloser, accept func(r parquet.Row) bool) *filterRowReader { - return &filterRowReader{rr: parquet.FilterRowReader(rr, accept), closer: rr} -} - -func (f *filterRowReader) ReadRows(r []parquet.Row) (int, error) { - return f.rr.ReadRows(r) -} - -func (f *filterRowReader) Close() error { - return f.closer.Close() -} -// Copied from parquet-go https://github.com/parquet-go/parquet-go/blob/main/row_group.go -// Needs to be upstreamed eventually; Adapted to work with column chunks and joined schema + rrs, err := filter(ctx, labelRowGroups[i], cs...) + if err != nil { + return fmt.Errorf("unable to compute ranges: %w", err) + } + if len(rrs) == 0 { + return nil + } + series, warns, err := materializeSeries( + ctx, + meta, + i, + mint, + maxt, + hints, + rrs, + ) + if err != nil { + return fmt.Errorf("unable to materialize series: %w", err) + } -type columnChunkValueReader struct { - pages parquet.Pages - page parquet.Page - values parquet.ValueReader - release func(parquet.Page) -} + mu.Lock() + res = append(res, series...) + annos = annos.Merge(warns) + mu.Unlock() -func (r *columnChunkValueReader) clear() { - if r.page != nil { - r.release(r.page) - r.page = nil - r.values = nil + return nil + }) } -} -func (r *columnChunkValueReader) Reset() { - if r.pages != nil { - // Ignore errors because we are resetting the reader, if the error - // persists we will see it on the next read, and otherwise we can - // read back from the beginning. - r.pages.SeekToRow(0) + if err := g.Wait(); err != nil { + return nil, annos, fmt.Errorf("unable to process row groups: %w", err) } - r.clear() + return res, annos, nil } -func (r *columnChunkValueReader) Close() error { - var err error - if r.pages != nil { - err = r.pages.Close() - r.pages = nil - } - r.clear() - return err +type LabelValuesReadMeta struct { + // The actual data for this LabelValues call and metadata for how to read it + Meta schema.Meta + LabelPfile *parquet.File + + // Thanos labels processing hints + ExternalLabels labels.Labels + ReplicaLabelNames []string } -func (r *columnChunkValueReader) ReadValues(values []parquet.Value) (int, error) { - if r.pages == nil { - return 0, io.EOF +func LabelValues( + ctx context.Context, + meta LabelValuesReadMeta, + name string, + hints *storage.LabelHints, + ms ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + ctx = contextWithMethod(ctx, methodLabelValues) + + ms, ok := matchExternalLabels(meta.ExternalLabels, ms) + if !ok { + return nil, nil, nil } + extVal := externalLabelValues(meta.ExternalLabels, meta.ReplicaLabelNames, name) + + var ( + res []string + annos annotations.Annotations + ) + + if len(ms) == 0 { + // No matchers means we can read label values from the column dictionaries + vals := make([]string, 0) + for _, rg := range meta.LabelPfile.RowGroups() { + lc, ok := rg.Schema().Lookup(schema.LabelNameToColumn(name)) + if !ok { + continue + } + pg := rg.ColumnChunks()[lc.ColumnIndex].Pages() + defer pg.Close() - for { - if r.values == nil { - p, err := r.pages.ReadPage() + p, err := pg.ReadPage() if err != nil { - return 0, err + return nil, annos, fmt.Errorf("unable to read page: %w", err) } - r.page = p - r.values = p.Values() - } + d := p.Dictionary() - n, err := r.values.ReadValues(values) - if n > 0 { - return n, nil - } - if err == nil { - return 0, io.ErrNoProgress + for i := range d.Len() { + vals = append(vals, d.Index(int32(i)).Clone().String()) + } } - if err != io.EOF { - return 0, err + if len(extVal) != 0 { + if len(vals) != 0 { + annos = annos.Add(warnings.ErrorDroppedLabelValuesAfterExternalLabelMangling) + } + res = append(res, extVal) + } else { + res = append(res, vals...) } - r.clear() - } -} + } else { + // matchers means we need to actually read values of the column + labelRowGroups := meta.LabelPfile.RowGroups() + numRowGroups := len(labelRowGroups) + + var mu sync.Mutex + + g, ctx := errgroup.WithContext(ctx) + for i := range numRowGroups { + g.Go(func() error { + cs, err := matchersToConstraint(ms...) + if err != nil { + return fmt.Errorf("unable to convert matchers to constraints: %w", err) + } + if err := initialize(meta.LabelPfile.Schema(), cs...); err != nil { + return fmt.Errorf("unable to initialize constraints: %w", err) + } -func (r *columnChunkValueReader) SeekToRow(rowIndex int64) error { - if r.pages == nil { - return io.ErrClosedPipe - } - if err := r.pages.SeekToRow(rowIndex); err != nil { - return err - } - r.clear() - return nil -} + rrs, err := filter(ctx, labelRowGroups[i], cs...) + if err != nil { + return fmt.Errorf("unable to compute ranges: %w", err) + } + if len(rrs) == 0 { + return nil + } -type rowGroupRows struct { - schema *parquet.Schema - bufsize int - buffers []parquet.Value - columns []columnChunkRows - closed bool - rowIndex int64 -} + labelValues, warns, err := materializeLabelValues(ctx, meta, name, i, rrs) + if err != nil { + return fmt.Errorf("unable to materialize label values: %w", err) + } -type columnChunkRows struct { - offset int32 - length int32 - reader columnChunkValueReader -} + mu.Lock() + defer mu.Unlock() -func (r *rowGroupRows) buffer(i int) []parquet.Value { - j := (i + 0) * r.bufsize - k := (i + 1) * r.bufsize - return r.buffers[j:k:k] -} + annos = annos.Merge(warns) -func newRowGroupRows(schema *parquet.Schema, columns []parquet.ColumnChunk) *rowGroupRows { - bufferSize := 64 - r := &rowGroupRows{ - schema: schema, - bufsize: bufferSize, - buffers: make([]parquet.Value, len(columns)*bufferSize), - columns: make([]columnChunkRows, len(columns)), - rowIndex: -1, - } + if len(extVal) != 0 { + if len(labelValues) != 0 { + annos = annos.Add(warnings.ErrorDroppedLabelValuesAfterExternalLabelMangling) + } + res = append(res, extVal) + } else { + res = append(res, labelValues...) + } + return nil + }) + } - for i, column := range columns { - var release func(parquet.Page) - // Only release pages that are not byte array because the values - // that were read from the page might be retained by the program - // after calls to ReadRows. - switch column.Type().Kind() { - case parquet.ByteArray, parquet.FixedLenByteArray: - release = func(parquet.Page) {} - default: - release = parquet.Release + if err := g.Wait(); err != nil { + return nil, annos, fmt.Errorf("unable to process row groups: %w", err) } - r.columns[i].reader.release = release - r.columns[i].reader.pages = column.Pages() } - return r -} + limit := hints.Limit -func (r *rowGroupRows) clear() { - for i, c := range r.columns { - r.columns[i] = columnChunkRows{reader: c.reader} + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) } - clear(r.buffers) + return res, annos, nil } -func (r *rowGroupRows) Reset() { - for i := range r.columns { - r.columns[i].reader.Reset() - } - r.clear() -} +type LabelNamesReadMeta struct { + // The actual data for this LabelNames call and metadata for how to read it + Meta schema.Meta + LabelPfile *parquet.File -func (r *rowGroupRows) Close() error { - var errs []error - for i := range r.columns { - c := &r.columns[i] - c.offset = 0 - c.length = 0 - if err := c.reader.Close(); err != nil { - errs = append(errs, err) - } - } - r.clear() - r.closed = true - return errors.Join(errs...) + // Thanos labels processing hints + ExternalLabels labels.Labels + ReplicaLabelNames []string } -func (r *rowGroupRows) SeekToRow(rowIndex int64) error { - if r.closed { - return io.ErrClosedPipe +func LabelNames( + ctx context.Context, + meta LabelNamesReadMeta, + hints *storage.LabelHints, + ms ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + ctx = contextWithMethod(ctx, methodLabelNames) + + ms, ok := matchExternalLabels(meta.ExternalLabels, ms) + if !ok { + return nil, nil, nil } - if rowIndex != r.rowIndex { - for i := range r.columns { - if err := r.columns[i].reader.SeekToRow(rowIndex); err != nil { - return err + var ( + res []string + annos annotations.Annotations + ) + + if len(ms) == 0 { + // No matchers means we can read label names from the schema + for _, c := range meta.LabelPfile.Schema().Columns() { + if strings.HasPrefix(c[0], schema.LabelColumnPrefix) { + res = append(res, schema.ColumnToLabelName(c[0])) } } - r.clear() - r.rowIndex = rowIndex - } - return nil -} + res = append(res, externalLabelNames(meta.ExternalLabels, meta.ReplicaLabelNames)...) + } else { + // matchers means we need to fetch label names for matching rows from the table + labelRowGroups := meta.LabelPfile.RowGroups() + numRowGroups := len(labelRowGroups) + + var mu sync.Mutex + + g, ctx := errgroup.WithContext(ctx) + for i := range numRowGroups { + g.Go(func() error { + cs, err := matchersToConstraint(ms...) + if err != nil { + return fmt.Errorf("unable to convert matchers to constraints: %w", err) + } + if err := initialize(meta.LabelPfile.Schema(), cs...); err != nil { + return fmt.Errorf("unable to initialize constraints: %w", err) + } -func (r *rowGroupRows) ReadRows(rows []parquet.Row) (int, error) { - if r.closed { - return 0, io.EOF - } + rrs, err := filter(ctx, labelRowGroups[i], cs...) + if err != nil { + return fmt.Errorf("unable to compute ranges: %w", err) + } + if len(rrs) == 0 { + return nil + } + labelNames, warns, err := materializeLabelNames(ctx, meta, i, rrs) + if err != nil { + return fmt.Errorf("unable to materialize label names: %w", err) + } - for rowIndex := range rows { - rows[rowIndex] = rows[rowIndex][:0] - } + mu.Lock() + defer mu.Unlock() + + res = append(res, externalLabelNames(meta.ExternalLabels, meta.ReplicaLabelNames)...) + res = append(res, labelNames...) + annos = annos.Merge(warns) - // When this is the first call to ReadRows, we issue a seek to the first row - // because this starts prefetching pages asynchronously on columns. - // - // This condition does not apply if SeekToRow was called before ReadRows, - // only when ReadRows is the very first method called on the row reader. - if r.rowIndex < 0 { - if err := r.SeekToRow(0); err != nil { - return 0, err + return nil + }) + } + if err := g.Wait(); err != nil { + return nil, annos, fmt.Errorf("unable to process row groups: %w", err) } } + limit := hints.Limit - eofCount := 0 - rowCount := 0 - -readColumnValues: - for columnIndex := range r.columns { - c := &r.columns[columnIndex] - b := r.buffer(columnIndex) - eof := false - - for rowIndex := range rows { - numValuesInRow := 1 - - for { - if c.offset == c.length { - n, err := c.reader.ReadValues(b) - c.offset = 0 - c.length = int32(n) - - if n == 0 { - if err == io.EOF { - eof = true - eofCount++ - break - } - return 0, err - } - } - - values := b[c.offset:c.length:c.length] - for numValuesInRow < len(values) && values[numValuesInRow].RepetitionLevel() != 0 { - numValuesInRow++ - } - if numValuesInRow == 0 { - break - } - - rows[rowIndex] = append(rows[rowIndex], values[:numValuesInRow]...) - rowCount = max(rowCount, rowIndex+1) - c.offset += int32(numValuesInRow) + res = util.SortUnique(res) + if limit > 0 && len(res) > limit { + res = res[:limit] + annos = annos.Add(warnings.ErrorTruncatedResponse) + } + return res, annos, nil +} - if numValuesInRow != len(values) { - break - } - if eof { - continue readColumnValues - } - numValuesInRow = 0 +func matchExternalLabels(extLabels labels.Labels, matchers []*labels.Matcher) ([]*labels.Matcher, bool) { + // if the matchers match on some external label, we need to consume that matcher here + remain := make([]*labels.Matcher, 0, len(matchers)) + for i := range matchers { + exclude, consumed := false, false + extLabels.Range(func(lbl labels.Label) { + if matchers[i].Name != lbl.Name { + return } + exclude = exclude || !matchers[i].Matches(lbl.Value) + consumed = true + }) + if exclude { + return nil, false + } + if !consumed { + remain = append(remain, matchers[i]) } } + return remain, true +} - var err error - if eofCount > 0 { - err = io.EOF - } - r.rowIndex += int64(rowCount) - return rowCount, err +func externalLabelValues(extLabels labels.Labels, replicaLabelNames []string, name string) string { + extVal := "" + extLabels.Range(func(lbl labels.Label) { + if slices.Contains(replicaLabelNames, lbl.Name) { + return + } + if lbl.Name != name { + return + } + extVal = lbl.Value + }) + return extVal } -func (r *rowGroupRows) Schema() *parquet.Schema { - return r.schema +func externalLabelNames(extLabels labels.Labels, replicaLabelNames []string) []string { + res := make([]string, 0, extLabels.Len()) + extLabels.Range(func(lbl labels.Label) { + if slices.Contains(replicaLabelNames, lbl.Name) { + return + } + res = append(res, lbl.Name) + }) + return res } diff --git a/search/search_test.go b/search/search_test.go deleted file mode 100644 index 802e609..0000000 --- a/search/search_test.go +++ /dev/null @@ -1,228 +0,0 @@ -// Copyright (c) 2025 Cloudflare, Inc. -// Licensed under the Apache 2.0 license found in the LICENSE file or at: -// https://opensource.org/licenses/Apache-2.0 - -package search - -import ( - "bytes" - "context" - "io" - "slices" - "testing" - - "github.com/parquet-go/parquet-go" - - "github.com/cloudflare/parquet-tsdb-poc/schema" -) - -func TestSearch(t *testing.T) { - ctx := context.Background() - t.Run("", func(t *testing.T) { - type S struct { - A int64 - B int64 - C string - } - srows := []S{ - { - A: 1, - B: 2, - C: "a", - }, - { - A: 3, - B: 4, - C: "b", - }, - { - A: 7, - B: 12, - C: "c", - }, - { - A: 9, - B: 22, - C: "d", - }, - { - A: 0, - B: 1, - C: "e", - }, - { - A: 0, - B: 1, - C: "f", - }, - { - A: 0, - B: 1, - C: "g", - }, - { - A: 0, - B: 1, - C: "h", - }, - } - type T struct { - D string - } - - trows := []T{ - { - D: "h", - }, - { - D: "g", - }, - { - D: "f", - }, - { - D: "e", - }, - { - D: "d", - }, - { - D: "c", - }, - { - D: "b", - }, - { - D: "a", - }, - } - sfile := buildFile(t, srows) - tfile := buildFile(t, trows) - - t.Run("", func(t *testing.T) { - constraint := And( - Not(Equal("B", parquet.ValueOf(3))), - Or( - Equal("C", parquet.ValueOf("f")), - Equal("C", parquet.ValueOf("g")), - ), - ) - - sproj := schema.Projection(sfile.Schema(), []string{"B", "C"}) - tproj := schema.Projection(tfile.Schema(), []string{"D"}) - - rr, err := Match( - ctx, - constraint, - sfile, sproj, - tfile, tproj, - ) - if err != nil { - t.Fatal(err) - } - defer rr.Close() - - got := readAll(t, rr) - expect := []parquet.Row{ - {parquet.ValueOf(1), parquet.ValueOf("f"), parquet.ValueOf("c")}, - {parquet.ValueOf(1), parquet.ValueOf("g"), parquet.ValueOf("b")}, - } - - if !equalRows(got, expect) { - t.Fatalf("expected %q to equal %q", got, expect) - } - }) - }) -} - -func equalRows(l, r []parquet.Row) bool { - return slices.EqualFunc(l, r, func(ll, rr parquet.Row) bool { - return equalRow(ll, rr) - }) -} - -func equalRow(l, r parquet.Row) bool { - return slices.EqualFunc(l, r, func(lv, rv parquet.Value) bool { - return lv.String() == rv.String() - }) -} - -func buildFile[T any](t testing.TB, rows []T) *parquet.File { - buf := bytes.NewBuffer(nil) - w := parquet.NewGenericWriter[T](buf, parquet.PageBufferSize(12), parquet.WriteBufferSize(0)) - for _, row := range rows { - if _, err := w.Write([]T{row}); err != nil { - t.Fatal(err) - } - } - if err := w.Close(); err != nil { - t.Fatal(err) - } - reader := bytes.NewReader(buf.Bytes()) - file, err := parquet.OpenFile(reader, reader.Size()) - if err != nil { - t.Fatal(err) - } - return file -} - -type testRows struct { - p int - rows []parquet.Row -} - -// parquet.Rows interface stuff we care about -func (trs *testRows) Close() error { return nil } - -func (trs *testRows) ReadRows(r []parquet.Row) (int, error) { - if trs.p >= len(trs.rows) { - return 0, io.EOF - } - n := copy(r, trs.rows) - trs.p += n - - if trs.p == len(trs.rows) { - return n, io.EOF - } - return n, nil -} - -func TestConcatRowReader(t *testing.T) { - ccr := newConcatRowReader([]RowReaderCloser{ - &testRows{ - rows: []parquet.Row{{parquet.ValueOf(0), parquet.ValueOf(1)}, {parquet.ValueOf(3)}}, - }, - &testRows{ - rows: []parquet.Row{{parquet.ValueOf(4), parquet.ValueOf(5)}, {parquet.ValueOf(6)}}, - }, - }) - - got := readAll(t, ccr) - expect := []parquet.Row{ - {parquet.ValueOf(0), parquet.ValueOf(1)}, - {parquet.ValueOf(3)}, - {parquet.ValueOf(4), parquet.ValueOf(5)}, - {parquet.ValueOf(6)}, - } - - if !equalRows(got, expect) { - t.Fatalf("expected %q to equal %q", got, expect) - } -} - -func readAll(t *testing.T, rr parquet.RowReader) []parquet.Row { - res := make([]parquet.Row, 0) - - rw := parquet.RowWriterFunc(func(rs []parquet.Row) (int, error) { - res = slices.Grow(res, len(res)) - for _, r := range rs { - res = append(res, r.Clone()) - } - return len(res), nil - }) - if _, err := parquet.CopyRows(rw, rr); err != nil { - t.Fatal(err) - } - - return res -} diff --git a/search/symbol_table.go b/search/symbol_table.go new file mode 100644 index 0000000..883ecc5 --- /dev/null +++ b/search/symbol_table.go @@ -0,0 +1,58 @@ +// Copyright (c) 2025 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +package search + +import ( + "slices" + + "github.com/parquet-go/parquet-go" +) + +// symbolTable is a helper that can decode the i-th value of a page. +// Using it we only need to allocate an int32 slice and not a slice of +// string values. +// It only works for optional dictionary encoded columns. All of our label +// columns are that though. +type symbolTable struct { + dict parquet.Dictionary + syms []int32 +} + +func (s *symbolTable) Get(i int) parquet.Value { + switch s.syms[i] { + case -1: + return parquet.NullValue() + default: + return s.dict.Index(s.syms[i]) + } +} + +func (s *symbolTable) GetIndex(i int) int32 { + return s.syms[i] +} + +func (s *symbolTable) Reset(pg parquet.Page) { + dict := pg.Dictionary() + data := pg.Data() + syms := data.Int32() + defs := pg.DefinitionLevels() + + if s.syms == nil { + s.syms = make([]int32, len(defs)) + } else { + s.syms = slices.Grow(s.syms, len(defs))[:len(defs)] + } + + sidx := 0 + for i := range defs { + if defs[i] == 1 { + s.syms[i] = syms[sidx] + sidx++ + } else { + s.syms[i] = -1 + } + } + s.dict = dict +} diff --git a/shell.nix b/shell.nix index 4afe746..71b5599 100644 --- a/shell.nix +++ b/shell.nix @@ -5,7 +5,7 @@ pkgs.mkShell { name = "env"; hardeningDisable = [ "fortify" ]; buildInputs = with pkgs; [ - go_1_23 + go_1_24 gotools delve revive