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

564 lines
17 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
3 years ago
"fmt"
"math"
"time"
"github.com/go-kit/log"
"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/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/chunk/fetcher"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/index"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient"
"github.com/grafana/loki/pkg/storage/stores/series"
series_index "github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway"
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>
3 years ago
"github.com/grafana/loki/pkg/storage/stores/tsdb"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
"github.com/grafana/loki/pkg/util/deletion"
)
var (
indexTypeStats = usagestats.NewString("store_index_type")
objectTypeStats = usagestats.NewString("store_object_type")
schemaStats = usagestats.NewString("store_schema")
errWritingChunkUnsupported = errors.New("writing chunks is not supported while running store in read-only mode")
)
// Store is the Loki chunk store to retrieve and save chunks.
type Store interface {
stores.Store
SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error)
SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error)
Series(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error)
GetSchemaConfigs() []config.PeriodConfig
SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer)
}
type store struct {
stores.Store
composite *stores.CompositeStore
cfg Config
storeCfg config.ChunkStoreConfig
schemaCfg config.SchemaConfig
chunkMetrics *ChunkMetrics
chunkClientMetrics client.ChunkClientMetrics
clientMetrics ClientMetrics
registerer prometheus.Registerer
indexReadCache cache.Cache
chunksCache cache.Cache
writeDedupeCache cache.Cache
limits StoreLimits
logger log.Logger
chunkFilterer chunk.RequestChunkFilterer
// Keep a reference to the tsdb index store as we use one store for multiple schema period configs.
tsdbStore index.ReaderWriter
tsdbStoreStopFunc func()
}
// 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,
) (Store, 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)
if err != nil {
return nil, err
}
writeDedupeCache, err := cache.New(storeCfg.WriteDedupeCacheConfig, registerer, logger, stats.WriteDedupeCache)
if err != nil {
return nil, err
}
chunkCacheCfg := storeCfg.ChunkCacheConfig
chunkCacheCfg.Prefix = "chunks"
chunksCache, err := cache.New(chunkCacheCfg, registerer, logger, stats.ChunkCache)
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)
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 := &store{
Store: stores,
composite: stores,
cfg: cfg,
storeCfg: storeCfg,
schemaCfg: schemaCfg,
chunkClientMetrics: client.NewChunkClientMetrics(registerer),
clientMetrics: clientMetrics,
chunkMetrics: NewChunkMetrics(registerer, cfg.MaxChunkBatchSize),
registerer: registerer,
indexReadCache: indexReadCache,
chunksCache: chunksCache,
writeDedupeCache: writeDedupeCache,
logger: logger,
limits: limits,
}
if err := s.init(); err != nil {
return nil, err
}
return s, nil
}
func (s *store) init() error {
for _, p := range s.schemaCfg.Configs {
chunkClient, err := s.chunkClientForPeriod(p)
if err != nil {
return err
}
f, err := fetcher.New(s.chunksCache, s.storeCfg.ChunkCacheStubs(), s.schemaCfg, chunkClient, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackConcurrency, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackBufferSize)
if err != nil {
return err
}
w, idx, stop, err := s.storeForPeriod(p, chunkClient, f)
if err != nil {
return err
}
s.composite.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 *store) 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)
chunks, err := NewChunkClient(objectStoreType, s.cfg, s.schemaCfg, s.clientMetrics, chunkClientReg)
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 *store) storeForPeriod(p config.PeriodConfig, 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)
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>
3 years ago
if p.IndexType == config.TSDBType {
if shouldUseIndexGatewayClient(s.cfg.TSDBShipperConfig) {
// inject the index-gateway client into the index store
gw, err := gatewayclient.NewGatewayClient(s.cfg.TSDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.logger)
if err != nil {
return nil, nil, nil, err
}
idx := series.NewIndexGatewayClientStore(gw, nil)
return failingChunkWriter{}, index.NewMonitoredReaderWriter(idx, indexClientReg), func() {
f.Stop()
gw.Stop()
}, nil
}
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>
3 years ago
objectClient, err := NewObjectClient(s.cfg.TSDBShipperConfig.SharedStoreType, s.cfg, s.clientMetrics)
if err != nil {
return nil, nil, nil, err
}
var backupIndexWriter index.Writer
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
3 years ago
backupStoreStop := func() {}
if s.cfg.TSDBShipperConfig.UseBoltDBShipperAsBackup {
pCopy := p
pCopy.IndexType = config.BoltDBShipperType
pCopy.IndexTables.Prefix = fmt.Sprintf("%sbackup_", pCopy.IndexTables.Prefix)
_, backupIndexWriter, backupStoreStop, err = s.storeForPeriod(pCopy, chunkClient, f)
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
3 years ago
if err != nil {
return nil, nil, nil, err
}
}
// We should only create one tsdb.Store per storage.Store and reuse it over all TSDB schema periods.
if s.tsdbStore == nil {
indexReaderWriter, stopTSDBStoreFunc, err := tsdb.NewStore(s.cfg.TSDBShipperConfig, p, f, objectClient, s.limits,
getIndexStoreTableRanges(config.TSDBType, s.schemaCfg.Configs), backupIndexWriter, indexClientReg)
if err != nil {
return nil, nil, nil, err
}
s.tsdbStore = indexReaderWriter
s.tsdbStoreStopFunc = stopTSDBStoreFunc
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>
3 years ago
}
indexReaderWriter := index.NewMonitoredReaderWriter(s.tsdbStore, indexClientReg)
chunkWriter := stores.NewChunkWriter(f, s.schemaCfg, s.tsdbStore, 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>
3 years ago
func() {
f.Stop()
chunkClient.Stop()
s.tsdbStoreStopFunc()
objectClient.Stop()
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
3 years ago
backupStoreStop()
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>
3 years ago
}, nil
}
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
idx, err := NewIndexClient(p.IndexType, s.cfg, s.schemaCfg, s.limits, s.clientMetrics, nil, indexClientReg)
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
5 years ago
}
idx = series_index.NewCachingIndexClient(idx, s.indexReadCache, s.cfg.IndexCacheValidity, s.limits, s.logger, 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)
indexReaderWriter = index.NewMonitoredReaderWriter(indexReaderWriter, indexClientReg)
chunkWriter := stores.NewChunkWriter(f, s.schemaCfg, indexReaderWriter, s.storeCfg.DisableIndexDeduplication)
// (Sandeep): Disable IndexGatewayClientStore for stores other than tsdb until we are ready to enable it again
/*if s.cfg.BoltDBShipperConfig != nil && shouldUseIndexGatewayClient(s.cfg.BoltDBShipperConfig) {
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
3 years ago
// inject the index-gateway client into the index store
gw, err := shipper.NewGatewayClient(s.cfg.BoltDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.logger)
if err != nil {
return nil, nil, nil, err
}
indexReaderWriter = series.NewIndexGatewayClientStore(gw, indexReaderWriter)
}*/
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
3 years ago
return chunkWriter,
indexReaderWriter,
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
5 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>
5 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>
5 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>
5 years ago
return nil, 0, 0, err
}
matchers = append(matchers, nameLabelMatcher)
if err != nil {
return nil, 0, 0, err
}
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
}
func injectShardLabel(shards []string, matchers []*labels.Matcher) ([]*labels.Matcher, error) {
if shards != 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
5 years ago
parsed, err := logql.ParseShards(shards)
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
5 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
5 years ago
}
matchers = append(matchers, shardMatcher)
break // nolint:staticcheck
}
}
return matchers, nil
}
func (s *store) SetChunkFilterer(chunkFilterer chunk.RequestChunkFilterer) {
s.chunkFilterer = chunkFilterer
s.Store.SetChunkFilterer(chunkFilterer)
}
// lazyChunks is an internal function used to resolve a set of lazy chunks from the store without actually loading them. It's used internally by `LazyQuery` and `GetSeries`
func (s *store) lazyChunks(ctx context.Context, matchers []*labels.Matcher, from, through model.Time) ([]*LazyChunk, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
stats := stats.FromContext(ctx)
chks, fetchers, err := s.GetChunkRefs(ctx, userID, from, through, matchers...)
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])
}
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 *store) Series(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>
5 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 {
result[i] = logproto.SeriesIdentifier{
Labels: s.Map(),
}
}
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>
5 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.
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>
5 years ago
func (s *store) 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>
5 years ago
matchers, from, through, err := decodeReq(req)
if err != nil {
return nil, err
}
lazyChunks, err := s.lazyChunks(ctx, matchers, from, through)
if err != nil {
return nil, err
}
if len(lazyChunks) == 0 {
return iter.NoopIterator, 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>
5 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
}
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)
}
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>
5 years ago
func (s *store) 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>
5 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>
5 years ago
if err != nil {
return nil, err
}
lazyChunks, err := s.lazyChunks(ctx, matchers, from, through)
if err != nil {
return nil, err
}
if len(lazyChunks) == 0 {
return iter.NoopIterator, 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>
5 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>
5 years ago
if err != nil {
return nil, err
}
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>
5 years ago
}
func (s *store) 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
}
func getIndexStoreTableRanges(indexType string, periodicConfigs []config.PeriodConfig) config.TableRanges {
var ranges config.TableRanges
for i := range periodicConfigs {
if periodicConfigs[i].IndexType != indexType {
continue
}
periodEndTime := config.DayTime{Time: math.MaxInt64}
if i < len(periodicConfigs)-1 {
periodEndTime = config.DayTime{Time: periodicConfigs[i+1].From.Time.Add(-time.Millisecond)}
}
ranges = append(ranges, periodicConfigs[i].GetIndexTableNumberRange(periodEndTime))
}
return ranges
}