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/querier/queryrange/split_by_interval.go

281 lines
7.3 KiB

Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
package queryrange
import (
"context"
"net/http"
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
"time"
"github.com/grafana/dskit/httpgrpc"
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
"github.com/pkg/errors"
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
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/v3/pkg/storage/config"
"github.com/grafana/loki/v3/pkg/util/constants"
"github.com/grafana/loki/v3/pkg/util/validation"
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
)
type lokiResult struct {
req queryrangebase.Request
ch chan *packedResp
}
type packedResp struct {
resp queryrangebase.Response
err error
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
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
type SplitByMetrics struct {
splits prometheus.Histogram
}
func NewSplitByMetrics(r prometheus.Registerer) *SplitByMetrics {
return &SplitByMetrics{
splits: promauto.With(r).NewHistogram(prometheus.HistogramOpts{
Namespace: constants.Loki,
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
Name: "query_frontend_partitions",
Help: "Number of time-based partitions (sub-requests) per request",
Buckets: prometheus.ExponentialBuckets(1, 4, 5), // 1 -> 1024
}),
}
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
type splitByInterval struct {
configs []config.PeriodConfig
next queryrangebase.Handler
limits Limits
merger queryrangebase.Merger
metrics *SplitByMetrics
Query-frontend: customisable query splitting for queries overlapping `query_ingester_within` window (#11535) **What this PR does / why we need it**: The config option `query_ingesters_within` defines the window during which logs _could_ be present on ingesters, and as such queriers will send queries to ingesters instead. `split_queries_by_interval` is defined to split queries into subqueries for increased parallelism. Aggressive query splitting within the `query_ingesters_within` window can result in overloading ingesters with unnecessarily large numbers of subqueries, which perversely can impact writes. `query_ingesters_within` is set to 3h by default. In Grafana Cloud Logs we set `split_queries_by_interval` as low as 15m (defaults to 1h), which would result in result in 3*60/15=12 requests. Every querier queries every ingester during this window, so that's 12 requests _per ingester per query_ which has the `query_ingesters_within` window in its time range _(i.e. a query from now to now-7d would include the `query_ingesters_within` window as well, now-3h to now-7d would not)_. However, we _do_ want to split queries so an ingester won't have to handle a query for a full `query_ingesters_within` window - this could involve a large amount of data. To account for this, this PR introduces a new option `split_ingester_queries_by_interval` on the query-frontend; this setting is disabled by default. ![image](https://github.com/grafana/loki/assets/373762/2e671bd8-9e8d-4bf3-addf-bebcfc25e8d7)
1 year ago
splitter splitter
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
}
// SplitByIntervalMiddleware creates a new Middleware that splits log requests by a given interval.
Query-frontend: customisable query splitting for queries overlapping `query_ingester_within` window (#11535) **What this PR does / why we need it**: The config option `query_ingesters_within` defines the window during which logs _could_ be present on ingesters, and as such queriers will send queries to ingesters instead. `split_queries_by_interval` is defined to split queries into subqueries for increased parallelism. Aggressive query splitting within the `query_ingesters_within` window can result in overloading ingesters with unnecessarily large numbers of subqueries, which perversely can impact writes. `query_ingesters_within` is set to 3h by default. In Grafana Cloud Logs we set `split_queries_by_interval` as low as 15m (defaults to 1h), which would result in result in 3*60/15=12 requests. Every querier queries every ingester during this window, so that's 12 requests _per ingester per query_ which has the `query_ingesters_within` window in its time range _(i.e. a query from now to now-7d would include the `query_ingesters_within` window as well, now-3h to now-7d would not)_. However, we _do_ want to split queries so an ingester won't have to handle a query for a full `query_ingesters_within` window - this could involve a large amount of data. To account for this, this PR introduces a new option `split_ingester_queries_by_interval` on the query-frontend; this setting is disabled by default. ![image](https://github.com/grafana/loki/assets/373762/2e671bd8-9e8d-4bf3-addf-bebcfc25e8d7)
1 year ago
func SplitByIntervalMiddleware(configs []config.PeriodConfig, limits Limits, merger queryrangebase.Merger, splitter splitter, metrics *SplitByMetrics) queryrangebase.Middleware {
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
2 years ago
if metrics == nil {
metrics = NewSplitByMetrics(nil)
}
return queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler {
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
return &splitByInterval{
configs: configs,
next: next,
limits: limits,
merger: merger,
metrics: metrics,
splitter: splitter,
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
}
})
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
func (h *splitByInterval) Feed(ctx context.Context, input []*lokiResult) chan *lokiResult {
ch := make(chan *lokiResult)
go func() {
defer close(ch)
for _, d := range input {
select {
case <-ctx.Done():
return
case ch <- d:
continue
}
}
}()
return ch
}
func (h *splitByInterval) Process(
ctx context.Context,
parallelism int,
threshold int64,
input []*lokiResult,
maxSeries int,
) ([]queryrangebase.Response, error) {
var responses []queryrangebase.Response
ctx, cancel := context.WithCancelCause(ctx)
defer cancel(errors.New("split by interval process canceled"))
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
ch := h.Feed(ctx, input)
// queries with 0 limits should not be exited early
var unlimited bool
if threshold == 0 {
unlimited = true
}
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
2 years ago
// Parallelism will be at least 1
p := max(parallelism, 1)
// don't spawn unnecessary goroutines
if len(input) < parallelism {
p = len(input)
}
// per request wrapped handler for limiting the amount of series.
next := newSeriesLimiter(maxSeries).Wrap(h.next)
for i := 0; i < p; i++ {
go h.loop(ctx, ch, next)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
for _, x := range input {
select {
case <-ctx.Done():
return nil, ctx.Err()
case data := <-x.ch:
if data.err != nil {
return nil, data.err
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
responses = append(responses, data.resp)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
// see if we can exit early if a limit has been reached
if casted, ok := data.resp.(*LokiResponse); !unlimited && ok {
threshold -= casted.Count()
if threshold <= 0 {
return responses, nil
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
}
return responses, nil
}
func (h *splitByInterval) loop(ctx context.Context, ch <-chan *lokiResult, next queryrangebase.Handler) {
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
for data := range ch {
sp, ctx := opentracing.StartSpanFromContext(ctx, "interval")
data.req.LogToSpan(sp)
resp, err := next.Do(ctx, data.req)
sp.Finish()
select {
case <-ctx.Done():
return
case data.ch <- &packedResp{resp, err}:
Stop processing split queries on first error. (#11067) **What this PR does / why we need it**: The `TestMetricsTripperware` became flaky after #10688. The race condition in the splitter has been there before but the change exposed it. When an error occurs, e.g. when query would be too large, the splitter would return. However, there was a small time window when the loop could still be fed a new request. #10688 just amplified this effect because it removed the serialization of requests and responses. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. <!-- TODO(salvacorts): Add example PR --> --------- Co-authored-by: Danny Kopping <dannykopping@gmail.com>
2 years ago
// The parent Process method will return on the first error. So stop
// processng.
if err != nil {
return
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
}
}
func (h *splitByInterval) Do(ctx context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
tenantIDs, err := tenant.TenantIDs(ctx)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error())
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
feat(caching): Support caching `/series` and `/labels` query results (#11539) **What this PR does / why we need it**: Add support for caching metadata queries (both series and labels). caching happens after splitting similar to other types of queries. This pr adds the following configs to enable them. ``` cache_series_results: true|false (default false) cache_label_results: true|false (default false) ``` And the cache backend for them can be configured using `series_results_cache` and `label_results_cache` blocks under the `query_range` section. Currently the split interval for metadata queries is fixed and defaults to 24h, this pr makes it configurable by introducing `split_metadata_queries_by_interval` **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **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 - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> Co-authored-by: Ashwanth Goli <iamashwanth@gmail.com>
2 years ago
var interval time.Duration
switch r.(type) {
case *LokiSeriesRequest, *LabelRequest:
interval = validation.MaxDurationOrZeroPerTenant(tenantIDs, h.limits.MetadataQuerySplitDuration)
default:
interval = validation.SmallestPositiveNonZeroDurationPerTenant(tenantIDs, h.limits.QuerySplitDuration)
feat(caching): Support caching `/series` and `/labels` query results (#11539) **What this PR does / why we need it**: Add support for caching metadata queries (both series and labels). caching happens after splitting similar to other types of queries. This pr adds the following configs to enable them. ``` cache_series_results: true|false (default false) cache_label_results: true|false (default false) ``` And the cache backend for them can be configured using `series_results_cache` and `label_results_cache` blocks under the `query_range` section. Currently the split interval for metadata queries is fixed and defaults to 24h, this pr makes it configurable by introducing `split_metadata_queries_by_interval` **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **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 - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> Co-authored-by: Ashwanth Goli <iamashwanth@gmail.com>
2 years ago
}
// skip split by if unset
if interval == 0 {
return h.next.Do(ctx, r)
}
intervals := h.splitter.split(time.Now().UTC(), tenantIDs, r, interval)
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
2 years ago
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
h.metrics.splits.Observe(float64(len(intervals)))
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
// no interval should not be processed by the frontend.
if len(intervals) == 0 {
return h.next.Do(ctx, r)
}
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogFields(otlog.Int("n_intervals", len(intervals)))
}
if len(intervals) == 1 {
return h.next.Do(ctx, intervals[0])
}
var limit int64
switch req := r.(type) {
case *LokiRequest:
limit = int64(req.Limit)
if req.Direction == logproto.BACKWARD {
for i, j := 0, len(intervals)-1; i < j; i, j = i+1, j-1 {
intervals[i], intervals[j] = intervals[j], intervals[i]
}
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
case *DetectedFieldsRequest:
limit = int64(req.LineLimit)
for i, j := 0, len(intervals)-1; i < j; i, j = i+1, j-1 {
intervals[i], intervals[j] = intervals[j], intervals[i]
}
case *LokiSeriesRequest, *LabelRequest, *logproto.IndexStatsRequest, *logproto.VolumeRequest, *logproto.ShardsRequest, *DetectedLabelsRequest:
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
2 years ago
// Set this to 0 since this is not used in Series/Labels/Index Request.
limit = 0
default:
return nil, httpgrpc.Errorf(http.StatusBadRequest, "unknown request type")
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
}
input := make([]*lokiResult, 0, len(intervals))
for _, interval := range intervals {
input = append(input, &lokiResult{
req: interval,
ch: make(chan *packedResp),
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
})
}
maxSeriesCapture := func(id string) int { return h.limits.MaxQuerySeries(ctx, id) }
maxSeries := validation.SmallestPositiveIntPerTenant(tenantIDs, maxSeriesCapture)
maxParallelism := MinWeightedParallelism(ctx, tenantIDs, h.configs, h.limits, model.Time(r.GetStart().UnixMilli()), model.Time(r.GetEnd().UnixMilli()))
resps, err := h.Process(ctx, maxParallelism, limit, input, maxSeries)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
6 years ago
if err != nil {
return nil, err
}
return h.merger.MergeResponse(resps...)
}
// maxRangeVectorAndOffsetDurationFromQueryString
func maxRangeVectorAndOffsetDurationFromQueryString(q string) (time.Duration, time.Duration, error) {
parsed, err := syntax.ParseExpr(q)
if err != nil {
return 0, 0, err
}
dur, offset := maxRangeVectorAndOffsetDuration(parsed)
return dur, offset, nil
}
// maxRangeVectorAndOffsetDuration returns the maximum range vector and offset duration within a LogQL query.
func maxRangeVectorAndOffsetDuration(expr syntax.Expr) (time.Duration, time.Duration) {
if _, ok := expr.(syntax.SampleExpr); !ok {
return 0, 0
}
var maxRVDuration, maxOffset time.Duration
expr.Walk(func(e syntax.Expr) bool {
if r, ok := e.(*syntax.LogRangeExpr); ok {
if r.Interval > maxRVDuration {
maxRVDuration = r.Interval
}
if r.Offset > maxOffset {
maxOffset = r.Offset
}
}
return true
})
return maxRVDuration, maxOffset
}