Documentation ¶
Index ¶
- Constants
- Variables
- func ExpandPostingsWithContext(ctx context.Context, p index.Postings) (res []storage.SeriesRef, err error)
- func LabelSetsMatch(matchers []*labels.Matcher, lset ...labels.Labels) bool
- func MatchersForLabelSets(labelSets []labels.Labels) []storepb.LabelMatcher
- func NewDefaultChunkBytesPool(maxChunkPoolBytes uint64) (pool.Bytes, error)
- func NewInstrumentedStoreServer(reg prometheus.Registerer, store storepb.StoreServer) storepb.StoreServer
- func NewLimitedStoreServer(store storepb.StoreServer, reg prometheus.Registerer, ...) storepb.StoreServer
- func NewNoCopyScanner(b []byte, splitFunc bufio.SplitFunc) *noCopyScanner
- func NewProxyResponseLoserTree(seriesSets ...respSet) *losertree.Tree[*storepb.SeriesResponse, respSet]
- func NewRecoverableStoreServer(logger log.Logger, storeServer storepb.StoreServer) *recoverableStoreServer
- func NewResponseDeduplicator(h *losertree.Tree[*storepb.SeriesResponse, respSet]) *responseDeduplicator
- func NewSeriesStatsAggregator(reg prometheus.Registerer, durationQuantiles []float64, ...) *seriesStatsAggregator
- func NewSeriesStatsAggregatorFactory(reg prometheus.Registerer, durationQuantiles []float64, ...) *seriesStatsAggregatorFactory
- func NoopRequestLoggerFunc(_ context.Context, logger log.Logger) log.Logger
- func RegisterStoreServer(storeSrv storepb.StoreServer, logger log.Logger) func(*grpc.Server)
- func RegisterWritableStoreServer(storeSrv storepb.WriteableStoreServer) func(*grpc.Server)
- func ScanGRPCCurlProtoStreamMessages(data []byte, atEOF bool) (advance int, token []byte, err error)
- type BlockEstimator
- type BucketStore
- func (s *BucketStore) Close() (err error)
- func (s *BucketStore) Info(context.Context, *storepb.InfoRequest) (*storepb.InfoResponse, error)
- func (s *BucketStore) InitialSync(ctx context.Context) error
- func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
- func (s *BucketStore) LabelSet() []labelpb.ZLabelSet
- func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
- func (s *BucketStore) Series(req *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) (err error)
- func (s *BucketStore) SyncBlocks(ctx context.Context) error
- func (s *BucketStore) TSDBInfos() []infopb.TSDBInfo
- func (s *BucketStore) TimeRange() (mint, maxt int64)
- type BucketStoreOption
- func WithBlockEstimatedMaxChunkFunc(f BlockEstimator) BucketStoreOption
- func WithBlockEstimatedMaxSeriesFunc(f BlockEstimator) BucketStoreOption
- func WithChunkHashCalculation(enableChunkHashCalculation bool) BucketStoreOption
- func WithChunkPool(chunkPool pool.Bytes) BucketStoreOption
- func WithDebugLogging() BucketStoreOption
- func WithDontResort(true bool) BucketStoreOption
- func WithFilterConfig(filter *FilterConfig) BucketStoreOption
- func WithIndexCache(cache storecache.IndexCache) BucketStoreOption
- func WithIndexHeaderLazyDownloadStrategy(strategy indexheader.LazyDownloadIndexHeaderFunc) BucketStoreOption
- func WithLazyExpandedPostings(enabled bool) BucketStoreOption
- func WithLogger(logger log.Logger) BucketStoreOption
- func WithQueryGate(queryGate gate.Gate) BucketStoreOption
- func WithRegistry(reg prometheus.Registerer) BucketStoreOption
- func WithRequestLoggerFunc(loggerFunc RequestLoggerFunc) BucketStoreOption
- func WithSeriesBatchSize(seriesBatchSize int) BucketStoreOption
- type BytesCounter
- type BytesLimiter
- type BytesLimiterFactory
- type ChunksLimiter
- type ChunksLimiterFactory
- type Client
- type CloseDelegator
- type FilterConfig
- type Limiter
- type LocalStore
- func (s *LocalStore) Close() (err error)
- func (s *LocalStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
- func (s *LocalStore) LabelNames(_ context.Context, _ *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
- func (s *LocalStore) LabelValues(_ context.Context, r *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
- func (s *LocalStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error
- type NoopSeriesStatsAggregator
- type NoopSeriesStatsAggregatorFactory
- type Part
- type Partitioner
- type PrometheusStore
- func (p *PrometheusStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
- func (p *PrometheusStore) LabelNames(ctx context.Context, r *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
- func (p *PrometheusStore) LabelSet() []labelpb.ZLabelSet
- func (p *PrometheusStore) LabelValues(ctx context.Context, r *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
- func (p *PrometheusStore) Series(r *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) error
- func (p *PrometheusStore) TSDBInfos() []infopb.TSDBInfo
- func (p *PrometheusStore) Timestamps() (mint int64, maxt int64)
- type ProxyStore
- func (s *ProxyStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
- func (s *ProxyStore) LabelNames(ctx context.Context, originalRequest *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
- func (s *ProxyStore) LabelSet() []labelpb.ZLabelSet
- func (s *ProxyStore) LabelValues(ctx context.Context, originalRequest *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
- func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error
- func (s *ProxyStore) TSDBInfos() []infopb.TSDBInfo
- func (s *ProxyStore) TimeRange() (int64, int64)
- type ProxyStoreOption
- type ReadWriteTSDBStore
- type RequestLoggerFunc
- type RetrievalStrategy
- type SeriesLimiter
- type SeriesLimiterFactory
- type SeriesQueryPerformanceMetricsAggregator
- type SeriesQueryPerformanceMetricsAggregatorFactory
- type SeriesSelectLimits
- type StoreDataType
- type TSDBReader
- type TSDBSelector
- type TSDBStore
- func (s *TSDBStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
- func (s *TSDBStore) LabelNames(ctx context.Context, r *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
- func (s *TSDBStore) LabelSet() []labelpb.ZLabelSet
- func (s *TSDBStore) LabelValues(ctx context.Context, r *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
- func (s *TSDBStore) Series(r *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) error
- func (s *TSDBStore) SeriesLocal(ctx context.Context, r *storepb.SeriesRequest) ([]*storepb.Series, error)
- func (s *TSDBStore) SetExtLset(extLset labels.Labels)
- func (p *TSDBStore) TSDBInfos() []infopb.TSDBInfo
- func (s *TSDBStore) TimeRange() (int64, int64)
Constants ¶
const ( // MaxSamplesPerChunk is approximately the max number of samples that we may have in any given chunk. This is needed // for precalculating the number of samples that we may have to retrieve and decode for any given query // without downloading them. Please take a look at https://github.com/prometheus/tsdb/pull/397 to know // where this number comes from. Long story short: TSDB is made in such a way, and it is made in such a way // because you barely get any improvements in compression when the number of samples is beyond this. // Take a look at Figure 6 in this whitepaper http://www.vldb.org/pvldb/vol8/p1816-teller.pdf. MaxSamplesPerChunk = 120 // EstimatedMaxChunkSize is average max of chunk size. This can be exceeded though in very rare (valid) cases. EstimatedMaxChunkSize = 16000 EstimatedMaxSeriesSize = 64 * 1024 // CompatibilityTypeLabelName is an artificial label that Store Gateway can optionally advertise. This is required for compatibility // with pre v0.8.0 Querier. Previous Queriers was strict about duplicated external labels of all StoreAPIs that had any labels. // Now with newer Store Gateway advertising all the external labels it has access to, there was simple case where // Querier was blocking Store Gateway as duplicate with sidecar. // // Newer Queriers are not strict, no duplicated external labels check is there anymore. // Additionally newer Queriers removes/ignore this exact labels from UI and querying. // // This label name is intentionally against Prometheus label style. // TODO(bwplotka): Remove it at some point. CompatibilityTypeLabelName = "@thanos_compatibility_store_type" // DefaultPostingOffsetInMemorySampling represents default value for --store.index-header-posting-offsets-in-mem-sampling. // 32 value is chosen as it's a good balance for common setups. Sampling that is not too large (too many CPU cycles) and // not too small (too much memory). DefaultPostingOffsetInMemorySampling = 32 PartitionerMaxGapSize = 512 * 1024 // SeriesBatchSize is the default batch size when fetching series from object storage. SeriesBatchSize = 10000 )
const RemoteReadFrameLimit = 1048576
const StoreMatcherKey = ctxKey(0)
StoreMatcherKey is the context key for the store's allow list.
const UninitializedTSDBTime = math.MaxInt64
UninitializedTSDBTime is the TSDB start time of an uninitialized TSDB instance.
Variables ¶
var DefaultSelector = noopSelector()
var ErrorNoStoresMatched = errors.New("No StoreAPIs matched for this query")
ErrorNoStoresMatched is returned if the query does not match any data. This can happen with Query servers trees and external labels.
Functions ¶
func ExpandPostingsWithContext ¶ added in v0.32.0
func ExpandPostingsWithContext(ctx context.Context, p index.Postings) (res []storage.SeriesRef, err error)
ExpandPostingsWithContext returns the postings expanded as a slice and considers context.
func LabelSetsMatch ¶ added in v0.36.0
LabelSetsMatch returns false if all label-set do not match the matchers (aka: OR is between all label-sets).
func MatchersForLabelSets ¶ added in v0.35.0
func MatchersForLabelSets(labelSets []labels.Labels) []storepb.LabelMatcher
MatchersForLabelSets generates a list of label matchers for the given label sets.
func NewDefaultChunkBytesPool ¶ added in v0.19.0
NewDefaultChunkBytesPool returns a chunk bytes pool with default settings.
func NewInstrumentedStoreServer ¶ added in v0.31.0
func NewInstrumentedStoreServer(reg prometheus.Registerer, store storepb.StoreServer) storepb.StoreServer
NewInstrumentedStoreServer creates a new instrumentedStoreServer.
func NewLimitedStoreServer ¶ added in v0.31.0
func NewLimitedStoreServer(store storepb.StoreServer, reg prometheus.Registerer, selectLimits SeriesSelectLimits) storepb.StoreServer
NewLimitedStoreServer creates a new limitedStoreServer.
func NewNoCopyScanner ¶ added in v0.13.0
NewNoCopyScanner returns bufio.Scanner-like scanner that is meant to be used on already allocated byte slice (or mmapped) one. Returned tokens are shared.
func NewProxyResponseLoserTree ¶ added in v0.35.0
func NewProxyResponseLoserTree(seriesSets ...respSet) *losertree.Tree[*storepb.SeriesResponse, respSet]
NewProxyResponseLoserTree returns heap that k-way merge series together. It's agnostic to duplicates and overlaps, it forwards all duplicated series in random order.
func NewRecoverableStoreServer ¶ added in v0.31.0
func NewRecoverableStoreServer(logger log.Logger, storeServer storepb.StoreServer) *recoverableStoreServer
func NewResponseDeduplicator ¶ added in v0.35.0
func NewResponseDeduplicator(h *losertree.Tree[*storepb.SeriesResponse, respSet]) *responseDeduplicator
NewResponseDeduplicator returns a wrapper around a loser tree that merges duplicated series messages into one. It also deduplicates identical chunks identified by the same checksum from each series message.
func NewSeriesStatsAggregator ¶ added in v0.29.0
func NewSeriesStatsAggregator( reg prometheus.Registerer, durationQuantiles []float64, sampleQuantiles []float64, seriesQuantiles []float64, ) *seriesStatsAggregator
NewSeriesStatsAggregator is a constructor for seriesStatsAggregator.
func NewSeriesStatsAggregatorFactory ¶ added in v0.32.0
func NewSeriesStatsAggregatorFactory( reg prometheus.Registerer, durationQuantiles []float64, sampleQuantiles []float64, seriesQuantiles []float64, ) *seriesStatsAggregatorFactory
func NoopRequestLoggerFunc ¶ added in v0.36.0
func RegisterStoreServer ¶ added in v0.16.0
func RegisterWritableStoreServer ¶ added in v0.16.0
func RegisterWritableStoreServer(storeSrv storepb.WriteableStoreServer) func(*grpc.Server)
Types ¶
type BlockEstimator ¶ added in v0.32.0
type BucketStore ¶
type BucketStore struct {
// contains filtered or unexported fields
}
BucketStore implements the store API backed by a bucket. It loads all index files to local disk.
NOTE: Bucket store reencodes postings using diff+varint+snappy when storing to cache. This makes them smaller, but takes extra CPU and memory. When used with in-memory cache, memory usage should decrease overall, thanks to postings being smaller.
func NewBucketStore ¶
func NewBucketStore( bkt objstore.InstrumentedBucketReader, fetcher block.MetadataFetcher, dir string, chunksLimiterFactory ChunksLimiterFactory, seriesLimiterFactory SeriesLimiterFactory, bytesLimiterFactory BytesLimiterFactory, partitioner Partitioner, blockSyncConcurrency int, enableCompatibilityLabel bool, postingOffsetsInMemSampling int, enableSeriesResponseHints bool, lazyIndexReaderEnabled bool, lazyIndexReaderIdleTimeout time.Duration, options ...BucketStoreOption, ) (*BucketStore, error)
NewBucketStore creates a new bucket backed store that implements the store API against an object store bucket. It is optimized to work against high latency backends.
func (*BucketStore) Info ¶
func (s *BucketStore) Info(context.Context, *storepb.InfoRequest) (*storepb.InfoResponse, error)
Info implements the storepb.StoreServer interface.
func (*BucketStore) InitialSync ¶
func (s *BucketStore) InitialSync(ctx context.Context) error
InitialSync perform blocking sync with extra step at the end to delete locally saved blocks that are no longer present in the bucket. The mismatch of these can only happen between restarts, so we can do that only once per startup.
func (*BucketStore) LabelNames ¶
func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
LabelNames implements the storepb.StoreServer interface.
func (*BucketStore) LabelSet ¶ added in v0.24.0
func (s *BucketStore) LabelSet() []labelpb.ZLabelSet
func (*BucketStore) LabelValues ¶
func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
LabelValues implements the storepb.StoreServer interface.
func (*BucketStore) Series ¶
func (s *BucketStore) Series(req *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) (err error)
Series implements the storepb.StoreServer interface.
func (*BucketStore) SyncBlocks ¶
func (s *BucketStore) SyncBlocks(ctx context.Context) error
SyncBlocks synchronizes the stores state with the Bucket bucket. It will reuse disk space as persistent cache based on s.dir param.
func (*BucketStore) TSDBInfos ¶ added in v0.32.0
func (s *BucketStore) TSDBInfos() []infopb.TSDBInfo
TSDBInfos returns a list of infopb.TSDBInfos for blocks in the bucket store.
func (*BucketStore) TimeRange ¶
func (s *BucketStore) TimeRange() (mint, maxt int64)
TimeRange returns the minimum and maximum timestamp of data available in the store.
type BucketStoreOption ¶ added in v0.20.0
type BucketStoreOption func(s *BucketStore)
BucketStoreOption are functions that configure BucketStore.
func WithBlockEstimatedMaxChunkFunc ¶ added in v0.32.0
func WithBlockEstimatedMaxChunkFunc(f BlockEstimator) BucketStoreOption
func WithBlockEstimatedMaxSeriesFunc ¶ added in v0.32.0
func WithBlockEstimatedMaxSeriesFunc(f BlockEstimator) BucketStoreOption
func WithChunkHashCalculation ¶ added in v0.30.0
func WithChunkHashCalculation(enableChunkHashCalculation bool) BucketStoreOption
func WithChunkPool ¶ added in v0.20.0
func WithChunkPool(chunkPool pool.Bytes) BucketStoreOption
WithChunkPool sets a pool.Bytes to use for chunks.
func WithDebugLogging ¶ added in v0.20.0
func WithDebugLogging() BucketStoreOption
WithDebugLogging enables debug logging.
func WithDontResort ¶ added in v0.33.0
func WithDontResort(true bool) BucketStoreOption
WithDontResort disables series resorting in Store Gateway.
func WithFilterConfig ¶ added in v0.20.0
func WithFilterConfig(filter *FilterConfig) BucketStoreOption
WithFilterConfig sets a filter which Store uses for filtering metrics based on time.
func WithIndexCache ¶ added in v0.20.0
func WithIndexCache(cache storecache.IndexCache) BucketStoreOption
WithIndexCache sets a indexCache to use instead of a noopCache.
func WithIndexHeaderLazyDownloadStrategy ¶ added in v0.34.0
func WithIndexHeaderLazyDownloadStrategy(strategy indexheader.LazyDownloadIndexHeaderFunc) BucketStoreOption
WithIndexHeaderLazyDownloadStrategy specifies what block to lazy download its index header. Only used when lazy mmap is enabled at the same time.
func WithLazyExpandedPostings ¶ added in v0.33.0
func WithLazyExpandedPostings(enabled bool) BucketStoreOption
WithLazyExpandedPostings enables lazy expanded postings.
func WithLogger ¶ added in v0.20.0
func WithLogger(logger log.Logger) BucketStoreOption
WithLogger sets the BucketStore logger to the one you pass.
func WithQueryGate ¶ added in v0.20.0
func WithQueryGate(queryGate gate.Gate) BucketStoreOption
WithQueryGate sets a queryGate to use instead of a noopGate.
func WithRegistry ¶ added in v0.20.0
func WithRegistry(reg prometheus.Registerer) BucketStoreOption
WithRegistry sets a registry that BucketStore uses to register metrics with.
func WithRequestLoggerFunc ¶ added in v0.36.0
func WithRequestLoggerFunc(loggerFunc RequestLoggerFunc) BucketStoreOption
WithRequestLoggerFunc sets the BucketStore to use the passed RequestLoggerFunc to initialize logger during query time.
func WithSeriesBatchSize ¶ added in v0.30.0
func WithSeriesBatchSize(seriesBatchSize int) BucketStoreOption
type BytesCounter ¶ added in v0.23.0
type BytesCounter struct { io.ReadCloser // contains filtered or unexported fields }
func NewBytesRead ¶ added in v0.23.0
func NewBytesRead(rc io.ReadCloser) *BytesCounter
func (*BytesCounter) BytesCount ¶ added in v0.23.0
func (s *BytesCounter) BytesCount() int
type BytesLimiter ¶ added in v0.30.0
type BytesLimiter interface { // Reserve bytes out of the total amount of bytes enforced by the limiter. // Returns an error if the limit has been exceeded. This function must be // goroutine safe. ReserveWithType(num uint64, dataType StoreDataType) error }
type BytesLimiterFactory ¶ added in v0.30.0
type BytesLimiterFactory func(failedCounter prometheus.Counter) BytesLimiter
BytesLimiterFactory is used to create a new BytesLimiter.
func NewBytesLimiterFactory ¶ added in v0.30.0
func NewBytesLimiterFactory(limit units.Base2Bytes) BytesLimiterFactory
NewBytesLimiterFactory makes a new BytesLimiterFactory with a static limit.
type ChunksLimiter ¶ added in v0.15.0
type ChunksLimiterFactory ¶ added in v0.15.0
type ChunksLimiterFactory func(failedCounter prometheus.Counter) ChunksLimiter
ChunksLimiterFactory is used to create a new ChunksLimiter. The factory is useful for projects depending on Thanos (eg. Cortex) which have dynamic limits.
func NewChunksLimiterFactory ¶ added in v0.15.0
func NewChunksLimiterFactory(limit uint64) ChunksLimiterFactory
NewChunksLimiterFactory makes a new ChunksLimiterFactory with a static limit.
type Client ¶
type Client interface { // StoreClient to access the store. storepb.StoreClient // LabelSets that each apply to some data exposed by the backing store. LabelSets() []labels.Labels // TimeRange returns minimum and maximum time range of data in the store. TimeRange() (mint int64, maxt int64) // TSDBInfos returns metadata about each TSDB backed by the client. TSDBInfos() []infopb.TSDBInfo // SupportsSharding returns true if sharding is supported by the underlying store. SupportsSharding() bool // SupportsWithoutReplicaLabels returns true if trimming replica labels // and sorted response is supported by the underlying store. SupportsWithoutReplicaLabels() bool // String returns the string representation of the store client. String() string // Addr returns address of the store client. If second parameter is true, the client // represents a local client (server-as-client) and has no remote address. Addr() (addr string, isLocalClient bool) }
Client holds meta information about a store.
type CloseDelegator ¶ added in v0.15.0
CloseDelegator allows to delegate close (releasing resources used by request to the server). This is useful when we invoke StoreAPI within another StoreAPI and results are ephemeral until copied.
type FilterConfig ¶ added in v0.7.0
type FilterConfig struct {
MinTime, MaxTime model.TimeOrDurationValue
}
FilterConfig is a configuration, which Store uses for filtering metrics based on time.
type Limiter ¶ added in v0.7.0
type Limiter struct {
// contains filtered or unexported fields
}
Limiter is a simple mechanism for checking if something has passed a certain threshold.
func NewLimiter ¶ added in v0.7.0
func NewLimiter(limit uint64, ctr prometheus.Counter) *Limiter
NewLimiter returns a new limiter with a specified limit. 0 disables the limit.
func (*Limiter) ReserveWithType ¶ added in v0.36.0
func (l *Limiter) ReserveWithType(num uint64, _ StoreDataType) error
type LocalStore ¶ added in v0.13.0
type LocalStore struct {
// contains filtered or unexported fields
}
LocalStore implements the store API against single file with stream of proto-based SeriesResponses in JSON format. Inefficient implementation for quick StoreAPI view. Chunk order is exactly the same as in a given file.
func NewLocalStoreFromJSONMmappableFile ¶ added in v0.13.0
func NewLocalStoreFromJSONMmappableFile( logger log.Logger, component component.StoreAPI, extLabels labels.Labels, path string, split bufio.SplitFunc, ) (*LocalStore, error)
TODO(bwplotka): Add remote read so Prometheus users can use this. Potentially after streaming will be added https://github.com/prometheus/prometheus/issues/5926. TODO(bwplotka): Consider non mmaped version of this, as well different versions.
func (*LocalStore) Close ¶ added in v0.13.0
func (s *LocalStore) Close() (err error)
func (*LocalStore) Info ¶ added in v0.13.0
func (s *LocalStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
Info returns store information about the Prometheus instance.
func (*LocalStore) LabelNames ¶ added in v0.13.0
func (s *LocalStore) LabelNames(_ context.Context, _ *storepb.LabelNamesRequest) ( *storepb.LabelNamesResponse, error, )
LabelNames returns all known label names.
func (*LocalStore) LabelValues ¶ added in v0.13.0
func (s *LocalStore) LabelValues(_ context.Context, r *storepb.LabelValuesRequest) ( *storepb.LabelValuesResponse, error, )
LabelValues returns all known label values for a given label name.
func (*LocalStore) Series ¶ added in v0.13.0
func (s *LocalStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error
Series returns all series for a requested time range and label matcher. The returned data may exceed the requested time bounds.
type NoopSeriesStatsAggregator ¶ added in v0.29.0
type NoopSeriesStatsAggregator struct{}
NoopSeriesStatsAggregator is a query performance series aggregator that does nothing.
func (*NoopSeriesStatsAggregator) Aggregate ¶ added in v0.29.0
func (s *NoopSeriesStatsAggregator) Aggregate(_ storepb.SeriesStatsCounter)
func (*NoopSeriesStatsAggregator) Observe ¶ added in v0.29.0
func (s *NoopSeriesStatsAggregator) Observe(_ float64)
type NoopSeriesStatsAggregatorFactory ¶ added in v0.32.0
type NoopSeriesStatsAggregatorFactory struct{}
NoopSeriesStatsAggregatorFactory is a query performance series aggregator factory that does nothing.
func (*NoopSeriesStatsAggregatorFactory) NewAggregator ¶ added in v0.32.0
func (s *NoopSeriesStatsAggregatorFactory) NewAggregator(tenant string) SeriesQueryPerformanceMetricsAggregator
type Partitioner ¶ added in v0.19.0
type Partitioner interface { // Partition partitions length entries into n <= length ranges that cover all // input ranges // It supports overlapping ranges. // NOTE: It expects range to be sorted by start time. Partition(length int, rng func(int) (uint64, uint64)) []Part }
func NewGapBasedPartitioner ¶ added in v0.19.0
func NewGapBasedPartitioner(maxGapSize uint64) Partitioner
type PrometheusStore ¶
type PrometheusStore struct {
// contains filtered or unexported fields
}
PrometheusStore implements the store node API on top of the Prometheus remote read API.
func NewPrometheusStore ¶
func NewPrometheusStore( logger log.Logger, reg prometheus.Registerer, client *promclient.Client, baseURL *url.URL, component component.StoreAPI, externalLabelsFn func() labels.Labels, timestamps func() (mint int64, maxt int64), promVersion func() string, ) (*PrometheusStore, error)
NewPrometheusStore returns a new PrometheusStore that uses the given HTTP client to talk to Prometheus. It attaches the provided external labels to all results. Provided external labels has to be sorted.
func (*PrometheusStore) Info ¶
func (p *PrometheusStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
Info returns store information about the Prometheus instance. NOTE(bwplotka): MaxTime & MinTime are not accurate nor adjusted dynamically. This is fine for now, but might be needed in future.
func (*PrometheusStore) LabelNames ¶
func (p *PrometheusStore) LabelNames(ctx context.Context, r *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
LabelNames returns all known label names of series that match the given matchers.
func (*PrometheusStore) LabelSet ¶ added in v0.24.0
func (p *PrometheusStore) LabelSet() []labelpb.ZLabelSet
func (*PrometheusStore) LabelValues ¶
func (p *PrometheusStore) LabelValues(ctx context.Context, r *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error)
LabelValues returns all known label values for a given label name.
func (*PrometheusStore) Series ¶
func (p *PrometheusStore) Series(r *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) error
Series returns all series for a requested time range and label matcher.
func (*PrometheusStore) TSDBInfos ¶ added in v0.32.0
func (p *PrometheusStore) TSDBInfos() []infopb.TSDBInfo
func (*PrometheusStore) Timestamps ¶ added in v0.24.0
func (p *PrometheusStore) Timestamps() (mint int64, maxt int64)
type ProxyStore ¶
type ProxyStore struct {
// contains filtered or unexported fields
}
ProxyStore implements the store API that proxies request to all given underlying stores.
func NewProxyStore ¶
func NewProxyStore( logger log.Logger, reg prometheus.Registerer, stores func() []Client, component component.StoreAPI, selectorLabels labels.Labels, responseTimeout time.Duration, retrievalStrategy RetrievalStrategy, options ...ProxyStoreOption, ) *ProxyStore
NewProxyStore returns a new ProxyStore that uses the given clients that implements storeAPI to fan-in all series to the client. Note that there is no deduplication support. Deduplication should be done on the highest level (just before PromQL).
func (*ProxyStore) Info ¶
func (s *ProxyStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
Info returns store information about the external labels this store have.
func (*ProxyStore) LabelNames ¶
func (s *ProxyStore) LabelNames(ctx context.Context, originalRequest *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error)
LabelNames returns all known label names.
func (*ProxyStore) LabelSet ¶ added in v0.24.0
func (s *ProxyStore) LabelSet() []labelpb.ZLabelSet
func (*ProxyStore) LabelValues ¶
func (s *ProxyStore) LabelValues(ctx context.Context, originalRequest *storepb.LabelValuesRequest) ( *storepb.LabelValuesResponse, error, )
LabelValues returns all known label values for a given label name.
func (*ProxyStore) Series ¶
func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error
func (*ProxyStore) TSDBInfos ¶ added in v0.32.0
func (s *ProxyStore) TSDBInfos() []infopb.TSDBInfo
func (*ProxyStore) TimeRange ¶ added in v0.24.0
func (s *ProxyStore) TimeRange() (int64, int64)
type ProxyStoreOption ¶ added in v0.32.0
type ProxyStoreOption func(s *ProxyStore)
BucketStoreOption are functions that configure BucketStore.
func WithProxyStoreDebugLogging ¶ added in v0.32.0
func WithProxyStoreDebugLogging(enable bool) ProxyStoreOption
WithProxyStoreDebugLogging toggles debug logging.
func WithTSDBSelector ¶ added in v0.35.0
func WithTSDBSelector(selector *TSDBSelector) ProxyStoreOption
WithTSDBSelector sets the TSDB selector for the proxy.
type ReadWriteTSDBStore ¶ added in v0.11.0
type ReadWriteTSDBStore struct { storepb.StoreServer storepb.WriteableStoreServer }
ReadWriteTSDBStore is a TSDBStore that can also be written to.
type RequestLoggerFunc ¶ added in v0.36.0
type RetrievalStrategy ¶ added in v0.29.0
type RetrievalStrategy string
RetrievalStrategy stores what kind of retrieval strategy shall be used for the async response set.
const ( // LazyRetrieval allows readers (e.g. PromQL engine) to use (stream) data as soon as possible. LazyRetrieval RetrievalStrategy = "lazy" // EagerRetrieval is optimized to read all into internal buffer before returning to readers (e.g. PromQL engine). // This currently preferred because: // * Both PromQL engines (old and new) want all series ASAP to make decisions. // * Querier buffers all responses when using StoreAPI internally. EagerRetrieval RetrievalStrategy = "eager" )
type SeriesLimiter ¶ added in v0.18.0
type SeriesLimiterFactory ¶ added in v0.18.0
type SeriesLimiterFactory func(failedCounter prometheus.Counter) SeriesLimiter
SeriesLimiterFactory is used to create a new SeriesLimiter.
func NewSeriesLimiterFactory ¶ added in v0.18.0
func NewSeriesLimiterFactory(limit uint64) SeriesLimiterFactory
NewSeriesLimiterFactory makes a new SeriesLimiterFactory with a static limit.
type SeriesQueryPerformanceMetricsAggregator ¶ added in v0.32.0
type SeriesQueryPerformanceMetricsAggregator interface { Aggregate(seriesStats storepb.SeriesStatsCounter) Observe(duration float64) }
type SeriesQueryPerformanceMetricsAggregatorFactory ¶ added in v0.32.0
type SeriesQueryPerformanceMetricsAggregatorFactory interface {
NewAggregator(tenant string) SeriesQueryPerformanceMetricsAggregator
}
type SeriesSelectLimits ¶ added in v0.31.0
SeriesSelectLimits are limits applied against individual Series calls.
func (*SeriesSelectLimits) RegisterFlags ¶ added in v0.31.0
func (l *SeriesSelectLimits) RegisterFlags(cmd extkingpin.FlagClause)
type StoreDataType ¶ added in v0.36.0
type StoreDataType int
const ( PostingsFetched StoreDataType = iota PostingsTouched SeriesFetched SeriesTouched ChunksFetched ChunksTouched )
type TSDBReader ¶ added in v0.15.0
type TSDBReader interface { storage.ChunkQueryable StartTime() (int64, error) }
type TSDBSelector ¶ added in v0.35.0
type TSDBSelector struct {
// contains filtered or unexported fields
}
func NewTSDBSelector ¶ added in v0.35.0
func NewTSDBSelector(relabelConfig []*relabel.Config) *TSDBSelector
func (*TSDBSelector) MatchLabelSets ¶ added in v0.35.0
MatchLabelSets returns true if the given label sets match the TSDBSelector. As a second parameter, it returns the matched label sets if they are a subset of the given input. Otherwise the second return value is nil.
type TSDBStore ¶
type TSDBStore struct {
// contains filtered or unexported fields
}
TSDBStore implements the store API against a local TSDB instance. It attaches the provided external labels to all results. It only responds with raw data and does not support downsampling.
func NewTSDBStore ¶
func NewTSDBStore(logger log.Logger, db TSDBReader, component component.StoreAPI, extLset labels.Labels) *TSDBStore
NewTSDBStore creates a new TSDBStore. NOTE: Given lset has to be sorted.
func (*TSDBStore) Info ¶
func (s *TSDBStore) Info(_ context.Context, _ *storepb.InfoRequest) (*storepb.InfoResponse, error)
Info returns store information about the Prometheus instance.
func (*TSDBStore) LabelNames ¶
func (s *TSDBStore) LabelNames(ctx context.Context, r *storepb.LabelNamesRequest) ( *storepb.LabelNamesResponse, error, )
LabelNames returns all known label names constrained with the given matchers.
func (*TSDBStore) LabelValues ¶
func (s *TSDBStore) LabelValues(ctx context.Context, r *storepb.LabelValuesRequest) ( *storepb.LabelValuesResponse, error, )
LabelValues returns all known label values for a given label name.
func (*TSDBStore) Series ¶
func (s *TSDBStore) Series(r *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) error
Series returns all series for a requested time range and label matcher. The returned data may exceed the requested time bounds.