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

532 lines
16 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"
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
"flag"
"net/http"
"strings"
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
"time"
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
"github.com/go-kit/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/weaveworks/common/httpgrpc"
"github.com/grafana/dskit/tenant"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/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
)
// Config is the configuration for the queryrange tripperware
type Config struct {
queryrangebase.Config `yaml:",inline"`
Transformer UserIDTransformer `yaml:"-"`
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
}
// RegisterFlags adds the flags required to configure this flag set.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.Config.RegisterFlags(f)
}
// Stopper gracefully shutdown resources created
type Stopper interface {
Stop()
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
}
// NewTripperware returns a Tripperware configured with middlewares to align, split and cache requests.
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
func NewTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema config.SchemaConfig,
cacheGenNumLoader queryrangebase.CacheGenNumberLoader,
retentionEnabled bool,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
registerer prometheus.Registerer,
) (queryrangebase.Tripperware, Stopper, error) {
metrics := NewMetrics(registerer)
var (
c cache.Cache
err error
)
embedded-cache: Bring `fifocache` and `groupcache` into single tent. (#6821) * chore(groupcache): Rename groupcache into memcache{distributed: true} 1. Introduced new config called memorycache config. 2. It has flag `distributed`. By enabling it, we use groupcache Going forward. We will bring fifocache into memcache config(with distributed:false) This PR is just a POC. To show how we can simplify in-built memory cache config and bring it into single config. Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Memorycache -> EmbeddedCache * Bring fifocache into embeddedcache tent Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Fix edge case with enabling fifocache Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Fix missing flags for embedded cache Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Add changelog Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Warning message for fifocache config Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Make linter happy Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Fix listenport issue Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * idk. review remarks Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * PR remarks 1. update changelog and upgrade guide 2. s/Embeddedcache/EmbeddedCache/g 3. /groupcache/ handler -> /embedded-cache/ 4. some typos 5. Fix some cache prefixes Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Fixing corresponding tests Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Support extra timeouts on embedded-cache Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Remove unwanted single struct field Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * fix(stats-collector): Let cache.New() wrap stats collector for distributed cache Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * PR remarks Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Deprecate max_size_items from fifocache Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * PR remarks about the flag description Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com> * Update docs/sources/upgrading/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
if cfg.CacheResults {
c, err = cache.New(cfg.CacheConfig, registerer, log, stats.ResultCache)
if err != nil {
return nil, nil, err
}
if cfg.Compression == "snappy" {
c = cache.NewSnappy(c, log)
}
}
metricsTripperware, err := NewMetricTripperware(cfg, log, limits, schema, LokiCodec, c,
cacheGenNumLoader, retentionEnabled, PrometheusExtractor{}, metrics, registerer)
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, nil, err
}
// NOTE: When we would start caching response from non-metric queries we would have to consider cache gen headers as well in
// MergeResponse implementation for Loki codecs same as it is done in Cortex at https://github.com/cortexproject/cortex/blob/21bad57b346c730d684d6d0205efef133422ab28/pkg/querier/queryrange/query_range.go#L170
logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, LokiCodec, c, metrics)
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, nil, err
}
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
seriesTripperware, err := NewSeriesTripperware(cfg, log, limits, LokiCodec, metrics, schema)
if err != nil {
return nil, nil, err
}
labelsTripperware, err := NewLabelsTripperware(cfg, log, limits, LokiCodec, metrics, schema)
if err != nil {
return nil, nil, err
}
instantMetricTripperware, err := NewInstantMetricTripperware(cfg, log, limits, schema, LokiCodec, metrics)
if err != nil {
return nil, nil, err
}
return func(next http.RoundTripper) http.RoundTripper {
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
metricRT := metricsTripperware(next)
logFilterRT := logFilterTripperware(next)
seriesRT := seriesTripperware(next)
labelsRT := labelsTripperware(next)
instantRT := instantMetricTripperware(next)
return newRoundTripper(next, logFilterRT, metricRT, seriesRT, labelsRT, instantRT, limits)
}, c, 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
}
type roundTripper struct {
next, log, metric, series, labels, instantMetric http.RoundTripper
limits Limits
}
// newRoundTripper creates a new queryrange roundtripper
func newRoundTripper(next, log, metric, series, labels, instantMetric http.RoundTripper, limits Limits) roundTripper {
return roundTripper{
log: log,
limits: limits,
metric: metric,
series: series,
labels: labels,
instantMetric: instantMetric,
next: next,
}
}
func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
err := req.ParseForm()
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
switch op := getOperation(req.URL.Path); op {
case QueryRangeOp:
rangeQuery, err := loghttp.ParseRangeQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
expr, err := syntax.ParseExpr(rangeQuery.Query)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
switch e := expr.(type) {
case syntax.SampleExpr:
return r.metric.RoundTrip(req)
case syntax.LogSelectorExpr:
Loki: Remove the bypass for "limited" queries (#7510) **What this PR does / why we need it**: Limited queries are queries which don't have a filter expression. Now all log queries will be handled by the LogFilterTripperware which will result in them being split by time (they previously were not) Signed-off-by: Edward Welch <edward.welch@grafana.com> **Which issue(s) this PR fixes**: We've found that very large timeframe `limited` queries will be sent to queriers and then ingesters and can stall out the read path because of the large time ranges. Splitting these by time avoids this problem by keeping any subquery limited in length to the `split_by_interval`. It's important to note that this will likely increase some extra work done by limited queries as more work for these will be parallelized and can result in extra data being processed. This will be the same as how filter queries are handled now, so it will be no worse than that. In fact this was an optimization based on the premise that it's advantageous not to split/shard limited queries but we are seeing this not be the case when limited queries are run over very large time ranges matching streams with huge volumes and combined with parsers like `| json` **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the `CONTRIBUTING.md` guide - [ ] 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` Signed-off-by: Edward Welch <edward.welch@grafana.com>
3 years ago
// Note, this function can mutate the request
_, err := transformRegexQuery(req, e)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
if err := validateLimits(req, rangeQuery.Limit, r.limits); err != nil {
return nil, err
}
return r.log.RoundTrip(req)
default:
return r.next.RoundTrip(req)
}
case SeriesOp:
_, err := loghttp.ParseAndValidateSeriesQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
return r.series.RoundTrip(req)
case LabelNamesOp:
_, err := loghttp.ParseLabelQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
return r.labels.RoundTrip(req)
case InstantQueryOp:
instantQuery, err := loghttp.ParseInstantQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
expr, err := syntax.ParseExpr(instantQuery.Query)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
switch expr.(type) {
case syntax.SampleExpr:
return r.instantMetric.RoundTrip(req)
default:
return r.next.RoundTrip(req)
}
default:
return r.next.RoundTrip(req)
}
}
// transformRegexQuery backport the old regexp params into the v1 query format
func transformRegexQuery(req *http.Request, expr syntax.LogSelectorExpr) (syntax.LogSelectorExpr, error) {
regexp := req.Form.Get("regexp")
if regexp != "" {
filterExpr, err := syntax.AddFilterExpr(expr, labels.MatchRegexp, "", regexp)
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
if err != nil {
return nil, err
}
params := req.URL.Query()
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
params.Set("query", filterExpr.String())
req.URL.RawQuery = params.Encode()
// force the form and query to be parsed again.
req.Form = nil
req.PostForm = nil
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
return filterExpr, nil
}
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
return expr, nil
}
// validates log entries limits
func validateLimits(req *http.Request, reqLimit uint32, limits Limits) error {
tenantIDs, err := tenant.TenantIDs(req.Context())
if err != nil {
return httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
maxEntriesLimit := validation.SmallestPositiveNonZeroIntPerTenant(tenantIDs, limits.MaxEntriesLimitPerQuery)
if int(reqLimit) > maxEntriesLimit && maxEntriesLimit != 0 {
return httpgrpc.Errorf(http.StatusBadRequest,
"max entries limit per query exceeded, limit > max_entries_limit (%d > %d)", reqLimit, maxEntriesLimit)
}
return nil
}
const (
InstantQueryOp = "instant_query"
QueryRangeOp = "query_range"
SeriesOp = "series"
LabelNamesOp = "labels"
IndexStatsOp = "index_stats"
)
func getOperation(path string) string {
switch {
case strings.HasSuffix(path, "/query_range") || strings.HasSuffix(path, "/prom/query"):
return QueryRangeOp
case strings.HasSuffix(path, "/series"):
return SeriesOp
case strings.HasSuffix(path, "/labels") || strings.HasSuffix(path, "/label") || strings.HasSuffix(path, "/values"):
return LabelNamesOp
case strings.HasSuffix(path, "/v1/query"):
return InstantQueryOp
case path == "/loki/api/v1/index/stats":
return IndexStatsOp
default:
return ""
}
}
Loki: Remove the bypass for "limited" queries (#7510) **What this PR does / why we need it**: Limited queries are queries which don't have a filter expression. Now all log queries will be handled by the LogFilterTripperware which will result in them being split by time (they previously were not) Signed-off-by: Edward Welch <edward.welch@grafana.com> **Which issue(s) this PR fixes**: We've found that very large timeframe `limited` queries will be sent to queriers and then ingesters and can stall out the read path because of the large time ranges. Splitting these by time avoids this problem by keeping any subquery limited in length to the `split_by_interval`. It's important to note that this will likely increase some extra work done by limited queries as more work for these will be parallelized and can result in extra data being processed. This will be the same as how filter queries are handled now, so it will be no worse than that. In fact this was an optimization based on the premise that it's advantageous not to split/shard limited queries but we are seeing this not be the case when limited queries are run over very large time ranges matching streams with huge volumes and combined with parsers like `| json` **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the `CONTRIBUTING.md` guide - [ ] 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` Signed-off-by: Edward Welch <edward.welch@grafana.com>
3 years ago
// NewLogFilterTripperware creates a new frontend tripperware responsible for handling log requests.
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 NewLogFilterTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema config.SchemaConfig,
codec queryrangebase.Codec,
c cache.Cache,
metrics *Metrics,
) (queryrangebase.Tripperware, error) {
queryRangeMiddleware := []queryrangebase.Middleware{
StatsCollectorMiddleware(),
NewLimitsMiddleware(limits),
queryrangebase.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics),
SplitByIntervalMiddleware(schema.Configs, limits, codec, splitByTime, metrics.SplitByMetrics),
}
if cfg.CacheResults {
queryCacheMiddleware := NewLogResultCache(
log,
limits,
c,
func(r queryrangebase.Request) bool {
return !r.GetCachingOptions().Disabled
},
cfg.Transformer,
metrics.LogResultCacheMetrics,
)
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("log_results_cache", metrics.InstrumentMiddlewareMetrics),
queryCacheMiddleware,
)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
if cfg.ShardedQueries {
queryRangeMiddleware = append(queryRangeMiddleware,
NewQueryShardMiddleware(
log,
schema.Configs,
metrics.InstrumentMiddlewareMetrics, // instrumentation is included in the sharding middleware
metrics.MiddlewareMapperMetrics.shardMapper,
limits,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
),
)
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
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
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(
queryRangeMiddleware, queryrangebase.InstrumentMiddleware("retry", metrics.InstrumentMiddlewareMetrics),
queryrangebase.NewRetryMiddleware(log, cfg.MaxRetries, metrics.RetryMiddlewareMetrics),
)
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 func(next http.RoundTripper) http.RoundTripper {
if len(queryRangeMiddleware) > 0 {
return NewLimitedRoundTripper(next, codec, limits, schema.Configs, queryRangeMiddleware...)
}
return next
}, nil
}
// NewSeriesTripperware creates a new frontend tripperware responsible for handling series requests
func NewSeriesTripperware(
cfg Config,
log log.Logger,
limits Limits,
codec queryrangebase.Codec,
metrics *Metrics,
schema config.SchemaConfig,
) (queryrangebase.Tripperware, error) {
queryRangeMiddleware := []queryrangebase.Middleware{
StatsCollectorMiddleware(),
NewLimitsMiddleware(limits),
queryrangebase.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics),
// The Series API needs to pull one chunk per series to extract the label set, which is much cheaper than iterating through all matching chunks.
// Force a 24 hours split by for series API, this will be more efficient with our static daily bucket storage.
// This would avoid queriers downloading chunks for same series over and over again for serving smaller queries.
SplitByIntervalMiddleware(schema.Configs, WithSplitByLimits(limits, 24*time.Hour), codec, splitByTime, metrics.SplitByMetrics),
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("retry", metrics.InstrumentMiddlewareMetrics),
queryrangebase.NewRetryMiddleware(log, cfg.MaxRetries, metrics.RetryMiddlewareMetrics),
)
}
if cfg.ShardedQueries {
queryRangeMiddleware = append(queryRangeMiddleware,
NewSeriesQueryShardMiddleware(
log,
schema.Configs,
metrics.InstrumentMiddlewareMetrics,
metrics.MiddlewareMapperMetrics.shardMapper,
limits,
codec,
),
)
}
return func(next http.RoundTripper) http.RoundTripper {
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 len(queryRangeMiddleware) > 0 {
return NewLimitedRoundTripper(next, codec, limits, schema.Configs, queryRangeMiddleware...)
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 next
}, 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
}
// NewLabelsTripperware creates a new frontend tripperware responsible for handling labels requests.
func NewLabelsTripperware(
cfg Config,
log log.Logger,
limits Limits,
codec queryrangebase.Codec,
metrics *Metrics,
schema config.SchemaConfig,
) (queryrangebase.Tripperware, error) {
queryRangeMiddleware := []queryrangebase.Middleware{
StatsCollectorMiddleware(),
NewLimitsMiddleware(limits),
queryrangebase.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics),
// Force a 24 hours split by for labels API, this will be more efficient with our static daily bucket storage.
// This is because the labels API is an index-only operation.
SplitByIntervalMiddleware(schema.Configs, WithSplitByLimits(limits, 24*time.Hour), codec, splitByTime, metrics.SplitByMetrics),
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("retry", metrics.InstrumentMiddlewareMetrics),
queryrangebase.NewRetryMiddleware(log, cfg.MaxRetries, metrics.RetryMiddlewareMetrics),
)
}
return func(next http.RoundTripper) http.RoundTripper {
if len(queryRangeMiddleware) > 0 {
// Do not forward any request header.
return queryrangebase.NewRoundTripper(next, codec, nil, queryRangeMiddleware...)
}
return next
}, nil
}
// NewMetricTripperware creates a new frontend tripperware responsible for handling metric queries
func NewMetricTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema config.SchemaConfig,
codec queryrangebase.Codec,
c cache.Cache,
cacheGenNumLoader queryrangebase.CacheGenNumberLoader,
retentionEnabled bool,
extractor queryrangebase.Extractor,
metrics *Metrics,
registerer prometheus.Registerer,
) (queryrangebase.Tripperware, error) {
queryRangeMiddleware := []queryrangebase.Middleware{StatsCollectorMiddleware(), NewLimitsMiddleware(limits)}
if cfg.AlignQueriesWithStep {
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("step_align", metrics.InstrumentMiddlewareMetrics),
queryrangebase.StepAlignMiddleware,
)
}
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics),
SplitByIntervalMiddleware(schema.Configs, limits, codec, splitMetricByTime, metrics.SplitByMetrics),
)
cacheKey := cacheKeyLimits{limits, cfg.Transformer}
if cfg.CacheResults {
queryCacheMiddleware, err := queryrangebase.NewResultsCacheMiddleware(
log,
c,
cacheKey,
limits,
codec,
extractor,
cacheGenNumLoader,
func(r queryrangebase.Request) bool {
return !r.GetCachingOptions().Disabled
},
func(ctx context.Context, tenantIDs []string, r queryrangebase.Request) int {
return MinWeightedParallelism(
ctx,
tenantIDs,
schema.Configs,
limits,
model.Time(r.GetStart()),
model.Time(r.GetEnd()),
)
},
retentionEnabled,
metrics.ResultsCacheMetrics,
)
if err != nil {
return nil, err
}
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("results_cache", metrics.InstrumentMiddlewareMetrics),
queryCacheMiddleware,
)
}
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
if cfg.ShardedQueries {
queryRangeMiddleware = append(queryRangeMiddleware,
NewQueryShardMiddleware(
log,
schema.Configs,
metrics.InstrumentMiddlewareMetrics, // instrumentation is included in the sharding middleware
metrics.MiddlewareMapperMetrics.shardMapper,
limits,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
),
)
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("retry", metrics.InstrumentMiddlewareMetrics),
queryrangebase.NewRetryMiddleware(log, cfg.MaxRetries, metrics.RetryMiddlewareMetrics),
)
}
return func(next http.RoundTripper) http.RoundTripper {
// Finally, if the user selected any query range middleware, stitch it in.
if len(queryRangeMiddleware) > 0 {
rt := NewLimitedRoundTripper(next, codec, limits, schema.Configs, queryRangeMiddleware...)
return queryrangebase.RoundTripFunc(func(r *http.Request) (*http.Response, error) {
if !strings.HasSuffix(r.URL.Path, "/query_range") {
return next.RoundTrip(r)
}
return rt.RoundTrip(r)
})
}
return next
}, nil
}
// NewInstantMetricTripperware creates a new frontend tripperware responsible for handling metric queries
func NewInstantMetricTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema config.SchemaConfig,
codec queryrangebase.Codec,
metrics *Metrics,
) (queryrangebase.Tripperware, error) {
queryRangeMiddleware := []queryrangebase.Middleware{StatsCollectorMiddleware(), NewLimitsMiddleware(limits)}
if cfg.ShardedQueries {
queryRangeMiddleware = append(queryRangeMiddleware,
NewSplitByRangeMiddleware(log, limits, metrics.MiddlewareMapperMetrics.rangeMapper),
NewQueryShardMiddleware(
log,
schema.Configs,
metrics.InstrumentMiddlewareMetrics, // instrumentation is included in the sharding middleware
metrics.MiddlewareMapperMetrics.shardMapper,
limits,
),
)
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrangebase.InstrumentMiddleware("retry", metrics.InstrumentMiddlewareMetrics),
queryrangebase.NewRetryMiddleware(log, cfg.MaxRetries, metrics.RetryMiddlewareMetrics),
)
}
return func(next http.RoundTripper) http.RoundTripper {
if len(queryRangeMiddleware) > 0 {
return NewLimitedRoundTripper(next, codec, limits, schema.Configs, queryRangeMiddleware...)
}
return next
}, nil
}