Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/storage/store.go

611 lines
19 KiB

package storage
import (
"context"
tsdb: boltdb shipper backup store tsdb (#7130) <!-- Thanks for sending a pull request! Before submitting: 1. Read our CONTRIBUTING.md guide 2. Name your PR as `<Feature Area>: Describe your change`. a. Do not end the title with punctuation. It will be added in the changelog. b. Start with an imperative verb. Example: Fix the latency between System A and System B. c. Use sentence case, not title case. d. Use a complete phrase or sentence. The PR title will appear in a changelog, so help other people understand what your change will be. 3. Rebase your PR if it gets out of sync with main --> **What this PR does / why we need it**: Since we are planning to start deploying `tsdb` to critical cells, we want to have a disaster recovery without any data loss. This PR adds support to `tsdb` store itself to configure `boltdb-shipper` as a backup chunk writer. When the flag is enabled, `tsdb` store would also index chunks in `boltdb-shipper` index store with table names set to `<configured-tsdb-table-prefix>`+`backup_`+`<table-number>`. This is the simplest option that I can come up with without maintaining a lot of code and we can simply drop the support before marking `tsdb` as production ready. The rollback/recovery plan would just be to change the schema config to use `boltdb-shipper` as index store with `<configured-tsdb-table-prefix>`+`backup_` as prefix like below: **tsdb enabled:** ``` schema_config: configs: - from: 2022-01-01 store: boltdb-shipper index: prefix: index_ - from: 2022-09-10 store: tsdb index: prefix: index_ ``` **rollback:** ``` schema_config: configs: - from: 2022-01-01 store: boltdb-shipper index: prefix: index_ - from: 2022-09-10 store: boltdb-shipper index: prefix: index_backup_ ``` <!-- Note about CHANGELOG entries, if a change adds: * an important feature * fixes an issue present in a previous release, * causes a change in operation that would be useful for an operator of Loki to know then please add a CHANGELOG entry. For documentation changes, build changes, simple fixes etc please skip this step. We are attempting to curate a changelog of the most relevant and important changes to be easier to ingest by end users of Loki. Note about the upgrade guide, if this changes: * default configuration values * metric names or label names * changes existing log lines such as the metrics.go query output line * configuration parameters * anything to do with any API * any other change that would require special attention or extra steps to upgrade Please document clearly what changed AND what needs to be done in the upgrade guide. --> **Checklist** - [x] Tests updated
4 years ago
"fmt"
"math"
"time"
"github.com/grafana/loki/v3/pkg/storage/types"
"github.com/grafana/loki/v3/pkg/util/httpreq"
lokilog "github.com/grafana/loki/v3/pkg/logql/log"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/grafana/dskit/tenant"
"github.com/grafana/loki/v3/pkg/analytics"
"github.com/grafana/loki/v3/pkg/indexgateway"
"github.com/grafana/loki/v3/pkg/iter"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
"github.com/grafana/loki/v3/pkg/logqlmodel/stats"
"github.com/grafana/loki/v3/pkg/querier/astmapper"
"github.com/grafana/loki/v3/pkg/storage/chunk"
"github.com/grafana/loki/v3/pkg/storage/chunk/cache"
"github.com/grafana/loki/v3/pkg/storage/chunk/client"
"github.com/grafana/loki/v3/pkg/storage/chunk/client/congestion"
"github.com/grafana/loki/v3/pkg/storage/chunk/fetcher"
"github.com/grafana/loki/v3/pkg/storage/config"
"github.com/grafana/loki/v3/pkg/storage/stores"
"github.com/grafana/loki/v3/pkg/storage/stores/index"
"github.com/grafana/loki/v3/pkg/storage/stores/series"
series_index "github.com/grafana/loki/v3/pkg/storage/stores/series/index"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb"
"github.com/grafana/loki/v3/pkg/util"
"github.com/grafana/loki/v3/pkg/util/deletion"
)
var (
indexTypeStats = analytics.NewString("store_index_type")
objectTypeStats = analytics.NewString("store_object_type")
schemaStats = analytics.NewString("store_schema")
errWritingChunkUnsupported = errors.New("writing chunks is not supported while running store in read-only mode")
)
type SelectStore interface {
SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error)
SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error)
SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error)
}
type SchemaConfigProvider interface {
GetSchemaConfigs() []config.PeriodConfig
}
type Instrumentable interface {
SetExtractorWrapper(wrapper lokilog.SampleExtractorWrapper)
SetPipelineWrapper(wrapper lokilog.PipelineWrapper)
}
type Store interface {
stores.Store
SelectStore
SchemaConfigProvider
Instrumentable
}
type LokiStore struct {
stores.Store
cfg Config
storeCfg config.ChunkStoreConfig
schemaCfg config.SchemaConfig
chunkMetrics *ChunkMetrics
chunkClientMetrics client.ChunkClientMetrics
clientMetrics ClientMetrics
registerer prometheus.Registerer
indexReadCache cache.Cache
chunksCache cache.Cache
chunksCacheL2 cache.Cache
writeDedupeCache cache.Cache
limits StoreLimits
logger log.Logger
Dynamic client-side throttling to avoid object storage rate-limits (GCS only) (#10140) **What this PR does / why we need it**: Across the various cloud providers' object storage services, there are different rate-limits implemented. Rate-limits can be imposed under multiple conditions, such as server-side scale up (ramping up from low volume to high, "local" limit), reaching some defined upper limit ("absolute" limit), etc. We cannot know apriori when these rate-limits will be imposed, so we can't set up a client-side limiter to only allow a certain number of requests through per second. Additionally, that would require global coordination between queriers - which is difficult. With the above constraints, I have instead taken inspiration from TCP's [congestion control algorithms](https://en.wikipedia.org/wiki/TCP_congestion_control). This PR implements [AIMD](https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease) (Additive Increase, Multiplicative Decrease), which is used in the congestion _avoidance_ phase of congestion control. The default window size (object store requests per second) is 2000; in other words, we skip the "slow start" phase. The controller uses the Go [`rate.Limiter`](https://pkg.go.dev/golang.org/x/time/rate), which implements the token-bucket algorithm. To put it simply: - every successful request widens the window (per second client rate-limit) - every rate-limited response reduces the window size by a backoff factor (0.5 by default, so it will halve) - when the limit has been reached, the querier will be delayed from making further requests until tokens are available
3 years ago
chunkFilterer chunk.RequestChunkFilterer
extractorWrapper lokilog.SampleExtractorWrapper
pipelineWrapper lokilog.PipelineWrapper
Dynamic client-side throttling to avoid object storage rate-limits (GCS only) (#10140) **What this PR does / why we need it**: Across the various cloud providers' object storage services, there are different rate-limits implemented. Rate-limits can be imposed under multiple conditions, such as server-side scale up (ramping up from low volume to high, "local" limit), reaching some defined upper limit ("absolute" limit), etc. We cannot know apriori when these rate-limits will be imposed, so we can't set up a client-side limiter to only allow a certain number of requests through per second. Additionally, that would require global coordination between queriers - which is difficult. With the above constraints, I have instead taken inspiration from TCP's [congestion control algorithms](https://en.wikipedia.org/wiki/TCP_congestion_control). This PR implements [AIMD](https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease) (Additive Increase, Multiplicative Decrease), which is used in the congestion _avoidance_ phase of congestion control. The default window size (object store requests per second) is 2000; in other words, we skip the "slow start" phase. The controller uses the Go [`rate.Limiter`](https://pkg.go.dev/golang.org/x/time/rate), which implements the token-bucket algorithm. To put it simply: - every successful request widens the window (per second client rate-limit) - every rate-limited response reduces the window size by a backoff factor (0.5 by default, so it will halve) - when the limit has been reached, the querier will be delayed from making further requests until tokens are available
3 years ago
congestionControllerFactory func(cfg congestion.Config, logger log.Logger, metrics *congestion.Metrics) congestion.Controller
metricsNamespace string
}
// NewStore creates a new Loki Store using configuration supplied.
func NewStore(cfg Config, storeCfg config.ChunkStoreConfig, schemaCfg config.SchemaConfig,
limits StoreLimits, clientMetrics ClientMetrics, registerer prometheus.Registerer, logger log.Logger,
metricsNamespace string,
) (*LokiStore, error) {
if len(schemaCfg.Configs) != 0 {
if index := config.ActivePeriodConfig(schemaCfg.Configs); index != -1 && index < len(schemaCfg.Configs) {
indexTypeStats.Set(schemaCfg.Configs[index].IndexType)
objectTypeStats.Set(schemaCfg.Configs[index].ObjectType)
schemaStats.Set(schemaCfg.Configs[index].Schema)
}
}
indexReadCache, err := cache.New(cfg.IndexQueriesCacheConfig, registerer, logger, stats.IndexCache, metricsNamespace)
if err != nil {
return nil, err
}
if cache.IsCacheConfigured(storeCfg.WriteDedupeCacheConfig) {
level.Warn(logger).Log("msg", "write dedupe cache is deprecated along with legacy index types. Consider using TSDB index which does not require a write dedupe cache.")
}
writeDedupeCache, err := cache.New(storeCfg.WriteDedupeCacheConfig, registerer, logger, stats.WriteDedupeCache, metricsNamespace)
if err != nil {
return nil, err
}
chunkCacheCfg := storeCfg.ChunkCacheConfig
chunkCacheCfg.Prefix = "chunks"
chunksCache, err := cache.New(chunkCacheCfg, registerer, logger, stats.ChunkCache, metricsNamespace)
if err != nil {
return nil, err
}
chunkCacheCfgL2 := storeCfg.ChunkCacheConfigL2
chunkCacheCfgL2.Prefix = "chunksl2"
// TODO(E.Welch) would we want to disambiguate this cache in the stats? I think not but we'd need to change stats.ChunkCache to do so.
chunksCacheL2, err := cache.New(chunkCacheCfgL2, registerer, logger, stats.ChunkCache, metricsNamespace)
if err != nil {
return nil, err
}
// Cache is shared by multiple stores, which means they will try and Stop
// it more than once. Wrap in a StopOnce to prevent this.
indexReadCache = cache.StopOnce(indexReadCache)
chunksCache = cache.StopOnce(chunksCache)
chunksCacheL2 = cache.StopOnce(chunksCacheL2)
writeDedupeCache = cache.StopOnce(writeDedupeCache)
// Lets wrap all caches except chunksCache with CacheGenMiddleware to facilitate cache invalidation using cache generation numbers.
// chunksCache is not wrapped because chunks content can't be anyways modified without changing its ID so there is no use of
// invalidating chunks cache. Also chunks can be fetched only by their ID found in index and we are anyways removing the index and invalidating index cache here.
indexReadCache = cache.NewCacheGenNumMiddleware(indexReadCache)
writeDedupeCache = cache.NewCacheGenNumMiddleware(writeDedupeCache)
err = schemaCfg.Load()
if err != nil {
return nil, errors.Wrap(err, "error loading schema config")
}
stores := stores.NewCompositeStore(limits)
s := &LokiStore{
Store: stores,
cfg: cfg,
storeCfg: storeCfg,
schemaCfg: schemaCfg,
Dynamic client-side throttling to avoid object storage rate-limits (GCS only) (#10140) **What this PR does / why we need it**: Across the various cloud providers' object storage services, there are different rate-limits implemented. Rate-limits can be imposed under multiple conditions, such as server-side scale up (ramping up from low volume to high, "local" limit), reaching some defined upper limit ("absolute" limit), etc. We cannot know apriori when these rate-limits will be imposed, so we can't set up a client-side limiter to only allow a certain number of requests through per second. Additionally, that would require global coordination between queriers - which is difficult. With the above constraints, I have instead taken inspiration from TCP's [congestion control algorithms](https://en.wikipedia.org/wiki/TCP_congestion_control). This PR implements [AIMD](https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease) (Additive Increase, Multiplicative Decrease), which is used in the congestion _avoidance_ phase of congestion control. The default window size (object store requests per second) is 2000; in other words, we skip the "slow start" phase. The controller uses the Go [`rate.Limiter`](https://pkg.go.dev/golang.org/x/time/rate), which implements the token-bucket algorithm. To put it simply: - every successful request widens the window (per second client rate-limit) - every rate-limited response reduces the window size by a backoff factor (0.5 by default, so it will halve) - when the limit has been reached, the querier will be delayed from making further requests until tokens are available
3 years ago
congestionControllerFactory: congestion.NewController,
chunkClientMetrics: client.NewChunkClientMetrics(registerer),
clientMetrics: clientMetrics,
chunkMetrics: NewChunkMetrics(registerer, cfg.MaxChunkBatchSize),
registerer: registerer,
indexReadCache: indexReadCache,
chunksCache: chunksCache,
chunksCacheL2: chunksCacheL2,
writeDedupeCache: writeDedupeCache,
logger: logger,
limits: limits,
metricsNamespace: metricsNamespace,
}
if err := s.init(); err != nil {
return nil, err
}
return s, nil
}
func (s *LokiStore) init() error {
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
for i, p := range s.schemaCfg.Configs {
p := p
chunkClient, err := s.chunkClientForPeriod(p)
if err != nil {
return err
}
f, err := fetcher.New(s.chunksCache, s.chunksCacheL2, s.storeCfg.ChunkCacheStubs(), s.schemaCfg, chunkClient, s.storeCfg.L2ChunkCacheHandoff)
if err != nil {
return err
}
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
periodEndTime := config.DayTime{Time: math.MaxInt64}
if i < len(s.schemaCfg.Configs)-1 {
periodEndTime = config.DayTime{Time: s.schemaCfg.Configs[i+1].From.Time.Add(-time.Millisecond)}
}
w, idx, stop, err := s.storeForPeriod(p, p.GetIndexTableNumberRange(periodEndTime), chunkClient, f)
if err != nil {
return err
}
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
// s.Store is always assigned the CompositeStore implementation of the Store interface
s.Store.(*stores.CompositeStore).AddStore(p.From.Time, f, idx, w, stop)
}
if s.cfg.EnableAsyncStore {
s.Store = NewAsyncStore(s.cfg.AsyncStoreConfig, s.Store, s.schemaCfg)
}
return nil
}
func (s *LokiStore) chunkClientForPeriod(p config.PeriodConfig) (client.Client, error) {
objectStoreType := p.ObjectType
if objectStoreType == "" {
objectStoreType = p.IndexType
}
chunkClientReg := prometheus.WrapRegistererWith(
prometheus.Labels{"component": "chunk-store-" + p.From.String()}, s.registerer)
Dynamic client-side throttling to avoid object storage rate-limits (GCS only) (#10140) **What this PR does / why we need it**: Across the various cloud providers' object storage services, there are different rate-limits implemented. Rate-limits can be imposed under multiple conditions, such as server-side scale up (ramping up from low volume to high, "local" limit), reaching some defined upper limit ("absolute" limit), etc. We cannot know apriori when these rate-limits will be imposed, so we can't set up a client-side limiter to only allow a certain number of requests through per second. Additionally, that would require global coordination between queriers - which is difficult. With the above constraints, I have instead taken inspiration from TCP's [congestion control algorithms](https://en.wikipedia.org/wiki/TCP_congestion_control). This PR implements [AIMD](https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease) (Additive Increase, Multiplicative Decrease), which is used in the congestion _avoidance_ phase of congestion control. The default window size (object store requests per second) is 2000; in other words, we skip the "slow start" phase. The controller uses the Go [`rate.Limiter`](https://pkg.go.dev/golang.org/x/time/rate), which implements the token-bucket algorithm. To put it simply: - every successful request widens the window (per second client rate-limit) - every rate-limited response reduces the window size by a backoff factor (0.5 by default, so it will halve) - when the limit has been reached, the querier will be delayed from making further requests until tokens are available
3 years ago
var cc congestion.Controller
ccCfg := s.cfg.CongestionControl
if ccCfg.Enabled {
cc = s.congestionControllerFactory(
ccCfg,
s.logger,
congestion.NewMetrics(fmt.Sprintf("%s-%s", objectStoreType, p.From.String()), ccCfg),
)
}
chunks, err := NewChunkClient(objectStoreType, s.cfg, s.schemaCfg, cc, chunkClientReg, s.clientMetrics, s.logger)
if err != nil {
return nil, errors.Wrap(err, "error creating object client")
}
chunks = client.NewMetricsChunkClient(chunks, s.chunkClientMetrics)
return chunks, nil
}
func shouldUseIndexGatewayClient(cfg indexshipper.Config) bool {
if cfg.Mode != indexshipper.ModeReadOnly || cfg.IndexGatewayClientConfig.Disabled {
return false
}
gatewayCfg := cfg.IndexGatewayClientConfig
if gatewayCfg.Mode == indexgateway.SimpleMode && gatewayCfg.Address == "" {
return false
}
return true
}
func (s *LokiStore) storeForPeriod(p config.PeriodConfig, tableRange config.TableRange, chunkClient client.Client, f *fetcher.Fetcher) (stores.ChunkWriter, index.ReaderWriter, func(), error) {
indexClientReg := prometheus.WrapRegistererWith(
prometheus.Labels{
"component": fmt.Sprintf(
"index-store-%s-%s",
p.IndexType,
p.From.String(),
),
}, s.registerer)
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
indexClientLogger := log.With(s.logger, "index-store", fmt.Sprintf("%s-%s", p.IndexType, p.From.String()))
if p.IndexType == types.TSDBType {
if shouldUseIndexGatewayClient(s.cfg.TSDBShipperConfig) {
// inject the index-gateway client into the index store
gw, err := indexgateway.NewGatewayClient(s.cfg.TSDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.limits, indexClientLogger, s.metricsNamespace)
if err != nil {
return nil, nil, nil, err
}
idx := series.NewIndexGatewayClientStore(gw, indexClientLogger)
return failingChunkWriter{}, index.NewMonitoredReaderWriter(idx, indexClientReg), func() {
f.Stop()
gw.Stop()
}, nil
}
Remove shared_store and shared_store_key_prefix from shipper and compactor (#10840) **What this PR does / why we need it**: #### Removes `shared_store` and `shared_store_key_prefix` from index shipper and compactor configs and their corresponding CLI flags. - `-tsdb.shipper.shared-store` - `-boltdb.shipper.shared-store` - `-tsdb.shipper.shared-store.key-prefix` - `-boltdb.shipper.shared-store.key-prefix` - `-boltdb.shipper.compactor.shared-store` - `-boltdb.shipper.compactor.shared-store.key-prefix` `shared_store` has been a confusing option allowing users to easily misconfigure Loki. Going forward `object_store` setting in the [period_config](https://grafana.com/docs/loki/latest/configure/#period_config) (which already configured the store for chunks) will be used to configure store for the index. And the newly added `path_prefix` option under the `index` key in `period_config` will configure the path under which index tables are stored. This change enforces chunks and index files for a given period reside together in the same storage bucket. More details in the upgrade guide. --- `-compactor.delete-request-store` has to be **explicitly configured** going forward. Without setting this, loki wouldn't know which object store to use for storing delete requests. Path prefix for storing deletes is decided by `-compactor.delete-request-store.key-prefix` which defaults to `index/`. **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [X] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
3 years ago
objectClient, err := NewObjectClient(p.ObjectType, s.cfg, s.clientMetrics)
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
if err != nil {
return nil, nil, nil, err
}
name := fmt.Sprintf("%s_%s", p.ObjectType, p.From.String())
indexReaderWriter, stopTSDBStoreFunc, err := tsdb.NewStore(name, p.IndexTables.PathPrefix, s.cfg.TSDBShipperConfig, s.schemaCfg, f, objectClient, s.limits, tableRange, indexClientReg, indexClientLogger)
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
if err != nil {
return nil, nil, nil, err
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
}
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
indexReaderWriter = index.NewMonitoredReaderWriter(indexReaderWriter, indexClientReg)
chunkWriter := stores.NewChunkWriter(f, s.schemaCfg, indexReaderWriter, s.storeCfg.DisableIndexDeduplication)
return chunkWriter, indexReaderWriter,
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
func() {
f.Stop()
chunkClient.Stop()
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
stopTSDBStoreFunc()
objectClient.Stop()
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
}, nil
}
idx, err := NewIndexClient(p, tableRange, s.cfg, s.schemaCfg, s.limits, s.clientMetrics, nil, indexClientReg, indexClientLogger, s.metricsNamespace)
if err != nil {
return nil, nil, nil, errors.Wrap(err, "error creating index client")
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
idx = series_index.NewCachingIndexClient(idx, s.indexReadCache, s.cfg.IndexCacheValidity, s.limits, indexClientLogger, s.cfg.DisableBroadIndexQueries)
schema, err := series_index.CreateSchema(p)
if err != nil {
return nil, nil, nil, err
}
if s.storeCfg.CacheLookupsOlderThan != 0 {
schema = series_index.NewSchemaCaching(schema, time.Duration(s.storeCfg.CacheLookupsOlderThan))
}
indexReaderWriter := series.NewIndexReaderWriter(s.schemaCfg, schema, idx, f, s.cfg.MaxChunkBatchSize, s.writeDedupeCache)
monitoredReaderWriter := index.NewMonitoredReaderWriter(indexReaderWriter, indexClientReg)
chunkWriter := stores.NewChunkWriter(f, s.schemaCfg, monitoredReaderWriter, s.storeCfg.DisableIndexDeduplication)
return chunkWriter,
monitoredReaderWriter,
func() {
chunkClient.Stop()
f.Stop()
idx.Stop()
},
nil
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
// decodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher,
// and adds the "__cortex_shard__" label if this is a sharded query.
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
// todo(cyriltovena) refactor this.
func decodeReq(req logql.QueryParams) ([]*labels.Matcher, model.Time, model.Time, error) {
expr, err := req.LogSelector()
if err != nil {
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
return nil, 0, 0, err
}
matchers := expr.Matchers()
nameLabelMatcher, err := labels.NewMatcher(labels.MatchEqual, labels.MetricName, "logs")
if err != nil {
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
return nil, 0, 0, err
}
matchers = append(matchers, nameLabelMatcher)
matchers, err = injectShardLabel(req.GetShards(), matchers)
if err != nil {
return nil, 0, 0, err
}
from, through := util.RoundToMilliseconds(req.GetStart(), req.GetEnd())
return matchers, from, through, nil
}
// TODO(owen-d): refactor this. Injecting shard labels via matchers is a big hack and we shouldn't continue
// doing it, _but_ it requires adding `fingerprintfilter` support to much of our storage interfaces
// or a way to transform the base store into a more specialized variant.
func injectShardLabel(shards []string, matchers []*labels.Matcher) ([]*labels.Matcher, error) {
if shards != nil {
parsed, _, err := logql.ParseShards(shards)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
if err != nil {
return nil, err
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
for _, s := range parsed {
shardMatcher, err := labels.NewMatcher(
labels.MatchEqual,
astmapper.ShardLabel,
s.String(),
)
if err != nil {
return nil, err
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
matchers = append(matchers, shardMatcher)
break // nolint:staticcheck
}
}
return matchers, nil
}
func (s *LokiStore) SetChunkFilterer(chunkFilterer chunk.RequestChunkFilterer) {
s.chunkFilterer = chunkFilterer
s.Store.SetChunkFilterer(chunkFilterer)
}
func (s *LokiStore) SetExtractorWrapper(wrapper lokilog.SampleExtractorWrapper) {
s.extractorWrapper = wrapper
}
func (s *LokiStore) SetPipelineWrapper(wrapper lokilog.PipelineWrapper) {
s.pipelineWrapper = wrapper
}
// lazyChunks is an internal function used to resolve a set of lazy chunks from the store without actually loading them.
func (s *LokiStore) lazyChunks(
ctx context.Context,
from, through model.Time,
predicate chunk.Predicate,
storeChunksOverride *logproto.ChunkRefGroup,
) ([]*LazyChunk, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
stats := stats.FromContext(ctx)
start := time.Now()
chks, fetchers, err := s.GetChunks(ctx, userID, from, through, predicate, storeChunksOverride)
stats.AddChunkRefsFetchTime(time.Since(start))
if err != nil {
return nil, err
}
var prefiltered int
var filtered int
for i := range chks {
prefiltered += len(chks[i])
stats.AddChunksRef(int64(len(chks[i])))
chks[i] = filterChunksByTime(from, through, chks[i])
filtered += len(chks[i])
}
if storeChunksOverride != nil {
s.chunkMetrics.refsBypassed.Add(float64(len(storeChunksOverride.Refs)))
}
s.chunkMetrics.refs.WithLabelValues(statusDiscarded).Add(float64(prefiltered - filtered))
s.chunkMetrics.refs.WithLabelValues(statusMatched).Add(float64(filtered))
// creates lazychunks with chunks ref.
lazyChunks := make([]*LazyChunk, 0, filtered)
for i := range chks {
for _, c := range chks[i] {
lazyChunks = append(lazyChunks, &LazyChunk{Chunk: c, Fetcher: fetchers[i]})
}
}
return lazyChunks, nil
}
func (s *LokiStore) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
var from, through model.Time
var matchers []*labels.Matcher
// The Loki parser doesn't allow for an empty label matcher but for the Series API
// we allow this to select all series in the time range.
if req.Selector == "" {
from, through = util.RoundToMilliseconds(req.Start, req.End)
nameLabelMatcher, err := labels.NewMatcher(labels.MatchEqual, labels.MetricName, "logs")
if err != nil {
return nil, err
}
matchers = []*labels.Matcher{nameLabelMatcher}
matchers, err = injectShardLabel(req.GetShards(), matchers)
if err != nil {
return nil, err
}
} else {
var err error
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
matchers, from, through, err = decodeReq(req)
if err != nil {
return nil, err
}
}
series, err := s.Store.GetSeries(ctx, userID, from, through, matchers...)
if err != nil {
return nil, err
}
result := make([]logproto.SeriesIdentifier, len(series))
for i, s := range series {
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
result[i] = logproto.SeriesIdentifierFromLabels(s)
}
return result, nil
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
// SelectLogs returns an iterator that will query the store for more chunks while iterating instead of fetching all chunks upfront
// for that request.
func (s *LokiStore) SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error) {
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
matchers, from, through, err := decodeReq(req)
if err != nil {
return nil, err
}
lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan), req.GetStoreChunks())
if err != nil {
return nil, err
}
if len(lazyChunks) == 0 {
return iter.NoopEntryIterator, nil
}
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
expr, err := req.LogSelector()
if err != nil {
return nil, err
}
pipeline, err := expr.Pipeline()
if err != nil {
return nil, err
}
pipeline, err = deletion.SetupPipeline(req, pipeline)
if err != nil {
return nil, err
}
if s.pipelineWrapper != nil && httpreq.ExtractHeader(ctx, httpreq.LokiDisablePipelineWrappersHeader) != "true" {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
pipeline = s.pipelineWrapper.Wrap(ctx, pipeline, req.Plan.String(), userID)
}
var chunkFilterer chunk.Filterer
if s.chunkFilterer != nil {
chunkFilterer = s.chunkFilterer.ForRequest(ctx)
}
return newLogBatchIterator(ctx, s.schemaCfg, s.chunkMetrics, lazyChunks, s.cfg.MaxChunkBatchSize, matchers, pipeline, req.Direction, req.Start, req.End, chunkFilterer)
}
func (s *LokiStore) SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error) {
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
6 years ago
matchers, from, through, err := decodeReq(req)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
if err != nil {
return nil, err
}
lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan), req.GetStoreChunks())
if err != nil {
return nil, err
}
if len(lazyChunks) == 0 {
return iter.NoopSampleIterator, nil
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
expr, err := req.Expr()
if err != nil {
return nil, err
}
extractor, err := expr.Extractor()
if err != nil {
return nil, err
}
extractor, err = deletion.SetupExtractor(req, extractor)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
if err != nil {
return nil, err
}
if s.extractorWrapper != nil && httpreq.ExtractHeader(ctx, httpreq.LokiDisablePipelineWrappersHeader) != "true" {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
extractor = s.extractorWrapper.Wrap(ctx, extractor, req.Plan.String(), userID)
}
var chunkFilterer chunk.Filterer
if s.chunkFilterer != nil {
chunkFilterer = s.chunkFilterer.ForRequest(ctx)
}
return newSampleBatchIterator(ctx, s.schemaCfg, s.chunkMetrics, lazyChunks, s.cfg.MaxChunkBatchSize, matchers, extractor, req.Start, req.End, chunkFilterer)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
}
func (s *LokiStore) GetSchemaConfigs() []config.PeriodConfig {
return s.schemaCfg.Configs
}
func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk.Chunk {
filtered := make([]chunk.Chunk, 0, len(chunks))
for _, chunk := range chunks {
if chunk.Through < from || through < chunk.From {
continue
}
filtered = append(filtered, chunk)
}
return filtered
}
type failingChunkWriter struct{}
func (f failingChunkWriter) Put(_ context.Context, _ []chunk.Chunk) error {
return errWritingChunkUnsupported
}
func (f failingChunkWriter) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) error {
return errWritingChunkUnsupported
}