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

1138 lines
30 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>
5 years ago
package queryrange
import (
"context"
"fmt"
"runtime"
"strconv"
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>
5 years ago
"sync"
"testing"
"time"
"github.com/prometheus/common/model"
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>
5 years ago
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"gopkg.in/yaml.v2"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/storage/config"
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>
5 years ago
)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
var nilMetrics = NewSplitByMetrics(nil)
var testSchemas = func() []config.PeriodConfig {
confS := `
- from: "1950-01-01"
store: boltdb-shipper
object_store: gcs
schema: v12
`
var confs []config.PeriodConfig
if err := yaml.Unmarshal([]byte(confS), &confs); err != nil {
panic(err)
}
return confs
}()
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
2 years ago
var testSchemasTSDB = func() []config.PeriodConfig {
confS := `
- from: "1950-01-01"
store: tsdb
object_store: gcs
schema: v12
`
var confs []config.PeriodConfig
if err := yaml.Unmarshal([]byte(confS), &confs); err != nil {
panic(err)
}
return confs
}()
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>
5 years ago
func Test_splitQuery(t *testing.T) {
buildLokiRequest := func(start, end time.Time) queryrangebase.Request {
return &LokiRequest{
Query: "foo",
Limit: 1,
Step: 2,
StartTs: start,
EndTs: end,
Direction: logproto.BACKWARD,
Path: "/path",
}
}
buildLokiRequestWithInterval := func(start, end time.Time) queryrangebase.Request {
return &LokiRequest{
Query: "foo",
Limit: 1,
Interval: 2,
StartTs: start,
EndTs: end,
Direction: logproto.BACKWARD,
Path: "/path",
}
}
buildLokiSeriesRequest := func(start, end time.Time) queryrangebase.Request {
return &LokiSeriesRequest{
Match: []string{"match1"},
StartTs: start,
EndTs: end,
Path: "/series",
Shards: []string{"shard1"},
}
}
buildLokiLabelNamesRequest := func(start, end time.Time) queryrangebase.Request {
return &LokiLabelNamesRequest{
StartTs: start,
EndTs: end,
Path: "/labels",
}
}
type interval struct {
start, end time.Time
}
for requestType, tc := range map[string]struct {
requestBuilderFunc func(start, end time.Time) queryrangebase.Request
endTimeInclusive bool
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>
5 years ago
}{
"LokiRequest": {
buildLokiRequest,
false,
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>
5 years ago
},
"LokiRequestWithInterval": {
buildLokiRequestWithInterval,
false,
},
"LokiSeriesRequest": {
buildLokiSeriesRequest,
true,
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>
5 years ago
},
"LokiLabelNamesRequest": {
buildLokiLabelNamesRequest,
true,
},
} {
expectedSplitGap := time.Duration(0)
if tc.endTimeInclusive {
expectedSplitGap = time.Millisecond
}
for name, intervals := range map[string]struct {
inp interval
expected []interval
}{
"no_change": {
inp: interval{
start: time.Unix(0, 0),
end: time.Unix(0, (1 * time.Hour).Nanoseconds()),
},
expected: []interval{
{
start: time.Unix(0, 0),
end: time.Unix(0, (1 * time.Hour).Nanoseconds()),
},
},
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>
5 years ago
},
"align_start": {
inp: interval{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (2 * time.Hour).Nanoseconds()),
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>
5 years ago
},
expected: []interval{
{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (1 * time.Hour).Nanoseconds()).Add(-expectedSplitGap),
},
{
start: time.Unix(0, (1 * time.Hour).Nanoseconds()),
end: time.Unix(0, (2 * time.Hour).Nanoseconds()),
},
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>
5 years ago
},
},
"align_end": {
inp: interval{
start: time.Unix(0, 0),
end: time.Unix(0, (115 * time.Minute).Nanoseconds()),
},
expected: []interval{
{
start: time.Unix(0, 0),
end: time.Unix(0, (1 * time.Hour).Nanoseconds()).Add(-expectedSplitGap),
},
{
start: time.Unix(0, (1 * time.Hour).Nanoseconds()),
end: time.Unix(0, (115 * time.Minute).Nanoseconds()),
},
},
},
"align_both": {
inp: interval{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (175 * time.Minute).Nanoseconds()),
},
expected: []interval{
{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (1 * time.Hour).Nanoseconds()).Add(-expectedSplitGap),
},
{
start: time.Unix(0, (1 * time.Hour).Nanoseconds()),
end: time.Unix(0, (2 * time.Hour).Nanoseconds()).Add(-expectedSplitGap),
},
{
start: time.Unix(0, (2 * time.Hour).Nanoseconds()),
end: time.Unix(0, (175 * time.Minute).Nanoseconds()),
},
},
},
"no_align": {
inp: interval{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (55 * time.Minute).Nanoseconds()),
},
expected: []interval{
{
start: time.Unix(0, (5 * time.Minute).Nanoseconds()),
end: time.Unix(0, (55 * time.Minute).Nanoseconds()),
},
},
},
} {
t.Run(fmt.Sprintf("%s - %s", name, requestType), func(t *testing.T) {
inp := tc.requestBuilderFunc(intervals.inp.start, intervals.inp.end)
var want []queryrangebase.Request
for _, interval := range intervals.expected {
want = append(want, tc.requestBuilderFunc(interval.start, interval.end))
}
splits, err := splitByTime(inp, time.Hour)
require.NoError(t, err)
require.Equal(t, want, splits)
})
}
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>
5 years ago
}
}
func Test_splitMetricQuery(t *testing.T) {
const seconds = 1e3 // 1e3 milliseconds per second.
for i, tc := range []struct {
input queryrangebase.Request
expected []queryrangebase.Request
interval time.Duration
}{
// the step is lower than the interval therefore we should split only once.
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, 60*time.Minute.Nanoseconds()),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, 60*time.Minute.Nanoseconds()),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 24 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(60*60, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(60*60, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(24*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(24*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 24 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(2*24*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix((24*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix((24 * 3600), 0),
EndTs: time.Unix((2 * 24 * 3600), 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 24 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(2*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix((3*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix((3 * 3600), 0),
EndTs: time.Unix((2 * 3 * 3600), 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(3*3600, 0),
EndTs: time.Unix(3*24*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(3*3600, 0),
EndTs: time.Unix((24*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(24*3600, 0),
EndTs: time.Unix((2*24*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(2*24*3600, 0),
EndTs: time.Unix(3*24*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 24 * time.Hour,
},
{
input: &LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix((3*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(3*3600, 0),
EndTs: time.Unix((2*3*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(2*3*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
// step not a multiple of interval
// start time already step aligned
{
input: &LokiRequest{
StartTs: time.Unix(2*3600-9, 0), // 2h mod 17s = 9s
EndTs: time.Unix(3*3*3600, 0),
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600-9, 0),
EndTs: time.Unix((3*3600)-5, 0), // 3h mod 17s = 5s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix((3*3600)+12, 0),
EndTs: time.Unix((2*3*3600)-10, 0), // 6h mod 17s = 10s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(2*3*3600+7, 0),
EndTs: time.Unix(3*3*3600+2, 0), // 9h mod 17s = 2s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
// end time already step aligned
{
input: &LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600+2, 0), // 9h mod 17s = 2s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600-9, 0), // 2h mod 17s = 9s
EndTs: time.Unix((3*3600)-5, 0), // 3h mod 17s = 5s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix((3*3600)+12, 0),
EndTs: time.Unix((2*3*3600)-10, 0), // 6h mod 17s = 10s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(2*3*3600+7, 0),
EndTs: time.Unix(3*3*3600+2, 0),
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
// start & end time not aligned with step
{
input: &LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600-9, 0), // 2h mod 17s = 9s
EndTs: time.Unix((3*3600)-5, 0), // 3h mod 17s = 5s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix((3*3600)+12, 0),
EndTs: time.Unix((2*3*3600)-10, 0), // 6h mod 17s = 10s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(2*3*3600+7, 0),
EndTs: time.Unix(3*3*3600+2, 0), // 9h mod 17s = 2s
Step: 17 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 3 * time.Hour,
},
// step larger than split interval
{
input: &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(25*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(6*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(6*3600, 0),
EndTs: time.Unix(12*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(12*3600, 0),
EndTs: time.Unix(18*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(18*3600, 0),
EndTs: time.Unix(24*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
&LokiRequest{
StartTs: time.Unix(24*3600, 0),
EndTs: time.Unix(30*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 15 * time.Minute,
},
{
input: &LokiRequest{
StartTs: time.Unix(1*3600, 0),
EndTs: time.Unix(3*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(6*3600, 0),
Step: 6 * 3600 * seconds,
Query: `rate({app="foo"}[1m])`,
},
},
interval: 15 * time.Minute,
},
// reduce split by to 6h instead of 1h
{
input: &LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[6h])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix((6*3600)-15, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[6h])`,
},
&LokiRequest{
StartTs: time.Unix(6*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[6h])`,
},
},
interval: 1 * time.Hour,
},
// range vector too large we don't want to split it
{
input: &LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[7d])`,
},
expected: []queryrangebase.Request{
&LokiRequest{
StartTs: time.Unix(2*3600, 0),
EndTs: time.Unix(3*3*3600, 0),
Step: 15 * seconds,
Query: `rate({app="foo"}[7d])`,
},
},
interval: 15 * time.Minute,
},
} {
t.Run(strconv.Itoa(i), func(t *testing.T) {
splits, err := splitMetricByTime(tc.input, tc.interval)
require.NoError(t, err)
for i, s := range splits {
s := s.(*LokiRequest)
t.Logf(" want: %d start:%s end:%s \n", i, s.StartTs, s.EndTs)
}
require.Equal(t, tc.expected, splits)
})
}
}
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>
5 years ago
func Test_splitByInterval_Do(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: r.(*LokiRequest).Direction,
Limit: r.(*LokiRequest).Limit,
Version: uint32(loghttp.VersionV1),
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, r.(*LokiRequest).StartTs.UnixNano()), Line: fmt.Sprintf("%d", r.(*LokiRequest).StartTs.UnixNano())},
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>
5 years ago
},
},
},
},
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour)
split := SplitByIntervalMiddleware(
testSchemas,
l,
DefaultCodec,
splitByTime,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
nilMetrics,
).Wrap(next)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
tests := []struct {
name string
req *LokiRequest
want *LokiResponse
}{
{
"backward",
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Query: "",
Limit: 1000,
Step: 1,
Direction: logproto.BACKWARD,
Path: "/api/prom/query_range",
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: 1000,
Version: 1,
Statistics: stats.Result{Summary: stats.Summary{Splits: 4}},
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>
5 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 3*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 3*time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, 2*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 2*time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, 0), Line: fmt.Sprintf("%d", 0)},
},
},
},
},
},
},
{
"forward",
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Query: "",
Limit: 1000,
Step: 1,
Direction: logproto.FORWARD,
Path: "/api/prom/query_range",
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.FORWARD,
Statistics: stats.Result{Summary: stats.Summary{Splits: 4}},
Limit: 1000,
Version: 1,
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>
5 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 0), Line: fmt.Sprintf("%d", 0)},
{Timestamp: time.Unix(0, time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, 2*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 2*time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, 3*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 3*time.Hour.Nanoseconds())},
},
},
},
},
},
},
{
"forward limited",
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Query: "",
Limit: 2,
Step: 1,
Direction: logproto.FORWARD,
Path: "/api/prom/query_range",
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.FORWARD,
Limit: 2,
Version: 1,
Statistics: stats.Result{Summary: stats.Summary{Splits: 2}},
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>
5 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 0), Line: fmt.Sprintf("%d", 0)},
{Timestamp: time.Unix(0, time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", time.Hour.Nanoseconds())},
},
},
},
},
},
},
{
"backward limited",
&LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Query: "",
Limit: 2,
Step: 1,
Direction: logproto.BACKWARD,
Path: "/api/prom/query_range",
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: 2,
Version: 1,
Statistics: stats.Result{Summary: stats.Summary{Splits: 2}},
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>
5 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 3*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 3*time.Hour.Nanoseconds())},
{Timestamp: time.Unix(0, 2*time.Hour.Nanoseconds()), Line: fmt.Sprintf("%d", 2*time.Hour.Nanoseconds())},
},
},
},
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
res, err := split.Do(ctx, tt.req)
require.NoError(t, err)
require.Equal(t, tt.want, res)
})
}
}
func Test_series_splitByInterval_Do(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &LokiSeriesResponse{
Status: "success",
Version: uint32(loghttp.VersionV1),
Data: []logproto.SeriesIdentifier{
{
Labels: map[string]string{"filename": "/var/hostlog/apport.log", "job": "varlogs"},
},
{
Labels: map[string]string{"filename": "/var/hostlog/test.log", "job": "varlogs"},
},
{
Labels: map[string]string{"filename": "/var/hostlog/test.log", "job": "varlogs"},
},
},
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour)
split := SplitByIntervalMiddleware(
testSchemas,
l,
DefaultCodec,
splitByTime,
nilMetrics,
).Wrap(next)
tests := []struct {
name string
req *LokiSeriesRequest
want *LokiSeriesResponse
}{
{
"backward",
&LokiSeriesRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Match: []string{`{job="varlogs"}`},
Path: "/loki/api/v1/series",
},
&LokiSeriesResponse{
Statistics: stats.Result{Summary: stats.Summary{Splits: 4}},
Status: "success",
Version: 1,
Data: []logproto.SeriesIdentifier{
{
Labels: map[string]string{"filename": "/var/hostlog/apport.log", "job": "varlogs"},
},
{
Labels: map[string]string{"filename": "/var/hostlog/test.log", "job": "varlogs"},
},
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
res, err := split.Do(ctx, tt.req)
require.NoError(t, err)
require.Equal(t, tt.want, res)
})
}
}
func Test_seriesvolume_splitByInterval_Do(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
setup := func(next queryrangebase.Handler, l Limits) queryrangebase.Handler {
return SplitByIntervalMiddleware(
testSchemas,
l,
DefaultCodec,
splitByTime,
nilMetrics,
).Wrap(next)
}
t.Run("volumes", func(t *testing.T) {
from := model.TimeFromUnixNano(start.UnixNano())
through := model.TimeFromUnixNano(end.UnixNano())
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Volume: 38},
{Name: `{bar="baz"}`, Volume: 28},
},
Limit: 2},
Headers: nil,
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour)
split := setup(next, l)
req := &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: "{}",
Limit: 2,
}
res, err := split.Do(ctx, req)
require.NoError(t, err)
response := res.(*VolumeResponse)
require.Len(t, response.Response.Volumes, 2)
require.Equal(t, response.Response, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: "{foo=\"bar\"}", Volume: 76},
{Name: "{bar=\"baz\"}", Volume: 56},
},
Limit: 2,
})
})
t.Run("volumes with limits", func(t *testing.T) {
from := model.TimeFromUnixNano(start.UnixNano())
through := model.TimeFromUnixNano(end.UnixNano())
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Volume: 38},
{Name: `{bar="baz"}`, Volume: 28},
{Name: `{foo="bar"}`, Volume: 38},
{Name: `{fizz="buzz"}`, Volume: 28},
},
Limit: 1},
Headers: nil,
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour)
split := setup(next, l)
req := &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: "{}",
Limit: 1,
}
res, err := split.Do(ctx, req)
require.NoError(t, err)
response := res.(*VolumeResponse)
require.Len(t, response.Response.Volumes, 1)
require.Equal(t, response.Response, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: "{foo=\"bar\"}", Volume: 152},
},
Limit: 1,
})
})
// This will never happen because we hardcode 24h spit by for this code path
// in the middleware. However, that split by is not validated here, so we either
// need to support this case or error.
t.Run("volumes with a query split by of 0", func(t *testing.T) {
from := model.TimeFromUnixNano(start.UnixNano())
through := model.TimeFromUnixNano(end.UnixNano())
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Volume: 38},
{Name: `{bar="baz"}`, Volume: 28},
},
Limit: 2},
Headers: nil,
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, 0)
split := setup(next, l)
req := &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: "{}",
Limit: 2,
}
res, err := split.Do(ctx, req)
require.NoError(t, err)
response := res.(*VolumeResponse)
require.Len(t, response.Response.Volumes, 2)
require.Contains(t, response.Response.Volumes, logproto.Volume{Name: `{foo="bar"}`, Volume: 38})
require.Contains(t, response.Response.Volumes, logproto.Volume{Name: `{bar="baz"}`, Volume: 28})
})
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>
5 years ago
}
func Test_ExitEarly(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
var callCt int
var mtx sync.Mutex
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
time.Sleep(time.Millisecond) // artificial delay to minimize race condition exposure in test
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>
5 years ago
mtx.Lock()
defer mtx.Unlock()
callCt++
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>
5 years ago
return &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: r.(*LokiRequest).Direction,
Limit: r.(*LokiRequest).Limit,
Version: uint32(loghttp.VersionV1),
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{
Timestamp: time.Unix(0, r.(*LokiRequest).StartTs.UnixNano()),
Line: fmt.Sprintf("%d", r.(*LokiRequest).StartTs.UnixNano()),
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>
5 years ago
},
},
},
},
},
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour)
split := SplitByIntervalMiddleware(
testSchemas,
l,
DefaultCodec,
splitByTime,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
nilMetrics,
).Wrap(next)
Loki Query Frontend (#1442) * Adds frontend to Loki. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improves tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sneaky bug in entries sorting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tweak jsonnet deployments and add a way to lint/fmt. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fix timezone issue. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tests and rollback change in loghttp package. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a flaky test that might run one more goroutine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes windows build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve tracing in the split by interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add test stream to proto conversion. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes flappy retry test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove err shadowing in stopQueryFrontend as it was confusing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor grpc message size in the libsonnet config file. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Don't check auth header for GRPC TransferChunks. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Query frontend (#3) * frontend codec merging optimizations * codec benchmarks * removes unused bounds code in queryrange ordering * [wip] splitby uses channels instead of sub batching intervals * splitBy channel limit test * single allocation for merging entries from a single stream * skip merging loki responses when limit is already hit * removes checks for unlimited queries in queryrange * removes splitByInterval{,.interval} spans * removes interval_batch_size from jsonnet lib * moves benchmark utils to own file * renames markers -> entries * priority queue comments * Removes unused logRequest. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Sets the cache interval to the same split interval. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Missing import libsonnet for the frontend. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Frontend should not be a cluster IP. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
req := &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (4 * time.Hour).Nanoseconds()),
Query: "",
Limit: 2,
Step: 1,
Direction: logproto.FORWARD,
Path: "/api/prom/query_range",
}
expected := &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.FORWARD,
Limit: 2,
Version: 1,
Statistics: stats.Result{
Summary: stats.Summary{
Splits: 2,
},
},
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>
5 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{
Timestamp: time.Unix(0, 0),
Line: fmt.Sprintf("%d", 0),
},
{
Timestamp: time.Unix(0, time.Hour.Nanoseconds()),
Line: fmt.Sprintf("%d", time.Hour.Nanoseconds()),
},
},
},
},
},
}
res, err := split.Do(ctx, req)
require.Equal(t, int(req.Limit), callCt)
require.NoError(t, err)
require.Equal(t, expected, res)
}
func Test_DoesntDeadlock(t *testing.T) {
n := 10
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: r.(*LokiRequest).Direction,
Limit: r.(*LokiRequest).Limit,
Version: uint32(loghttp.VersionV1),
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{
Timestamp: time.Unix(0, r.(*LokiRequest).StartTs.UnixNano()),
Line: fmt.Sprintf("%d", r.(*LokiRequest).StartTs.UnixNano()),
},
},
},
},
},
}, nil
})
l := WithSplitByLimits(fakeLimits{maxQueryParallelism: n}, time.Hour)
split := SplitByIntervalMiddleware(
testSchemas,
l,
DefaultCodec,
splitByTime,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
nilMetrics,
).Wrap(next)
// split into n requests w/ n/2 limit, ensuring unused responses are cleaned up properly
req := &LokiRequest{
StartTs: time.Unix(0, 0),
EndTs: time.Unix(0, (time.Duration(n) * time.Hour).Nanoseconds()),
Query: "",
Limit: uint32(n / 2),
Step: 1,
Direction: logproto.FORWARD,
Path: "/api/prom/query_range",
}
ctx := user.InjectOrgID(context.Background(), "1")
startingGoroutines := runtime.NumGoroutine()
// goroutines shouldn't blow up across 100 rounds
for i := 0; i < 100; i++ {
res, err := split.Do(ctx, req)
require.NoError(t, err)
require.Equal(t, 1, len(res.(*LokiResponse).Data.Result))
require.Equal(t, n/2, len(res.(*LokiResponse).Data.Result[0].Entries))
}
runtime.GC()
endingGoroutines := runtime.NumGoroutine()
// give runtime a bit of slack when catching up -- this isn't an exact science :(
// Allow for 1% increase in goroutines
require.LessOrEqual(t, endingGoroutines, startingGoroutines*101/100)
}