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/async_store.go

278 lines
8.9 KiB

package storage
import (
"context"
"fmt"
"time"
"github.com/opentracing/opentracing-go"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/fetcher"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/index/stats"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/spanlogger"
)
type IngesterQuerier interface {
GetChunkIDs(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([]string, error)
Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error)
Volume(ctx context.Context, userID string, from, through model.Time, limit int32, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error)
}
type AsyncStoreCfg struct {
IngesterQuerier IngesterQuerier
// QueryIngestersWithin defines maximum lookback beyond which ingesters are not queried for chunk ids.
QueryIngestersWithin time.Duration
}
// AsyncStore does querying to both ingesters and chunk store and combines the results after deduping them.
// This should be used when using an async store like boltdb-shipper.
// AsyncStore is meant to be used only in queriers or any other service other than ingesters.
// It should never be used in ingesters otherwise it would start spiraling around doing queries over and over again to other ingesters.
type AsyncStore struct {
stores.Store
scfg config.SchemaConfig
ingesterQuerier IngesterQuerier
queryIngestersWithin time.Duration
}
func NewAsyncStore(cfg AsyncStoreCfg, store stores.Store, scfg config.SchemaConfig) *AsyncStore {
return &AsyncStore{
Store: store,
Simpler new chunk key v12 (#5054) * starts hacking new chunk paths * Create new chunk key for S3; update parsing and add basic test and benchmark Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Fix `TestGrpcStore` to work with new chunk key structure Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Add a function to SchemaConfig that returns the correct PeriodConfig for a given time. Signed-off-by: Callum Styan <callumstyan@gmail.com> Add schema config to object client so it we can use the right external key function based on the schema version. Signed-off-by: Callum Styan <callumstyan@gmail.com> Fix failing compactor tests by passing SchemaConfig to chunkClient Remove empty conditional from SchemaForTime Define schema v12; wire-in ChunkPathShardFactor and ChunkPathPeriod as configurable values Add PeriodConfig test steps for new values ChunkPathShardFactor and ChunkPathPeriod in v12 Update test for chunk.NewExternalKey(); remove completed TODO Set defaults for new ChunkPathPeriod and ChunkPathShardFactor SchemaConfig values; change FSObjectClient to use IdentityEncoder instead of Base64Encoder Use IdentityEncoder everywhere we use FSObjectClient for Chunks Add ExternalKey() function to SchemaConfig; update ObjectClient for Chunks * Finish plumbing through the chunk.ExternalKey -> schemaConfig.ExternalKey change. Signed-off-by: Callum Styan <callumstyan@gmail.com> * Clean up lint failures Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Clean up chunk.go and fix tests Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Break SchemaConfig.ExternalKey() into smaller functions Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Quickly fix variable name in SchemaConfig.newerExternalKey() Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Clean up ExternalKey conditional logic; add better comments; add tests Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Fix a bug where we are prepending userID to legacy Chunk keys but never parsing it; refactor key tests and benchmarks Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Add small SchemaConfig test Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Update docs and CHANGELOG.md Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Correctly return an error when failing to parse a chunk external key Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Revert IdentityEncoder to Base64Encoder after testing Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Fix assignment in test for linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Remove leftover comments from development Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Change v12 version comment format; remove redundant login in `ParseExternalKey()` Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Change remaining v12 comment style Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Pass chunk.SchemaConfig to new parallel chunk client * Simplify chunk external key prefixes for schema v12 * Fix broken benchmark; add benchmarks for old parsing conditional Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Remove unneeded lines from upgrading doc * Add benchmarks for root chunk external key parsing functions Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * memoizes schema number calculations * Resolve linter issue with PeriodConfig receiver name Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
3 years ago
scfg: scfg,
ingesterQuerier: cfg.IngesterQuerier,
queryIngestersWithin: cfg.QueryIngestersWithin,
}
}
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
// queryIngesters uses the queryIngestersWithin flag but will always query them when it's 0.
func (a *AsyncStore) shouldQueryIngesters(through, now model.Time) bool {
// don't query ingesters if the query does not overlap with queryIngestersWithin.
return a.queryIngestersWithin == 0 || through.After(now.Add(-a.queryIngestersWithin))
}
func (a *AsyncStore) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) {
spanLogger := spanlogger.FromContext(ctx)
errs := make(chan error)
var storeChunks [][]chunk.Chunk
var fetchers []*fetcher.Fetcher
go func() {
var err error
storeChunks, fetchers, err = a.Store.GetChunks(ctx, userID, from, through, predicate)
errs <- err
}()
var ingesterChunks []string
go func() {
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
if !a.shouldQueryIngesters(through, model.Now()) {
level.Debug(util_log.Logger).Log("msg", "skipping querying ingesters for chunk ids", "query-from", from, "query-through", through)
errs <- nil
return
}
var err error
ingesterChunks, err = a.ingesterQuerier.GetChunkIDs(ctx, from, through, predicate.Matchers...)
if err == nil {
level.Debug(spanLogger).Log("ingester-chunks-count", len(ingesterChunks))
level.Debug(util_log.Logger).Log("msg", "got chunk ids from ingester", "count", len(ingesterChunks))
}
errs <- err
}()
for i := 0; i < 2; i++ {
err := <-errs
if err != nil {
return nil, nil, err
}
}
if len(ingesterChunks) == 0 {
return storeChunks, fetchers, nil
}
return a.mergeIngesterAndStoreChunks(userID, storeChunks, fetchers, ingesterChunks)
}
func (a *AsyncStore) Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) {
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
logger := util_log.WithContext(ctx, util_log.Logger)
matchersStr := syntax.MatchersString(matchers)
type f func() (*stats.Stats, error)
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
var jobs []f
if a.shouldQueryIngesters(through, model.Now()) {
jobs = append(jobs, f(func() (*stats.Stats, error) {
stats, err := a.ingesterQuerier.Stats(ctx, userID, from, through, matchers...)
level.Debug(logger).Log(
append(
stats.LoggingKeyValues(),
"msg", "queried statistics",
"matchers", matchersStr,
"source", "ingesters",
)...,
)
return stats, err
}))
}
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
jobs = append(jobs, f(func() (*stats.Stats, error) {
stats, err := a.Store.Stats(ctx, userID, from, through, matchers...)
level.Debug(logger).Log(
append(
stats.LoggingKeyValues(),
"msg", "queried statistics",
"matchers", matchersStr,
"source", "store",
)...,
)
return stats, err
}))
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
resps := make([]*stats.Stats, len(jobs))
if err := concurrency.ForEachJob(
ctx,
len(jobs),
fixes bug with queryIngesterWithin logic in asyncStore ingester stats… (#8145) Fixes a previous mistake in the logic calculating when to skip querying ingesters in the async store Statistics method. Notably `through.After` should be `through.Before` when skipping querying ingesters as that's when there's no overlap with the `query-ingesters-within` period: ```go // OLD CODE BELOW if a.queryIngestersWithin != 0 { // don't query ingesters if the query does not overlap with queryIngestersWithin. if !through.After(model.Now().Add(-a.queryIngestersWithin)) { // <----- should be through.Before return a.Store.Stats(ctx, userID, from, through, matchers...) } } ``` I discovered the problem while debugging querier OOMs during a boltdb-shipper -> tsdb migration and ultimately found this happened under the following circumstances: * Queries over high volumes of _recent_ data wouldn't query ingesters for index metadata * Without index metadata, we only checked storage metadata * There is a delay before we ship the index to storage, meaning we don't see it if ingesters are skipped * Calculating ideal shard factors without recent data for queries that only touch recent data _dramatically_ underestimates the desired shard factor * Queries aren't split enough and get scheduled onto too few querier replicas * They oom. We had some fun examples like a single replica trying to download 419,000 chunks/query To be clear, this is still a hypothesis, but a plausible one, especially after finding the boolean logic error this PR fixes. Instead of changing this one line, I took the opportunity to refactor this into a shared utility used by our other `GetChunkRefs` method which is already tested, ensuring the logic works as expected. I also added some more logging visibility into this code so we can understand what the difference is when querying statistics from storage vs ingesters. Finally, I added a helper to prepare our `Stats` objects to be logged which is now used in a few places.
2 years ago
len(jobs),
func(ctx context.Context, i int) error {
resp, err := jobs[i]()
resps[i] = resp
return err
},
); err != nil {
return nil, err
}
Add config to avoid caching stats for recent data (#9537) **What this PR does / why we need it**: When we query the stats for recent data, we query both the ingesters and the index gateways for the stats. https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L112-L114 https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L126-L127 Then we merge all the responses, which means summing up all the stats https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L157-L158 https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/stores/index/stats/stats.go#L23-L26 Because we have a replication factor of 3, this means that we will get the stats from the ingesters repeated up to 3 times, hence inflating the stats. In the stats cache, we store the stats for a given matcher set for the whole day, then we extract the stats from the cache by the factor of time from the request that is stored in the cache: https://github.com/grafana/loki/blob/336283acadb34f5fda9abce4e6fcef1dca9965d8/pkg/querier/queryrange/index_stats_cache.go#L33 https://github.com/grafana/loki/blob/336283acadb34f5fda9abce4e6fcef1dca9965d8/pkg/querier/queryrange/index_stats_cache.go#L40 Inflated stats for recent data will be cached, so subsequent stats extracted from the cache will be inflated regardless of the time. This PR adds a new per-tenant limit `max_stats_cache_freshness` to not cache requests with an end time that falls within Now minus this duration. Here's a scenario illustrating this. The graphs below show the bytes stats queried in the sharding middleware. We are running a log filter query that won't match any log, every 5 seconds with a length of 3h. ![image](https://github.com/grafana/loki/assets/8354290/45c2e6e9-185c-4a18-b290-47da27fc3e39) As can be seen, after enabling the stats cache and configuring`do_not_cache_request_within` to not cache stats for requests within 30m, the bytes stats used in the sharding middleware stopped increasing. In both cases the stats cache hit ration was 100%. ![image](https://github.com/grafana/loki/assets/8354290/cd35bcb8-0c77-4693-a06b-502741fd6e23) **Special notes for your reviewer**: - Blocked by https://github.com/grafana/loki/pull/9535 - Note that this PR doesn't fix the root issue of inflated stats form the ingesters, but rather buys us some time to work on that. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
// TODO: fix inflated stats. This happens because:
// - All ingesters are queried. Since we have a replication factor of 3, we get 3x the stats.
// - For the same timespan, we are querying the store as well. This means we can get duplicated stats for
// chunks that are already in the store but also still in the ingesters.
merged := stats.MergeStats(resps...)
return &merged, nil
}
func (a *AsyncStore) Volume(ctx context.Context, userID string, from, through model.Time, limit int32, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "AsyncStore.Volume")
defer sp.Finish()
logger := util_log.WithContext(ctx, util_log.Logger)
matchersStr := syntax.MatchersString(matchers)
type f func() (*logproto.VolumeResponse, error)
var jobs []f
if a.shouldQueryIngesters(through, model.Now()) {
jobs = append(jobs, func() (*logproto.VolumeResponse, error) {
vols, err := a.ingesterQuerier.Volume(ctx, userID, from, through, limit, targetLabels, aggregateBy, matchers...)
level.Debug(logger).Log(
"msg", "queried label volumes",
"matchers", matchersStr,
"source", "ingesters",
)
return vols, err
})
}
jobs = append(jobs, func() (*logproto.VolumeResponse, error) {
vols, err := a.Store.Volume(ctx, userID, from, through, limit, targetLabels, aggregateBy, matchers...)
level.Debug(logger).Log(
"msg", "queried label volume",
"matchers", matchersStr,
"source", "store",
)
return vols, err
})
resps := make([]*logproto.VolumeResponse, len(jobs))
if err := concurrency.ForEachJob(
ctx,
len(jobs),
len(jobs),
func(ctx context.Context, i int) error {
resp, err := jobs[i]()
resps[i] = resp
return err
},
); err != nil {
return nil, err
}
sp.LogKV(
"user", userID,
"from", from.Time(),
"through", through.Time(),
"matchers", syntax.MatchersString(matchers),
"limit", limit,
)
merged := seriesvolume.Merge(resps, limit)
return merged, nil
}
func (a *AsyncStore) mergeIngesterAndStoreChunks(userID string, storeChunks [][]chunk.Chunk, fetchers []*fetcher.Fetcher, ingesterChunkIDs []string) ([][]chunk.Chunk, []*fetcher.Fetcher, error) {
Simpler new chunk key v12 (#5054) * starts hacking new chunk paths * Create new chunk key for S3; update parsing and add basic test and benchmark Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Fix `TestGrpcStore` to work with new chunk key structure Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Add a function to SchemaConfig that returns the correct PeriodConfig for a given time. Signed-off-by: Callum Styan <callumstyan@gmail.com> Add schema config to object client so it we can use the right external key function based on the schema version. Signed-off-by: Callum Styan <callumstyan@gmail.com> Fix failing compactor tests by passing SchemaConfig to chunkClient Remove empty conditional from SchemaForTime Define schema v12; wire-in ChunkPathShardFactor and ChunkPathPeriod as configurable values Add PeriodConfig test steps for new values ChunkPathShardFactor and ChunkPathPeriod in v12 Update test for chunk.NewExternalKey(); remove completed TODO Set defaults for new ChunkPathPeriod and ChunkPathShardFactor SchemaConfig values; change FSObjectClient to use IdentityEncoder instead of Base64Encoder Use IdentityEncoder everywhere we use FSObjectClient for Chunks Add ExternalKey() function to SchemaConfig; update ObjectClient for Chunks * Finish plumbing through the chunk.ExternalKey -> schemaConfig.ExternalKey change. Signed-off-by: Callum Styan <callumstyan@gmail.com> * Clean up lint failures Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Clean up chunk.go and fix tests Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Break SchemaConfig.ExternalKey() into smaller functions Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Quickly fix variable name in SchemaConfig.newerExternalKey() Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Clean up ExternalKey conditional logic; add better comments; add tests Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Fix a bug where we are prepending userID to legacy Chunk keys but never parsing it; refactor key tests and benchmarks Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Add small SchemaConfig test Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Update docs and CHANGELOG.md Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Correctly return an error when failing to parse a chunk external key Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Revert IdentityEncoder to Base64Encoder after testing Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Fix assignment in test for linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Remove leftover comments from development Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Change v12 version comment format; remove redundant login in `ParseExternalKey()` Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Change remaining v12 comment style Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Pass chunk.SchemaConfig to new parallel chunk client * Simplify chunk external key prefixes for schema v12 * Fix broken benchmark; add benchmarks for old parsing conditional Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Remove unneeded lines from upgrading doc * Add benchmarks for root chunk external key parsing functions Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * memoizes schema number calculations * Resolve linter issue with PeriodConfig receiver name Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
3 years ago
ingesterChunkIDs = filterDuplicateChunks(a.scfg, storeChunks, ingesterChunkIDs)
level.Debug(util_log.Logger).Log("msg", "post-filtering ingester chunks", "count", len(ingesterChunkIDs))
fetcherToChunksGroupIdx := make(map[*fetcher.Fetcher]int, len(fetchers))
for i, fetcher := range fetchers {
fetcherToChunksGroupIdx[fetcher] = i
}
for _, chunkID := range ingesterChunkIDs {
chk, err := chunk.ParseExternalKey(userID, chunkID)
if err != nil {
return nil, nil, err
}
// ToDo(Sandeep) possible optimization: Keep the chunk fetcher reference handy after first call since it is expected to stay the same.
fetcher := a.Store.GetChunkFetcher(chk.Through)
if fetcher == nil {
return nil, nil, fmt.Errorf("got a nil fetcher for chunk %s", a.scfg.ExternalKey(chk.ChunkRef))
}
if _, ok := fetcherToChunksGroupIdx[fetcher]; !ok {
fetchers = append(fetchers, fetcher)
storeChunks = append(storeChunks, []chunk.Chunk{})
fetcherToChunksGroupIdx[fetcher] = len(fetchers) - 1
}
chunksGroupIdx := fetcherToChunksGroupIdx[fetcher]
storeChunks[chunksGroupIdx] = append(storeChunks[chunksGroupIdx], chk)
}
return storeChunks, fetchers, nil
}
func filterDuplicateChunks(scfg config.SchemaConfig, storeChunks [][]chunk.Chunk, ingesterChunkIDs []string) []string {
filteredChunkIDs := make([]string, 0, len(ingesterChunkIDs))
seen := make(map[string]struct{}, len(storeChunks))
for i := range storeChunks {
for j := range storeChunks[i] {
seen[scfg.ExternalKey(storeChunks[i][j].ChunkRef)] = struct{}{}
}
}
for _, chunkID := range ingesterChunkIDs {
if _, ok := seen[chunkID]; !ok {
filteredChunkIDs = append(filteredChunkIDs, chunkID)
seen[chunkID] = struct{}{}
}
}
return filteredChunkIDs
}