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 55d41b9519.

* 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
pull/2138/head
Owen Diehl 6 years ago committed by GitHub
parent 156023ae9a
commit 89d80a6fc4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 23
      pkg/logcli/query/query.go
  2. 4
      pkg/loghttp/params.go
  3. 3
      pkg/loghttp/query.go
  4. 7
      pkg/logproto/extensions.go
  5. 221
      pkg/logproto/logproto.pb.go
  6. 2
      pkg/logproto/logproto.proto
  7. 26
      pkg/logql/astmapper.go
  8. 140
      pkg/logql/engine.go
  9. 40
      pkg/logql/engine_test.go
  10. 48
      pkg/logql/evaluator.go
  11. 67
      pkg/logql/matrix.go
  12. 115
      pkg/logql/matrix_test.go
  13. 4
      pkg/logql/metrics.go
  14. 344
      pkg/logql/sharding.go
  15. 109
      pkg/logql/sharding_test.go
  16. 152
      pkg/logql/shardmapper.go
  17. 84
      pkg/logql/shardmapper_test.go
  18. 56
      pkg/logql/stats/context.go
  19. 47
      pkg/logql/stats/context_test.go
  20. 172
      pkg/logql/test_utils.go
  21. 9
      pkg/loki/modules.go
  22. 37
      pkg/querier/http.go
  23. 3
      pkg/querier/querier.go
  24. 22
      pkg/querier/queryrange/codec.go
  25. 185
      pkg/querier/queryrange/downstreamer.go
  26. 344
      pkg/querier/queryrange/downstreamer_test.go
  27. 160
      pkg/querier/queryrange/queryrange.pb.go
  28. 1
      pkg/querier/queryrange/queryrange.proto
  29. 239
      pkg/querier/queryrange/querysharding.go
  30. 189
      pkg/querier/queryrange/querysharding_test.go
  31. 62
      pkg/querier/queryrange/roundtrip.go
  32. 15
      pkg/querier/queryrange/roundtrip_test.go
  33. 48
      pkg/querier/queryrange/split_by_interval.go
  34. 5
      pkg/querier/queryrange/split_by_interval_test.go
  35. 17
      pkg/storage/iterator.go
  36. 4
      pkg/storage/iterator_test.go
  37. 37
      pkg/storage/store.go
  38. 63
      pkg/storage/store_test.go
  39. 9
      pkg/storage/util_test.go
  40. 37
      production/ksonnet/loki/config.libsonnet
  41. 3
      production/ksonnet/loki/querier.libsonnet
  42. 14
      production/ksonnet/loki/query-frontend.libsonnet
  43. 18
      production/loki-mixin/dashboards.libsonnet

@ -119,10 +119,29 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string
eng := logql.NewEngine(conf.Querier.Engine, querier)
var query logql.Query
if q.isInstant() {
query = eng.NewInstantQuery(q.QueryString, q.Start, q.resultsDirection(), uint32(q.Limit))
query = eng.Query(logql.NewLiteralParams(
q.QueryString,
q.Start,
q.Start,
0,
0,
q.resultsDirection(),
uint32(q.Limit),
nil,
))
} else {
query = eng.NewRangeQuery(q.QueryString, q.Start, q.End, q.Step, q.Interval, q.resultsDirection(), uint32(q.Limit))
query = eng.Query(logql.NewLiteralParams(
q.QueryString,
q.Start,
q.End,
q.Step,
q.Interval,
q.resultsDirection(),
uint32(q.Limit),
nil,
))
}
// execute the query

@ -44,6 +44,10 @@ func direction(r *http.Request) (logproto.Direction, error) {
return parseDirection(r.Form.Get("direction"), logproto.BACKWARD)
}
func shards(r *http.Request) []string {
return r.Form["shards"]
}
func bounds(r *http.Request) (time.Time, time.Time, error) {
now := time.Now()
start, err := parseTimestamp(r.Form.Get("start"), now.Add(-defaultSince))

@ -244,6 +244,7 @@ type RangeQuery struct {
Query string
Direction logproto.Direction
Limit uint32
Shards []string
}
// ParseRangeQuery parses a RangeQuery request from an http request.
@ -280,6 +281,8 @@ func ParseRangeQuery(r *http.Request) (*RangeQuery, error) {
return nil, errNegativeStep
}
result.Shards = shards(r)
// For safety, limit the number of returned points per timeseries.
// This is sufficient for 60s resolution for a week or 1h resolution for a year.
if (result.End.Sub(result.Start) / result.Step) > 11000 {

@ -2,6 +2,7 @@ package logproto
import "github.com/prometheus/prometheus/pkg/labels"
// Note, this is not very efficient and use should be minimized as it requires label construction on each comparison
type SeriesIdentifiers []SeriesIdentifier
func (ids SeriesIdentifiers) Len() int { return len(ids) }
@ -10,3 +11,9 @@ func (ids SeriesIdentifiers) Less(i, j int) bool {
a, b := labels.FromMap(ids[i].Labels), labels.FromMap(ids[j].Labels)
return labels.Compare(a, b) <= 0
}
type Streams []Stream
func (xs Streams) Len() int { return len(xs) }
func (xs Streams) Swap(i, j int) { xs[i], xs[j] = xs[j], xs[i] }
func (xs Streams) Less(i, j int) bool { return xs[i].Labels <= xs[j].Labels }

@ -131,6 +131,7 @@ type QueryRequest struct {
Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"`
End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"`
Direction Direction `protobuf:"varint,5,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"`
Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"`
}
func (m *QueryRequest) Reset() { *m = QueryRequest{} }
@ -200,6 +201,13 @@ func (m *QueryRequest) GetDirection() Direction {
return FORWARD
}
func (m *QueryRequest) GetShards() []string {
if m != nil {
return m.Shards
}
return nil
}
type QueryResponse struct {
Streams []Stream `protobuf:"bytes,1,rep,name=streams,proto3,customtype=Stream" json:"streams,omitempty"`
}
@ -1065,79 +1073,81 @@ func init() {
func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) }
var fileDescriptor_c28a5f14f1f4c79a = []byte{
// 1139 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0x4b, 0x6f, 0xdb, 0xc6,
0x13, 0xe7, 0x4a, 0x14, 0x2d, 0x8d, 0x1e, 0x11, 0x36, 0x8e, 0xad, 0xbf, 0xf2, 0x2f, 0x25, 0x10,
0x45, 0x22, 0xf4, 0x21, 0xb5, 0xee, 0xcb, 0x49, 0x5f, 0xb0, 0xec, 0x06, 0xb1, 0x5b, 0x20, 0x09,
0x6d, 0x20, 0x40, 0x80, 0x22, 0xa0, 0xcd, 0xb5, 0x4c, 0x58, 0x22, 0x15, 0x72, 0x65, 0xc0, 0xb7,
0x7e, 0x80, 0x16, 0xc8, 0xad, 0x87, 0x7e, 0x81, 0xa2, 0x87, 0x7e, 0x0e, 0x9f, 0x0a, 0x1f, 0x83,
0x1e, 0xd4, 0x5a, 0xbe, 0x14, 0x06, 0x0a, 0xe4, 0x23, 0x14, 0xfb, 0x20, 0xb9, 0x52, 0xec, 0xa2,
0xca, 0x45, 0xda, 0x99, 0x9d, 0xd9, 0x9d, 0xf9, 0xcd, 0x6f, 0x66, 0x09, 0x37, 0x87, 0x87, 0xbd,
0x4e, 0x3f, 0xe8, 0x0d, 0xc3, 0x80, 0x06, 0xc9, 0xa2, 0xcd, 0x7f, 0x71, 0x3e, 0x96, 0xeb, 0x8d,
0x5e, 0x10, 0xf4, 0xfa, 0xa4, 0xc3, 0xa5, 0xdd, 0xd1, 0x7e, 0x87, 0x7a, 0x03, 0x12, 0x51, 0x67,
0x30, 0x14, 0xa6, 0xf5, 0x77, 0x7b, 0x1e, 0x3d, 0x18, 0xed, 0xb6, 0xf7, 0x82, 0x41, 0xa7, 0x17,
0xf4, 0x82, 0xd4, 0x92, 0x49, 0xe2, 0x74, 0xb6, 0x12, 0xe6, 0xd6, 0x63, 0x28, 0x3e, 0x1c, 0x45,
0x07, 0x36, 0x79, 0x36, 0x22, 0x11, 0xc5, 0xf7, 0x61, 0x21, 0xa2, 0x21, 0x71, 0x06, 0x51, 0x0d,
0x35, 0xb3, 0xad, 0xe2, 0xca, 0x72, 0x3b, 0x09, 0x65, 0x9b, 0x6f, 0xac, 0xb9, 0xce, 0x90, 0x92,
0xb0, 0x7b, 0xe3, 0xf7, 0x71, 0xc3, 0x10, 0xaa, 0x8b, 0x71, 0x23, 0xf6, 0xb2, 0xe3, 0x85, 0x55,
0x81, 0x92, 0x38, 0x38, 0x1a, 0x06, 0x7e, 0x44, 0xac, 0xbf, 0x11, 0x94, 0x1e, 0x8d, 0x48, 0x78,
0x1c, 0x5f, 0x55, 0x87, 0x7c, 0x44, 0xfa, 0x64, 0x8f, 0x06, 0x61, 0x0d, 0x35, 0x51, 0xab, 0x60,
0x27, 0x32, 0x5e, 0x84, 0x5c, 0xdf, 0x1b, 0x78, 0xb4, 0x96, 0x69, 0xa2, 0x56, 0xd9, 0x16, 0x02,
0xbe, 0x0b, 0xb9, 0x88, 0x3a, 0x21, 0xad, 0x65, 0x9b, 0xa8, 0x55, 0x5c, 0xa9, 0xb7, 0x05, 0x16,
0xed, 0x38, 0xc3, 0xf6, 0x4e, 0x8c, 0x45, 0x37, 0x7f, 0x32, 0x6e, 0x68, 0xcf, 0xff, 0x68, 0x20,
0x5b, 0xb8, 0xe0, 0x8f, 0x21, 0x4b, 0x7c, 0xb7, 0xa6, 0xcf, 0xe1, 0xc9, 0x1c, 0xf0, 0xfb, 0x50,
0x70, 0xbd, 0x90, 0xec, 0x51, 0x2f, 0xf0, 0x6b, 0xb9, 0x26, 0x6a, 0x55, 0x56, 0xae, 0xa7, 0x90,
0x6c, 0xc4, 0x5b, 0x76, 0x6a, 0xb5, 0xa5, 0xe7, 0x8d, 0xea, 0x82, 0x65, 0x43, 0x59, 0xa6, 0x2b,
0x00, 0xc0, 0x6b, 0xff, 0x19, 0xda, 0xca, 0xc9, 0xb8, 0x81, 0x52, 0x78, 0x53, 0x4c, 0x7f, 0x45,
0x50, 0xfa, 0xc6, 0xd9, 0x25, 0xfd, 0x18, 0x43, 0x0c, 0xba, 0xef, 0x0c, 0x88, 0xc4, 0x8f, 0xaf,
0xf1, 0x12, 0x18, 0x47, 0x4e, 0x7f, 0x44, 0x22, 0x0e, 0x5e, 0xde, 0x96, 0xd2, 0xbc, 0xe8, 0xa1,
0xd7, 0x46, 0x0f, 0x25, 0xe8, 0x59, 0xb7, 0xa1, 0x2c, 0xe3, 0x95, 0x20, 0xa4, 0xc1, 0x31, 0x0c,
0x0a, 0x71, 0x70, 0xd6, 0x11, 0x94, 0xa7, 0x30, 0xc0, 0x16, 0x18, 0x7d, 0xe6, 0x19, 0x89, 0xdc,
0xba, 0x70, 0x31, 0x6e, 0x48, 0x8d, 0x2d, 0xff, 0x19, 0xa2, 0xc4, 0xa7, 0xa1, 0xc7, 0x53, 0x65,
0x88, 0x2e, 0xa5, 0x88, 0x7e, 0xe5, 0xd3, 0xf0, 0x38, 0x06, 0xf4, 0x1a, 0xab, 0x29, 0x63, 0xa9,
0x34, 0xb7, 0xe3, 0x85, 0x75, 0x04, 0x25, 0xd5, 0x12, 0xdf, 0x87, 0x42, 0xd2, 0x50, 0xfc, 0xe6,
0x7f, 0x4f, 0xb7, 0x22, 0x0f, 0xce, 0xd0, 0x88, 0x27, 0x9d, 0x3a, 0xe3, 0xff, 0x83, 0xde, 0xf7,
0x7c, 0xc2, 0x8b, 0x50, 0xe8, 0xe6, 0x2f, 0xc6, 0x0d, 0x2e, 0xdb, 0xfc, 0xd7, 0xfa, 0x11, 0x41,
0x71, 0xc7, 0xf1, 0x92, 0x42, 0x2e, 0x42, 0xee, 0x19, 0x63, 0x8b, 0xac, 0xa4, 0x10, 0x58, 0x8b,
0xb8, 0xa4, 0xef, 0x1c, 0xdf, 0x0b, 0x42, 0x5e, 0xb5, 0xb2, 0x9d, 0xc8, 0x69, 0x8b, 0xe8, 0x97,
0xb6, 0x48, 0x6e, 0xee, 0x16, 0xd9, 0xd2, 0xf3, 0x99, 0x6a, 0xd6, 0xfa, 0x1e, 0x41, 0x49, 0x44,
0x26, 0x4b, 0xf6, 0x29, 0x18, 0x82, 0x7f, 0x12, 0x8f, 0x2b, 0x69, 0x0b, 0x0a, 0x65, 0xa5, 0x0b,
0xfe, 0x12, 0x2a, 0x6e, 0x18, 0x0c, 0x87, 0xc4, 0xdd, 0x96, 0xdc, 0xcf, 0xcc, 0x72, 0x7f, 0x43,
0xdd, 0xb7, 0x67, 0xcc, 0xad, 0x9f, 0x10, 0x94, 0xb7, 0x09, 0x2f, 0x9a, 0x84, 0x2a, 0x49, 0x11,
0xbd, 0xf6, 0x14, 0xc8, 0xcc, 0x3b, 0x05, 0x96, 0xc0, 0xe8, 0x85, 0xc1, 0x68, 0x18, 0xd5, 0xb2,
0x82, 0xb6, 0x42, 0xb2, 0xb6, 0xa0, 0x12, 0x07, 0x27, 0xd1, 0x5a, 0x05, 0x23, 0xe2, 0x1a, 0xd9,
0xe4, 0x75, 0x05, 0x2d, 0xae, 0xdf, 0x74, 0x89, 0x4f, 0xbd, 0x7d, 0x8f, 0x84, 0x5d, 0x9d, 0x5d,
0x62, 0x4b, 0x7b, 0xeb, 0x07, 0x04, 0xd5, 0x59, 0x13, 0xfc, 0x85, 0xd2, 0x06, 0xec, 0xb8, 0x5b,
0x57, 0x1f, 0xd7, 0xe6, 0x9d, 0x16, 0x71, 0x3a, 0xc7, 0x2d, 0x52, 0xbf, 0x03, 0x45, 0x45, 0x8d,
0xab, 0x90, 0x3d, 0x24, 0x31, 0xc9, 0xd8, 0x92, 0xd1, 0x88, 0xb7, 0xa0, 0xe0, 0xa9, 0x2d, 0x84,
0xbb, 0x99, 0x55, 0xc4, 0x28, 0x5a, 0x9e, 0xaa, 0x0d, 0x5e, 0x05, 0x7d, 0x3f, 0x0c, 0x06, 0x73,
0x01, 0xcf, 0x3d, 0xf0, 0x87, 0x90, 0xa1, 0xc1, 0x5c, 0xb0, 0x67, 0x68, 0xc0, 0x50, 0x97, 0xc9,
0x67, 0x79, 0x70, 0x52, 0xb2, 0x7e, 0x41, 0x70, 0x8d, 0xf9, 0x08, 0x04, 0xd6, 0x0f, 0x46, 0xfe,
0x21, 0x6e, 0x41, 0x95, 0xdd, 0xf4, 0xd4, 0xf3, 0x7b, 0x24, 0xa2, 0x24, 0x7c, 0xea, 0xb9, 0x32,
0xcd, 0x0a, 0xd3, 0x6f, 0x4a, 0xf5, 0xa6, 0x8b, 0x97, 0x61, 0x61, 0x14, 0x09, 0x03, 0x91, 0xb3,
0xc1, 0xc4, 0x4d, 0x17, 0xbf, 0xad, 0x5c, 0xc7, 0xb0, 0x56, 0xe6, 0x3c, 0xc7, 0xf0, 0xa1, 0xe3,
0x85, 0xc9, 0xec, 0xb9, 0x0d, 0xc6, 0x1e, 0xbb, 0x38, 0xaa, 0xe9, 0xdc, 0xf8, 0x5a, 0x6a, 0xcc,
0x03, 0xb2, 0xe5, 0xb6, 0xf5, 0x11, 0x14, 0x12, 0xef, 0x4b, 0xe7, 0xf5, 0xa5, 0x15, 0xb0, 0x6e,
0x42, 0x4e, 0x24, 0x86, 0x41, 0x77, 0x1d, 0xea, 0x70, 0x97, 0x92, 0xcd, 0xd7, 0x56, 0x0d, 0x96,
0x76, 0x42, 0xc7, 0x8f, 0xf6, 0x49, 0xc8, 0x8d, 0x12, 0xfa, 0x59, 0x37, 0xe0, 0x3a, 0x6b, 0x5e,
0x12, 0x46, 0xeb, 0xc1, 0xc8, 0xa7, 0xb2, 0x67, 0xac, 0x77, 0x60, 0x71, 0x5a, 0x2d, 0xd9, 0xba,
0x08, 0xb9, 0x3d, 0xa6, 0xe0, 0xa7, 0x97, 0x6d, 0x21, 0xbc, 0x75, 0x0b, 0x0a, 0xc9, 0xc3, 0x86,
0x8b, 0xb0, 0x70, 0xef, 0x81, 0xfd, 0x78, 0xcd, 0xde, 0xa8, 0x6a, 0xb8, 0x04, 0xf9, 0xee, 0xda,
0xfa, 0xd7, 0x5c, 0x42, 0x2b, 0x6b, 0x60, 0xb0, 0x27, 0x9e, 0x84, 0xf8, 0x13, 0xd0, 0xd9, 0x0a,
0xdf, 0x48, 0x51, 0x50, 0xbe, 0x2a, 0xea, 0x4b, 0xb3, 0x6a, 0x19, 0xad, 0xb6, 0xf2, 0x5b, 0x06,
0x16, 0xd8, 0x33, 0xc9, 0xb8, 0xfe, 0x19, 0xe4, 0xf8, 0x8b, 0x89, 0x15, 0x73, 0xf5, 0x8b, 0xa1,
0xbe, 0xfc, 0x8a, 0x3e, 0x3e, 0xe7, 0x3d, 0xc4, 0xc6, 0x02, 0xc7, 0x59, 0xf5, 0x56, 0xdf, 0x4a,
0xd5, 0x7b, 0xea, 0x4d, 0xb2, 0x34, 0x7c, 0x07, 0x74, 0x06, 0x8f, 0x1a, 0xbe, 0x32, 0x9c, 0xd5,
0xf0, 0xd5, 0xc9, 0xc8, 0xaf, 0xfd, 0x1c, 0x0c, 0x41, 0x43, 0xbc, 0x3c, 0xdb, 0x9a, 0xb1, 0x7b,
0xed, 0xd5, 0x8d, 0xe4, 0xe6, 0x07, 0x62, 0xd8, 0xc6, 0x85, 0xc1, 0x6f, 0x4c, 0x5f, 0x35, 0x53,
0xc7, 0xba, 0x79, 0xd5, 0x76, 0x02, 0xe8, 0xb7, 0x90, 0x8f, 0xb9, 0x8e, 0x1f, 0x41, 0x65, 0x9a,
0x26, 0xf8, 0x7f, 0x8a, 0xff, 0x74, 0x03, 0xd5, 0x9b, 0xca, 0xd6, 0xe5, 0xdc, 0xd2, 0x5a, 0xa8,
0xfb, 0xe4, 0xf4, 0xcc, 0xd4, 0x5e, 0x9c, 0x99, 0xda, 0xcb, 0x33, 0x13, 0x7d, 0x37, 0x31, 0xd1,
0xcf, 0x13, 0x13, 0x9d, 0x4c, 0x4c, 0x74, 0x3a, 0x31, 0xd1, 0x9f, 0x13, 0x13, 0xfd, 0x35, 0x31,
0xb5, 0x97, 0x13, 0x13, 0x3d, 0x3f, 0x37, 0xb5, 0xd3, 0x73, 0x53, 0x7b, 0x71, 0x6e, 0x6a, 0x4f,
0xde, 0x54, 0xbf, 0x49, 0x43, 0x67, 0xdf, 0xf1, 0x9d, 0x4e, 0x3f, 0x38, 0xf4, 0x3a, 0xea, 0x37,
0xef, 0xae, 0xc1, 0xff, 0x3e, 0xf8, 0x27, 0x00, 0x00, 0xff, 0xff, 0x46, 0x78, 0x5f, 0x09, 0x0a,
0x0b, 0x00, 0x00,
// 1169 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xcd, 0x6e, 0xdb, 0x46,
0x10, 0xd6, 0x4a, 0x14, 0x25, 0x8d, 0x7e, 0x22, 0x6c, 0x1c, 0x5b, 0x55, 0x5a, 0x4a, 0x20, 0x8a,
0x44, 0x68, 0x5d, 0xa9, 0x75, 0xff, 0x9c, 0xf4, 0x0f, 0x96, 0xdd, 0x20, 0x76, 0x0b, 0x24, 0xa1,
0x0d, 0x04, 0x08, 0x50, 0x04, 0xb4, 0xb9, 0x96, 0x09, 0x4b, 0xa4, 0x42, 0xae, 0x0c, 0xf8, 0xd6,
0x07, 0x68, 0x81, 0xdc, 0x7a, 0xc8, 0x0b, 0x14, 0x3d, 0xf4, 0x39, 0x7c, 0x2a, 0x7c, 0x0c, 0x7a,
0x50, 0x6b, 0xf9, 0x52, 0xf8, 0x94, 0x47, 0x28, 0xf6, 0x87, 0xe4, 0x4a, 0xb1, 0x8b, 0x2a, 0x17,
0x69, 0x67, 0x76, 0x66, 0x67, 0xe6, 0x9b, 0x6f, 0x76, 0x09, 0x37, 0x87, 0x87, 0xbd, 0x4e, 0xdf,
0xef, 0x0d, 0x03, 0x9f, 0xfa, 0xf1, 0xa2, 0xcd, 0x7f, 0x71, 0x3e, 0x92, 0xeb, 0x8d, 0x9e, 0xef,
0xf7, 0xfa, 0xa4, 0xc3, 0xa5, 0xdd, 0xd1, 0x7e, 0x87, 0xba, 0x03, 0x12, 0x52, 0x7b, 0x30, 0x14,
0xa6, 0xf5, 0x0f, 0x7a, 0x2e, 0x3d, 0x18, 0xed, 0xb6, 0xf7, 0xfc, 0x41, 0xa7, 0xe7, 0xf7, 0xfc,
0xc4, 0x92, 0x49, 0xe2, 0x74, 0xb6, 0x12, 0xe6, 0xe6, 0x63, 0x28, 0x3e, 0x1c, 0x85, 0x07, 0x16,
0x79, 0x36, 0x22, 0x21, 0xc5, 0xf7, 0x21, 0x17, 0xd2, 0x80, 0xd8, 0x83, 0xb0, 0x86, 0x9a, 0x99,
0x56, 0x71, 0x65, 0xa9, 0x1d, 0xa7, 0xb2, 0xcd, 0x37, 0xd6, 0x1c, 0x7b, 0x48, 0x49, 0xd0, 0xbd,
0xf1, 0xe7, 0xb8, 0xa1, 0x0b, 0xd5, 0xc5, 0xb8, 0x11, 0x79, 0x59, 0xd1, 0xc2, 0xac, 0x40, 0x49,
0x1c, 0x1c, 0x0e, 0x7d, 0x2f, 0x24, 0xe6, 0x8b, 0x34, 0x94, 0x1e, 0x8d, 0x48, 0x70, 0x1c, 0x85,
0xaa, 0x43, 0x3e, 0x24, 0x7d, 0xb2, 0x47, 0xfd, 0xa0, 0x86, 0x9a, 0xa8, 0x55, 0xb0, 0x62, 0x19,
0x2f, 0x40, 0xb6, 0xef, 0x0e, 0x5c, 0x5a, 0x4b, 0x37, 0x51, 0xab, 0x6c, 0x09, 0x01, 0xdf, 0x85,
0x6c, 0x48, 0xed, 0x80, 0xd6, 0x32, 0x4d, 0xd4, 0x2a, 0xae, 0xd4, 0xdb, 0x02, 0x8b, 0x76, 0x54,
0x61, 0x7b, 0x27, 0xc2, 0xa2, 0x9b, 0x3f, 0x19, 0x37, 0x52, 0xcf, 0xff, 0x6a, 0x20, 0x4b, 0xb8,
0xe0, 0xcf, 0x20, 0x43, 0x3c, 0xa7, 0xa6, 0xcd, 0xe1, 0xc9, 0x1c, 0xf0, 0x47, 0x50, 0x70, 0xdc,
0x80, 0xec, 0x51, 0xd7, 0xf7, 0x6a, 0xd9, 0x26, 0x6a, 0x55, 0x56, 0xae, 0x27, 0x90, 0x6c, 0x44,
0x5b, 0x56, 0x62, 0x85, 0x97, 0x41, 0x0f, 0x0f, 0xec, 0xc0, 0x09, 0x6b, 0xb9, 0x66, 0xa6, 0x55,
0xe8, 0x2e, 0x5c, 0x8c, 0x1b, 0x55, 0xa1, 0x59, 0xf6, 0x07, 0x2e, 0x25, 0x83, 0x21, 0x3d, 0xb6,
0xa4, 0xcd, 0x96, 0x96, 0xd7, 0xab, 0x39, 0xd3, 0x82, 0xb2, 0x04, 0x47, 0xc0, 0x85, 0xd7, 0xfe,
0x77, 0x23, 0x2a, 0x27, 0xe3, 0x06, 0x4a, 0x9a, 0x91, 0x74, 0xe0, 0x77, 0x04, 0xa5, 0xef, 0xed,
0x5d, 0xd2, 0x8f, 0x10, 0xc7, 0xa0, 0x79, 0xf6, 0x80, 0x48, 0xb4, 0xf9, 0x1a, 0x2f, 0x82, 0x7e,
0x64, 0xf7, 0x47, 0x24, 0xe4, 0x50, 0xe7, 0x2d, 0x29, 0xcd, 0x8b, 0x35, 0x7a, 0x63, 0xac, 0x51,
0x8c, 0xb5, 0x79, 0x1b, 0xca, 0x32, 0x5f, 0x09, 0x42, 0x92, 0x1c, 0xc3, 0xa0, 0x10, 0x25, 0x67,
0x1e, 0x41, 0x79, 0x0a, 0x03, 0x6c, 0x82, 0xde, 0x67, 0x9e, 0xa1, 0xa8, 0xad, 0x0b, 0x17, 0xe3,
0x86, 0xd4, 0x58, 0xf2, 0x9f, 0x21, 0x4a, 0x3c, 0x1a, 0xb8, 0xbc, 0x54, 0x86, 0xe8, 0x62, 0x82,
0xe8, 0xb7, 0x1e, 0x0d, 0x8e, 0x23, 0x40, 0xaf, 0x31, 0x06, 0x30, 0x4e, 0x4b, 0x73, 0x2b, 0x5a,
0x98, 0x47, 0x50, 0x52, 0x2d, 0xf1, 0x7d, 0x28, 0xc4, 0xe3, 0xc7, 0x23, 0xff, 0x77, 0xb9, 0x15,
0x79, 0x70, 0x9a, 0x86, 0xbc, 0xe8, 0xc4, 0x19, 0xbf, 0x0d, 0x5a, 0xdf, 0xf5, 0x08, 0x6f, 0x42,
0xa1, 0x9b, 0xbf, 0x18, 0x37, 0xb8, 0x6c, 0xf1, 0x5f, 0xf3, 0x17, 0x04, 0xc5, 0x1d, 0xdb, 0x8d,
0x1b, 0xb9, 0x00, 0xd9, 0x67, 0x8c, 0x2d, 0xb2, 0x93, 0x42, 0x60, 0x03, 0xe5, 0x90, 0xbe, 0x7d,
0x7c, 0xcf, 0x0f, 0x78, 0xd7, 0xca, 0x56, 0x2c, 0x27, 0x03, 0xa5, 0x5d, 0x3a, 0x50, 0xd9, 0xb9,
0x07, 0x6a, 0x4b, 0xcb, 0xa7, 0xab, 0x19, 0xf3, 0x27, 0x04, 0x25, 0x91, 0x99, 0x6c, 0xd9, 0x17,
0xa0, 0x0b, 0xfe, 0x49, 0x3c, 0xae, 0xa4, 0x2d, 0x28, 0x94, 0x95, 0x2e, 0xf8, 0x1b, 0xa8, 0x38,
0x81, 0x3f, 0x1c, 0x12, 0x67, 0x5b, 0x72, 0x3f, 0x3d, 0xcb, 0xfd, 0x0d, 0x75, 0xdf, 0x9a, 0x31,
0x37, 0x5f, 0x20, 0x28, 0x6f, 0x13, 0xde, 0x34, 0x09, 0x55, 0x5c, 0x22, 0x7a, 0xe3, 0x3b, 0x23,
0x3d, 0xef, 0x9d, 0xb1, 0x08, 0x7a, 0x2f, 0xf0, 0x47, 0xc3, 0xb0, 0x96, 0x11, 0xb4, 0x15, 0x92,
0xb9, 0x05, 0x95, 0x28, 0x39, 0x89, 0xd6, 0x2a, 0xe8, 0x21, 0xd7, 0xc8, 0x21, 0xaf, 0x2b, 0x68,
0x71, 0xfd, 0xa6, 0x43, 0x3c, 0xea, 0xee, 0xbb, 0x24, 0xe8, 0x6a, 0x2c, 0x88, 0x25, 0xed, 0xcd,
0x9f, 0x11, 0x54, 0x67, 0x4d, 0xf0, 0xd7, 0xca, 0x18, 0xb0, 0xe3, 0x6e, 0x5d, 0x7d, 0x5c, 0x9b,
0x4f, 0x5a, 0xc8, 0xe9, 0x1c, 0x8d, 0x48, 0xfd, 0x0e, 0x14, 0x15, 0x35, 0xae, 0x42, 0xe6, 0x90,
0x44, 0x24, 0x63, 0x4b, 0x46, 0x23, 0x3e, 0x82, 0x82, 0xa7, 0x96, 0x10, 0xee, 0xa6, 0x57, 0x11,
0xa3, 0x68, 0x79, 0xaa, 0x37, 0x78, 0x15, 0xb4, 0xfd, 0xc0, 0x1f, 0xcc, 0x05, 0x3c, 0xf7, 0xc0,
0x9f, 0x40, 0x9a, 0xfa, 0x73, 0xc1, 0x9e, 0xa6, 0x3e, 0x43, 0x5d, 0x16, 0x9f, 0xe1, 0xc9, 0x49,
0xc9, 0xfc, 0x0d, 0xc1, 0x35, 0xe6, 0x23, 0x10, 0x58, 0x3f, 0x18, 0x79, 0x87, 0xb8, 0x05, 0x55,
0x16, 0xe9, 0xa9, 0xeb, 0xf5, 0x48, 0x48, 0x49, 0xf0, 0xd4, 0x75, 0x64, 0x99, 0x15, 0xa6, 0xdf,
0x94, 0xea, 0x4d, 0x07, 0x2f, 0x41, 0x6e, 0x14, 0x0a, 0x03, 0x51, 0xb3, 0xce, 0xc4, 0x4d, 0x07,
0xbf, 0xaf, 0x84, 0x63, 0x58, 0x2b, 0xaf, 0x02, 0xc7, 0xf0, 0xa1, 0xed, 0x06, 0xf1, 0xdd, 0x73,
0x1b, 0xf4, 0x3d, 0x16, 0x38, 0xac, 0x69, 0xdc, 0xf8, 0x5a, 0x62, 0xcc, 0x13, 0xb2, 0xe4, 0xb6,
0xf9, 0x29, 0x14, 0x62, 0xef, 0x4b, 0xef, 0xeb, 0x4b, 0x3b, 0x60, 0xde, 0x84, 0xac, 0x28, 0x0c,
0x83, 0xe6, 0xd8, 0xd4, 0xe6, 0x2e, 0x25, 0x8b, 0xaf, 0xcd, 0x1a, 0x2c, 0xee, 0x04, 0xb6, 0x17,
0xee, 0x93, 0x80, 0x1b, 0xc5, 0xf4, 0x33, 0x6f, 0xc0, 0x75, 0x36, 0xbc, 0x24, 0x08, 0xd7, 0xfd,
0x91, 0x47, 0xe5, 0xcc, 0x98, 0xcb, 0xb0, 0x30, 0xad, 0x96, 0x6c, 0x5d, 0x80, 0xec, 0x1e, 0x53,
0xf0, 0xd3, 0xcb, 0x96, 0x10, 0xde, 0xbb, 0x05, 0x85, 0xf8, 0x19, 0xc4, 0x45, 0xc8, 0xdd, 0x7b,
0x60, 0x3d, 0x5e, 0xb3, 0x36, 0xaa, 0x29, 0x5c, 0x82, 0x7c, 0x77, 0x6d, 0xfd, 0x3b, 0x2e, 0xa1,
0x95, 0x35, 0xd0, 0xd9, 0x07, 0x01, 0x09, 0xf0, 0xe7, 0xa0, 0xb1, 0x15, 0xbe, 0x91, 0xa0, 0xa0,
0x7c, 0x83, 0xd4, 0x17, 0x67, 0xd5, 0x32, 0xdb, 0xd4, 0xca, 0x1f, 0x69, 0xc8, 0xb1, 0x67, 0x92,
0x71, 0xfd, 0x4b, 0xc8, 0xf2, 0x17, 0x13, 0x2b, 0xe6, 0xea, 0xf7, 0x45, 0x7d, 0xe9, 0x35, 0x7d,
0x74, 0xce, 0x87, 0x88, 0x5d, 0x0b, 0x1c, 0x67, 0xd5, 0x5b, 0x7d, 0x2b, 0x55, 0xef, 0xa9, 0x37,
0xc9, 0x4c, 0xe1, 0x3b, 0xa0, 0x31, 0x78, 0xd4, 0xf4, 0x95, 0xcb, 0x59, 0x4d, 0x5f, 0xbd, 0x19,
0x79, 0xd8, 0xaf, 0x40, 0x17, 0x34, 0xc4, 0x4b, 0xb3, 0xa3, 0x19, 0xb9, 0xd7, 0x5e, 0xdf, 0x88,
0x23, 0x3f, 0x10, 0x97, 0x6d, 0xd4, 0x18, 0xfc, 0xce, 0x74, 0xa8, 0x99, 0x3e, 0xd6, 0x8d, 0xab,
0xb6, 0x63, 0x40, 0x7f, 0x80, 0x7c, 0xc4, 0x75, 0xfc, 0x08, 0x2a, 0xd3, 0x34, 0xc1, 0x6f, 0x29,
0xfe, 0xd3, 0x03, 0x54, 0x6f, 0x2a, 0x5b, 0x97, 0x73, 0x2b, 0xd5, 0x42, 0xdd, 0x27, 0xa7, 0x67,
0x46, 0xea, 0xe5, 0x99, 0x91, 0x7a, 0x75, 0x66, 0xa0, 0x1f, 0x27, 0x06, 0xfa, 0x75, 0x62, 0xa0,
0x93, 0x89, 0x81, 0x4e, 0x27, 0x06, 0xfa, 0x7b, 0x62, 0xa0, 0x7f, 0x26, 0x46, 0xea, 0xd5, 0xc4,
0x40, 0xcf, 0xcf, 0x8d, 0xd4, 0xe9, 0xb9, 0x91, 0x7a, 0x79, 0x6e, 0xa4, 0x9e, 0xbc, 0xab, 0x7e,
0xc1, 0x06, 0xf6, 0xbe, 0xed, 0xd9, 0x9d, 0xbe, 0x7f, 0xe8, 0x76, 0xd4, 0x2f, 0xe4, 0x5d, 0x9d,
0xff, 0x7d, 0xfc, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x02, 0x6d, 0x76, 0x3a, 0x38, 0x0b, 0x00,
0x00,
}
func (x Direction) String() string {
@ -1231,6 +1241,14 @@ func (this *QueryRequest) Equal(that interface{}) bool {
if this.Direction != that1.Direction {
return false
}
if len(this.Shards) != len(that1.Shards) {
return false
}
for i := range this.Shards {
if this.Shards[i] != that1.Shards[i] {
return false
}
}
return true
}
func (this *QueryResponse) Equal(that interface{}) bool {
@ -1766,13 +1784,14 @@ func (this *QueryRequest) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 9)
s := make([]string, 0, 10)
s = append(s, "&logproto.QueryRequest{")
s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n")
s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n")
s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n")
s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n")
s = append(s, "Direction: "+fmt.Sprintf("%#v", this.Direction)+",\n")
s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n")
s = append(s, "}")
return strings.Join(s, "")
}
@ -2516,6 +2535,21 @@ func (m *QueryRequest) MarshalTo(dAtA []byte) (int, error) {
i++
i = encodeVarintLogproto(dAtA, i, uint64(m.Direction))
}
if len(m.Shards) > 0 {
for _, s := range m.Shards {
dAtA[i] = 0x3a
i++
l = len(s)
for l >= 1<<7 {
dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
l >>= 7
i++
}
dAtA[i] = uint8(l)
i++
i += copy(dAtA[i:], s)
}
}
return i, nil
}
@ -3160,6 +3194,12 @@ func (m *QueryRequest) Size() (n int) {
if m.Direction != 0 {
n += 1 + sovLogproto(uint64(m.Direction))
}
if len(m.Shards) > 0 {
for _, s := range m.Shards {
l = len(s)
n += 1 + l + sovLogproto(uint64(l))
}
}
return n
}
@ -3490,6 +3530,7 @@ func (this *QueryRequest) String() string {
`Start:` + strings.Replace(strings.Replace(this.Start.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`,
`End:` + strings.Replace(strings.Replace(this.End.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`,
`Direction:` + fmt.Sprintf("%v", this.Direction) + `,`,
`Shards:` + fmt.Sprintf("%v", this.Shards) + `,`,
`}`,
}, "")
return s
@ -4002,6 +4043,38 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error {
break
}
}
case 7:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowLogproto
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthLogproto
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthLogproto
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipLogproto(dAtA[iNdEx:])

@ -37,6 +37,8 @@ message QueryRequest {
google.protobuf.Timestamp end = 4 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false];
Direction direction = 5;
reserved 6;
repeated string shards = 7 [(gogoproto.jsontag) = "shards,omitempty"];
}
enum Direction {

@ -1,26 +0,0 @@
package logql
import (
"fmt"
"github.com/pkg/errors"
)
// ASTMapper is the exported interface for mapping between multiple AST representations
type ASTMapper interface {
Map(Expr) (Expr, error)
}
// CloneExpr is a helper function to clone a node.
func CloneExpr(expr Expr) (Expr, error) {
return ParseExpr(expr.String())
}
func badASTMapping(expected string, got Expr) error {
return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got)
}
// MapperUnsupportedType is a helper for signaling that an evaluator does not support an Expr type
func MapperUnsupportedType(expr Expr, m ASTMapper) error {
return errors.Errorf("unexpected expr type (%T) for ASTMapper type (%T) ", expr, m)
}

@ -2,6 +2,7 @@ package logql
import (
"context"
"errors"
"sort"
"time"
@ -35,6 +36,12 @@ const ValueTypeStreams = "streams"
// Streams is promql.Value
type Streams []logproto.Stream
func (streams Streams) Len() int { return len(streams) }
func (streams Streams) Swap(i, j int) { streams[i], streams[j] = streams[j], streams[i] }
func (streams Streams) Less(i, j int) bool {
return streams[i].Labels <= streams[j].Labels
}
// Type implements `promql.Value`
func (Streams) Type() promql.ValueType { return ValueTypeStreams }
@ -67,31 +74,29 @@ func (opts *EngineOpts) applyDefault() {
}
}
// Engine interface used to construct queries
type Engine interface {
NewRangeQuery(qs string, start, end time.Time, step, interval time.Duration, direction logproto.Direction, limit uint32) Query
NewInstantQuery(qs string, ts time.Time, direction logproto.Direction, limit uint32) Query
}
// engine is the LogQL engine.
type engine struct {
// Engine is the LogQL engine.
type Engine struct {
timeout time.Duration
evaluator Evaluator
}
// NewEngine creates a new LogQL engine.
func NewEngine(opts EngineOpts, q Querier) Engine {
if q == nil {
panic("nil Querier")
}
// NewEngine creates a new LogQL Engine.
func NewEngine(opts EngineOpts, q Querier) *Engine {
opts.applyDefault()
return &Engine{
timeout: opts.Timeout,
evaluator: NewDefaultEvaluator(q, opts.MaxLookBackPeriod),
}
}
return &engine{
timeout: opts.Timeout,
evaluator: &defaultEvaluator{
querier: q,
maxLookBackPeriod: opts.MaxLookBackPeriod,
// Query creates a new LogQL query. Instant/Range type is derived from the parameters.
func (ng *Engine) Query(params Params) Query {
return &query{
timeout: ng.timeout,
params: params,
evaluator: ng.evaluator,
parse: func(_ context.Context, query string) (Expr, error) {
return ParseExpr(query)
},
}
}
@ -103,17 +108,18 @@ type Query interface {
}
type query struct {
LiteralParams
ng *engine
timeout time.Duration
params Params
parse func(context.Context, string) (Expr, error)
evaluator Evaluator
}
// Exec Implements `Query`
// Exec Implements `Query`. It handles instrumentation & defers to Eval.
func (q *query) Exec(ctx context.Context) (Result, error) {
log, ctx := spanlogger.New(ctx, "Engine.Exec")
log, ctx := spanlogger.New(ctx, "query.Exec")
defer log.Finish()
rangeType := GetRangeType(q)
rangeType := GetRangeType(q.params)
timer := prometheus.NewTimer(queryTime.WithLabelValues(string(rangeType)))
defer timer.ObserveDuration()
@ -122,7 +128,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) {
start := time.Now()
ctx = stats.NewContext(ctx)
data, err := q.ng.exec(ctx, q)
data, err := q.Eval(ctx)
statResult = stats.Snapshot(ctx, time.Since(start))
statResult.Log(level.Debug(log))
@ -134,7 +140,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) {
status = "400"
}
}
RecordMetrics(ctx, q, status, statResult)
RecordMetrics(ctx, q.params, status, statResult)
return Result{
Data: data,
@ -142,80 +148,41 @@ func (q *query) Exec(ctx context.Context) (Result, error) {
}, err
}
// NewRangeQuery creates a new LogQL range query.
func (ng *engine) NewRangeQuery(
qs string,
start, end time.Time, step time.Duration, interval time.Duration,
direction logproto.Direction, limit uint32) Query {
return &query{
LiteralParams: LiteralParams{
qs: qs,
start: start,
end: end,
step: step,
interval: interval,
direction: direction,
limit: limit,
},
ng: ng,
}
}
// NewInstantQuery creates a new LogQL instant query.
func (ng *engine) NewInstantQuery(
qs string,
ts time.Time,
direction logproto.Direction, limit uint32) Query {
return &query{
LiteralParams: LiteralParams{
qs: qs,
start: ts,
end: ts,
step: 0,
interval: 0,
direction: direction,
limit: limit,
},
ng: ng,
}
}
func (ng *engine) exec(ctx context.Context, q *query) (promql.Value, error) {
ctx, cancel := context.WithTimeout(ctx, ng.timeout)
func (q *query) Eval(ctx context.Context) (promql.Value, error) {
ctx, cancel := context.WithTimeout(ctx, q.timeout)
defer cancel()
qs := q.Query()
expr, err := ParseExpr(qs)
expr, err := q.parse(ctx, q.params.Query())
if err != nil {
return nil, err
}
switch e := expr.(type) {
case SampleExpr:
value, err := ng.evalSample(ctx, e, q)
value, err := q.evalSample(ctx, e)
return value, err
case LogSelectorExpr:
iter, err := ng.evaluator.Iterator(ctx, e, q)
iter, err := q.evaluator.Iterator(ctx, e, q.params)
if err != nil {
return nil, err
}
defer helpers.LogErrorWithContext(ctx, "closing iterator", iter.Close)
streams, err := readStreams(iter, q.limit, q.direction, q.interval)
streams, err := readStreams(iter, q.params.Limit(), q.params.Direction(), q.params.Interval())
return streams, err
default:
return nil, errors.New("Unexpected type (%T): cannot evaluate")
}
return nil, nil
}
// evalSample evaluate a sampleExpr
func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (promql.Value, error) {
func (q *query) evalSample(ctx context.Context, expr SampleExpr) (promql.Value, error) {
if lit, ok := expr.(*literalExpr); ok {
return ng.evalLiteral(ctx, lit, q)
return q.evalLiteral(ctx, lit)
}
stepEvaluator, err := ng.evaluator.StepEvaluator(ctx, ng.evaluator, expr, q)
stepEvaluator, err := q.evaluator.StepEvaluator(ctx, q.evaluator, expr, q.params)
if err != nil {
return nil, err
}
@ -224,7 +191,7 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr
seriesIndex := map[uint64]*promql.Series{}
next, ts, vec := stepEvaluator.Next()
if GetRangeType(q) == InstantType {
if GetRangeType(q.params) == InstantType {
sort.Slice(vec, func(i, j int) bool { return labels.Compare(vec[i].Metric, vec[j].Metric) < 0 })
return vec, nil
}
@ -262,21 +229,21 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr
return result, nil
}
func (ng *engine) evalLiteral(_ context.Context, expr *literalExpr, q *query) (promql.Value, error) {
func (q *query) evalLiteral(_ context.Context, expr *literalExpr) (promql.Value, error) {
s := promql.Scalar{
T: q.Start().UnixNano() / int64(time.Millisecond),
T: q.params.Start().UnixNano() / int64(time.Millisecond),
V: expr.value,
}
if GetRangeType(q) == InstantType {
if GetRangeType(q.params) == InstantType {
return s, nil
}
return PopulateMatrixFromScalar(s, q.LiteralParams), nil
return PopulateMatrixFromScalar(s, q.params), nil
}
func PopulateMatrixFromScalar(data promql.Scalar, params LiteralParams) promql.Matrix {
func PopulateMatrixFromScalar(data promql.Scalar, params Params) promql.Matrix {
var (
start = params.Start()
end = params.End()
@ -286,7 +253,7 @@ func PopulateMatrixFromScalar(data promql.Scalar, params LiteralParams) promql.M
[]promql.Point,
0,
// allocate enough space for all needed entries
int(params.End().Sub(params.Start())/params.Step())+1,
int(end.Sub(start)/step)+1,
),
}
)
@ -329,10 +296,11 @@ func readStreams(i iter.EntryIterator, size uint32, dir logproto.Direction, inte
}
}
result := make([]logproto.Stream, 0, len(streams))
result := make(Streams, 0, len(streams))
for _, stream := range streams {
result = append(result, *stream)
}
sort.Sort(result)
return result, i.Error()
}

@ -20,7 +20,7 @@ import (
var testSize = int64(300)
func TestEngine_NewInstantQuery(t *testing.T) {
func TestEngine_InstantQuery(t *testing.T) {
t.Parallel()
for _, test := range []struct {
qs string
@ -325,7 +325,13 @@ func TestEngine_NewInstantQuery(t *testing.T) {
t.Parallel()
eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params))
q := eng.NewInstantQuery(test.qs, test.ts, test.direction, test.limit)
q := eng.Query(LiteralParams{
qs: test.qs,
start: test.ts,
end: test.ts,
direction: test.direction,
limit: test.limit,
})
res, err := q.Exec(context.Background())
if err != nil {
t.Fatal(err)
@ -335,7 +341,7 @@ func TestEngine_NewInstantQuery(t *testing.T) {
}
}
func TestEngine_NewRangeQuery(t *testing.T) {
func TestEngine_RangeQuery(t *testing.T) {
t.Parallel()
for _, test := range []struct {
qs string
@ -1159,7 +1165,15 @@ func TestEngine_NewRangeQuery(t *testing.T) {
eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params))
q := eng.NewRangeQuery(test.qs, test.start, test.end, test.step, test.interval, test.direction, test.limit)
q := eng.Query(LiteralParams{
qs: test.qs,
start: test.start,
end: test.end,
step: test.step,
interval: test.interval,
direction: test.direction,
limit: test.limit,
})
res, err := q.Exec(context.Background())
if err != nil {
t.Fatal(err)
@ -1175,7 +1189,14 @@ func TestEngine_Stats(t *testing.T) {
st.DecompressedBytes++
return iter.NoopIterator, nil
}))
q := eng.NewInstantQuery(`{foo="bar"}`, time.Now(), logproto.BACKWARD, 1000)
q := eng.Query(LiteralParams{
qs: `{foo="bar"}`,
start: time.Now(),
end: time.Now(),
direction: logproto.BACKWARD,
limit: 1000,
})
r, err := q.Exec(context.Background())
require.NoError(t, err)
require.Equal(t, int64(1), r.Statistics.Store.DecompressedBytes)
@ -1231,7 +1252,14 @@ func benchmarkRangeQuery(testsize int64, b *testing.B) {
{`bottomk(2,rate(({app=~"foo|bar"} |~".+bar")[1m]))`, logproto.FORWARD},
{`bottomk(3,rate(({app=~"foo|bar"} |~".+bar")[1m])) without (app)`, logproto.FORWARD},
} {
q := eng.NewRangeQuery(test.qs, start, end, 60*time.Second, 0, test.direction, 1000)
q := eng.Query(LiteralParams{
qs: test.qs,
start: start,
end: end,
step: 60 * time.Second,
direction: test.direction,
limit: 1000,
})
res, err := q.Exec(context.Background())
if err != nil {
b.Fatal(err)

@ -28,8 +28,30 @@ type Params interface {
Start() time.Time
End() time.Time
Step() time.Duration
Interval() time.Duration
Limit() uint32
Direction() logproto.Direction
Shards() []string
}
func NewLiteralParams(
qs string,
start, end time.Time,
step, interval time.Duration,
direction logproto.Direction,
limit uint32,
shards []string,
) LiteralParams {
return LiteralParams{
qs: qs,
start: start,
end: end,
step: step,
interval: interval,
direction: direction,
limit: limit,
shards: shards,
}
}
// LiteralParams impls Params
@ -40,8 +62,11 @@ type LiteralParams struct {
interval time.Duration
direction logproto.Direction
limit uint32
shards []string
}
func (p LiteralParams) Copy() LiteralParams { return p }
// String impls Params
func (p LiteralParams) Query() string { return p.qs }
@ -54,12 +79,18 @@ func (p LiteralParams) End() time.Time { return p.end }
// Step impls Params
func (p LiteralParams) Step() time.Duration { return p.step }
// Interval impls Params
func (p LiteralParams) Interval() time.Duration { return p.interval }
// Limit impls Params
func (p LiteralParams) Limit() uint32 { return p.limit }
// Direction impls Params
func (p LiteralParams) Direction() logproto.Direction { return p.direction }
// Shards impls Params
func (p LiteralParams) Shards() []string { return p.shards }
// GetRangeType returns whether a query is an instant query or range query
func GetRangeType(q Params) QueryRangeType {
if q.Start() == q.End() && q.Step() == 0 {
@ -82,12 +113,21 @@ func EvaluatorUnsupportedType(expr Expr, ev Evaluator) error {
return errors.Errorf("unexpected expr type (%T) for Evaluator type (%T) ", expr, ev)
}
type defaultEvaluator struct {
type DefaultEvaluator struct {
maxLookBackPeriod time.Duration
querier Querier
}
func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) {
// NewDefaultEvaluator constructs a DefaultEvaluator
func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) *DefaultEvaluator {
return &DefaultEvaluator{
querier: querier,
maxLookBackPeriod: maxLookBackPeriod,
}
}
func (ev *DefaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) {
params := SelectParams{
QueryRequest: &logproto.QueryRequest{
Start: q.Start(),
@ -95,6 +135,7 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr,
Limit: q.Limit(),
Direction: q.Direction(),
Selector: expr.String(),
Shards: q.Shards(),
},
}
@ -106,7 +147,7 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr,
}
func (ev *defaultEvaluator) StepEvaluator(
func (ev *DefaultEvaluator) StepEvaluator(
ctx context.Context,
nextEv Evaluator,
expr SampleExpr,
@ -123,6 +164,7 @@ func (ev *defaultEvaluator) StepEvaluator(
Limit: 0,
Direction: logproto.FORWARD,
Selector: expr.Selector().String(),
Shards: q.Shards(),
},
})
if err != nil {

@ -0,0 +1,67 @@
package logql
import (
"time"
"github.com/prometheus/prometheus/promql"
)
// MatrixStepper exposes a promql.Matrix as a StepEvaluator.
// Ensure that the resulting StepEvaluator maintains
// the same shape that the parameters expect. For example,
// it's possible that a downstream query returns matches no
// log streams and thus returns an empty matrix.
// However, we still need to ensure that it can be merged effectively
// with another leg that may match series.
// Therefore, we determine our steps from the parameters
// and not the underlying Matrix.
type MatrixStepper struct {
start, end, ts time.Time
step time.Duration
m promql.Matrix
}
func NewMatrixStepper(start, end time.Time, step time.Duration, m promql.Matrix) *MatrixStepper {
return &MatrixStepper{
start: start,
end: end,
ts: start.Add(-step), // will be corrected on first Next() call
step: step,
m: m,
}
}
func (m *MatrixStepper) Next() (bool, int64, promql.Vector) {
m.ts = m.ts.Add(m.step)
if !m.ts.Before(m.end) {
return false, 0, nil
}
ts := m.ts.UnixNano() / int64(time.Millisecond)
vec := make(promql.Vector, 0, len(m.m))
for i, series := range m.m {
ln := len(series.Points)
if ln == 0 || series.Points[0].T != ts {
vec = append(vec, promql.Sample{
Point: promql.Point{
T: ts,
V: 0,
},
Metric: series.Metric,
})
continue
}
vec = append(vec, promql.Sample{
Point: series.Points[0],
Metric: series.Metric,
})
m.m[i].Points = m.m[i].Points[1:]
}
return true, ts, vec
}
func (m *MatrixStepper) Close() error { return nil }

@ -0,0 +1,115 @@
package logql
import (
"testing"
"time"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
)
func TestMatrixStepper(t *testing.T) {
var (
start = time.Unix(0, 0)
end = time.Unix(6, 0)
step = time.Second
)
m := promql.Matrix{
promql.Series{
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
Points: []promql.Point{
{T: start.UnixNano() / int64(step), V: 0},
{T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 1},
{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2},
{T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 3},
{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4},
{T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 5},
},
},
promql.Series{
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
Points: []promql.Point{
{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2},
{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4},
},
},
}
s := NewMatrixStepper(start, end, step, m)
expected := []promql.Vector{
{
promql.Sample{
Point: promql.Point{T: start.UnixNano() / int64(step), V: 0},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.UnixNano() / int64(step), V: 0},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
{
promql.Sample{
Point: promql.Point{T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 1},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 0},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
{
promql.Sample{
Point: promql.Point{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
{
promql.Sample{
Point: promql.Point{T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 3},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 0},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
{
promql.Sample{
Point: promql.Point{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
{
promql.Sample{
Point: promql.Point{T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 5},
Metric: labels.Labels{{Name: "foo", Value: "bar"}},
},
promql.Sample{
Point: promql.Point{T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 0},
Metric: labels.Labels{{Name: "bazz", Value: "buzz"}},
},
},
}
for i := 0; i < int(end.Sub(start)/step); i++ {
ok, ts, vec := s.Next()
require.Equal(t, ok, true)
require.Equal(t, start.Add(step*time.Duration(i)).UnixNano()/int64(time.Millisecond), ts)
require.Equal(t, expected[i], vec)
}
ok, _, _ := s.Next()
require.Equal(t, ok, false)
}

@ -28,8 +28,8 @@ var (
Namespace: "loki",
Name: "logql_querystats_bytes_processed_per_seconds",
Help: "Distribution of bytes processed per second for LogQL queries.",
// 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB
Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9},
// 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB 30GB, 40GB 50GB 60GB
Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9, 30 * 1e9, 40 * 1e9, 50 * 1e9, 60 * 1e9},
}, []string{"status_code", "type", "range", "latency_type"})
execLatency = promauto.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "loki",

@ -1,11 +1,75 @@
package logql
import (
"context"
"errors"
"fmt"
"time"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/cortexproject/cortex/pkg/util"
"github.com/cortexproject/cortex/pkg/util/spanlogger"
"github.com/go-kit/kit/log/level"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logql/stats"
)
/*
This includes a bunch of tooling for parallelization improvements based on backend shard factors.
In schemas 10+ a shard factor (default 16) is introduced in the index store,
calculated by hashing the label set of a log stream. This allows us to perform certain optimizations
that fall under the umbrella of query remapping and querying shards individually.
For instance, `{app="foo"} |= "bar"` can be executed on each shard independently, then reaggregated.
There are also a class of optimizations that can be performed by altering a query into a functionally equivalent,
but more parallelizable form. For instance, an average can be remapped into a sum/count,
which can then take advantage of our sharded execution model.
*/
// ShardedEngine is an Engine implementation that can split queries into more parallelizable forms via
// querying the underlying backend shards individually and reaggregating them.
type ShardedEngine struct {
timeout time.Duration
downstreamable Downstreamable
metrics *ShardingMetrics
}
// NewShardedEngine constructs a *ShardedEngine
func NewShardedEngine(opts EngineOpts, downstreamable Downstreamable, metrics *ShardingMetrics) *ShardedEngine {
opts.applyDefault()
return &ShardedEngine{
timeout: opts.Timeout,
downstreamable: downstreamable,
metrics: metrics,
}
}
// Query constructs a Query
func (ng *ShardedEngine) Query(p Params, shards int) Query {
return &query{
timeout: ng.timeout,
params: p,
evaluator: NewDownstreamEvaluator(ng.downstreamable.Downstreamer()),
parse: func(ctx context.Context, query string) (Expr, error) {
logger := spanlogger.FromContext(ctx)
mapper, err := NewShardMapper(shards, ng.metrics)
if err != nil {
return nil, err
}
noop, parsed, err := mapper.Parse(query)
if err != nil {
level.Warn(logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query)
return nil, err
}
level.Debug(logger).Log("no-op", noop, "mapped", parsed.String())
return parsed, nil
},
}
}
// DownstreamSampleExpr is a SampleExpr which signals downstream computation
type DownstreamSampleExpr struct {
shard *astmapper.ShardAnnotation
@ -30,28 +94,296 @@ func (d DownstreamLogSelectorExpr) String() string {
// Contract: The embedded SampleExprs within a linked list of ConcatSampleExprs must be of the
// same structure. This makes special implementations of SampleExpr.Associative() unnecessary.
type ConcatSampleExpr struct {
SampleExpr
DownstreamSampleExpr
next *ConcatSampleExpr
}
func (c ConcatSampleExpr) String() string {
if c.next == nil {
return c.SampleExpr.String()
return c.DownstreamSampleExpr.String()
}
return fmt.Sprintf("%s ++ %s", c.SampleExpr.String(), c.next.String())
return fmt.Sprintf("%s ++ %s", c.DownstreamSampleExpr.String(), c.next.String())
}
// ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr
type ConcatLogSelectorExpr struct {
LogSelectorExpr
DownstreamLogSelectorExpr
next *ConcatLogSelectorExpr
}
func (c ConcatLogSelectorExpr) String() string {
if c.next == nil {
return c.LogSelectorExpr.String()
return c.DownstreamLogSelectorExpr.String()
}
return fmt.Sprintf("%s ++ %s", c.DownstreamLogSelectorExpr.String(), c.next.String())
}
type Shards []astmapper.ShardAnnotation
func (xs Shards) Encode() (encoded []string) {
for _, shard := range xs {
encoded = append(encoded, shard.String())
}
return encoded
}
// ParseShards parses a list of string encoded shards
func ParseShards(strs []string) (Shards, error) {
if len(strs) == 0 {
return nil, nil
}
shards := make([]astmapper.ShardAnnotation, 0, len(strs))
for _, str := range strs {
shard, err := astmapper.ParseShard(str)
if err != nil {
return nil, err
}
shards = append(shards, shard)
}
return shards, nil
}
type Downstreamable interface {
Downstreamer() Downstreamer
}
type DownstreamQuery struct {
Expr Expr
Params Params
Shards Shards
}
// Downstreamer is an interface for deferring responsibility for query execution.
// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs.
type Downstreamer interface {
Downstream(context.Context, []DownstreamQuery) ([]Result, error)
}
// DownstreamEvaluator is an evaluator which handles shard aware AST nodes
type DownstreamEvaluator struct {
Downstreamer
defaultEvaluator *DefaultEvaluator
}
// Downstream runs queries and collects stats from the embedded Downstreamer
func (ev DownstreamEvaluator) Downstream(ctx context.Context, queries []DownstreamQuery) ([]Result, error) {
results, err := ev.Downstreamer.Downstream(ctx, queries)
if err != nil {
return nil, err
}
for _, res := range results {
if err := stats.JoinResults(ctx, res.Statistics); err != nil {
level.Warn(util.Logger).Log("msg", "unable to merge downstream results", "err", err)
}
}
return results, nil
}
func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator {
return &DownstreamEvaluator{
Downstreamer: downstreamer,
defaultEvaluator: NewDefaultEvaluator(
QuerierFunc(func(_ context.Context, p SelectParams) (iter.EntryIterator, error) {
// TODO(owen-d): add metric here, this should never happen.
return nil, errors.New("Unimplemented")
}),
0,
),
}
}
// Evaluator returns a StepEvaluator for a given SampleExpr
func (ev *DownstreamEvaluator) StepEvaluator(
ctx context.Context,
nextEv Evaluator,
expr SampleExpr,
params Params,
) (StepEvaluator, error) {
switch e := expr.(type) {
case DownstreamSampleExpr:
// downstream to a querier
var shards []astmapper.ShardAnnotation
if e.shard != nil {
shards = append(shards, *e.shard)
}
results, err := ev.Downstream(ctx, []DownstreamQuery{{
Expr: e.SampleExpr,
Params: params,
Shards: shards,
}})
if err != nil {
return nil, err
}
return ResultStepEvaluator(results[0], params)
case *ConcatSampleExpr:
cur := e
var queries []DownstreamQuery
for cur != nil {
qry := DownstreamQuery{
Expr: cur.DownstreamSampleExpr.SampleExpr,
Params: params,
}
if shard := cur.DownstreamSampleExpr.shard; shard != nil {
qry.Shards = Shards{*shard}
}
queries = append(queries, qry)
cur = cur.next
}
results, err := ev.Downstream(ctx, queries)
if err != nil {
return nil, err
}
xs := make([]StepEvaluator, 0, len(queries))
for i, res := range results {
stepper, err := ResultStepEvaluator(res, params)
if err != nil {
level.Warn(util.Logger).Log(
"msg", "could not extract StepEvaluator",
"err", err,
"expr", queries[i].Expr.String(),
)
return nil, err
}
xs = append(xs, stepper)
}
return ConcatEvaluator(xs)
default:
return ev.defaultEvaluator.StepEvaluator(ctx, nextEv, e, params)
}
}
// Iterator returns the iter.EntryIterator for a given LogSelectorExpr
func (ev *DownstreamEvaluator) Iterator(
ctx context.Context,
expr LogSelectorExpr,
params Params,
) (iter.EntryIterator, error) {
switch e := expr.(type) {
case DownstreamLogSelectorExpr:
// downstream to a querier
var shards Shards
if e.shard != nil {
shards = append(shards, *e.shard)
}
results, err := ev.Downstream(ctx, []DownstreamQuery{{
Expr: e.LogSelectorExpr,
Params: params,
Shards: shards,
}})
if err != nil {
return nil, err
}
return ResultIterator(results[0], params)
case *ConcatLogSelectorExpr:
cur := e
var queries []DownstreamQuery
for cur != nil {
qry := DownstreamQuery{
Expr: cur.DownstreamLogSelectorExpr.LogSelectorExpr,
Params: params,
}
if shard := cur.DownstreamLogSelectorExpr.shard; shard != nil {
qry.Shards = Shards{*shard}
}
queries = append(queries, qry)
cur = cur.next
}
results, err := ev.Downstream(ctx, queries)
if err != nil {
return nil, err
}
xs := make([]iter.EntryIterator, 0, len(queries))
for i, res := range results {
iter, err := ResultIterator(res, params)
if err != nil {
level.Warn(util.Logger).Log(
"msg", "could not extract Iterator",
"err", err,
"expr", queries[i].Expr.String(),
)
}
xs = append(xs, iter)
}
return iter.NewHeapIterator(ctx, xs, params.Direction()), nil
default:
return nil, EvaluatorUnsupportedType(expr, ev)
}
}
// ConcatEvaluator joins multiple StepEvaluators.
// Contract: They must be of identical start, end, and step values.
func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) {
return newStepEvaluator(
func() (done bool, ts int64, vec promql.Vector) {
var cur promql.Vector
for _, eval := range evaluators {
done, ts, cur = eval.Next()
vec = append(vec, cur...)
}
return done, ts, vec
},
func() (lastErr error) {
for _, eval := range evaluators {
if err := eval.Close(); err != nil {
lastErr = err
}
}
return lastErr
},
)
}
// ResultStepEvaluator coerces a downstream vector or matrix into a StepEvaluator
func ResultStepEvaluator(res Result, params Params) (StepEvaluator, error) {
var (
start = params.Start()
end = params.End()
step = params.Step()
)
switch data := res.Data.(type) {
case promql.Vector:
var exhausted bool
return newStepEvaluator(func() (bool, int64, promql.Vector) {
if !exhausted {
exhausted = true
return true, start.UnixNano() / int64(time.Millisecond), data
}
return false, 0, nil
}, nil)
case promql.Matrix:
return NewMatrixStepper(start, end, step, data), nil
default:
return nil, fmt.Errorf("unexpected type (%s) uncoercible to StepEvaluator", data.Type())
}
}
// ResultIterator coerces a downstream streams result into an iter.EntryIterator
func ResultIterator(res Result, params Params) (iter.EntryIterator, error) {
streams, ok := res.Data.(Streams)
if !ok {
return nil, fmt.Errorf("unexpected type (%s) for ResultIterator; expected %s", res.Data.Type(), ValueTypeStreams)
}
return iter.NewStreamsIterator(context.Background(), streams, params.Direction()), nil
return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String())
}

@ -0,0 +1,109 @@
package logql
import (
"context"
"math"
"testing"
"time"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logproto"
)
var nilMetrics = NewShardingMetrics(nil)
func TestMappingEquivalence(t *testing.T) {
var (
shards = 3
nStreams = 60
rounds = 20
streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"})
start = time.Unix(0, 0)
end = time.Unix(0, int64(time.Second*time.Duration(rounds)))
step = time.Second
interval = time.Duration(0)
limit = 100
)
for _, tc := range []struct {
query string
approximate bool
}{
{`1`, false},
{`1 + 1`, false},
{`{a="1"}`, false},
{`{a="1"} |= "number: 10"`, false},
{`rate({a=~".*"}[1s])`, false},
{`sum by (a) (rate({a=~".*"}[1s]))`, false},
{`max without (a) (rate({a=~".*"}[1s]))`, false},
{`count(rate({a=~".*"}[1s]))`, false},
{`avg(rate({a=~".*"}[1s]))`, true},
{`1 + sum by (cluster) (rate({a=~".*"}[1s]))`, false},
{`sum(max(rate({a=~".*"}[1s])))`, false},
{`max(count(rate({a=~".*"}[1s])))`, false},
{`max(sum by (cluster) (rate({a=~".*"}[1s]))) / count(rate({a=~".*"}[1s]))`, false},
// topk prefers already-seen values in tiebreakers. Since the test data generates
// the same log lines for each series & the resulting promql.Vectors aren't deterministically
// sorted by labels, we don't expect this to pass.
// We could sort them as stated, but it doesn't seem worth the performance hit.
// {`topk(3, rate({a=~".*"}[1s]))`, false},
} {
q := NewMockQuerier(
shards,
streams,
)
opts := EngineOpts{}
regular := NewEngine(opts, q)
sharded := NewShardedEngine(opts, MockDownstreamer{regular}, nilMetrics)
t.Run(tc.query, func(t *testing.T) {
params := NewLiteralParams(
tc.query,
start,
end,
step,
interval,
logproto.FORWARD,
uint32(limit),
nil,
)
qry := regular.Query(params)
shardedQry := sharded.Query(params, shards)
res, err := qry.Exec(context.Background())
require.Nil(t, err)
shardedRes, err := shardedQry.Exec(context.Background())
require.Nil(t, err)
if tc.approximate {
approximatelyEquals(t, res.Data.(promql.Matrix), shardedRes.Data.(promql.Matrix))
} else {
require.Equal(t, res.Data, shardedRes.Data)
}
})
}
}
// approximatelyEquals ensures two responses are approximately equal, up to 6 decimals precision per sample
func approximatelyEquals(t *testing.T, as, bs promql.Matrix) {
require.Equal(t, len(as), len(bs))
for i := 0; i < len(as); i++ {
a := as[i]
b := bs[i]
require.Equal(t, a.Metric, b.Metric)
require.Equal(t, len(a.Points), len(b.Points))
for j := 0; j < len(a.Points); j++ {
aSample := &a.Points[j]
aSample.V = math.Round(aSample.V*1e6) / 1e6
bSample := &b.Points[j]
bSample.V = math.Round(bSample.V*1e6) / 1e6
}
require.Equal(t, a, b)
}
}

@ -6,35 +6,141 @@ import (
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/cortexproject/cortex/pkg/util"
"github.com/go-kit/kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
)
func NewShardMapper(shards int) (ShardMapper, error) {
// keys used in metrics
const (
StreamsKey = "streams"
MetricsKey = "metrics"
SuccessKey = "success"
FailureKey = "failure"
NoopKey = "noop"
)
// ShardingMetrics is the metrics wrapper used in shard mapping
type ShardingMetrics struct {
shards *prometheus.CounterVec // sharded queries total, partitioned by (streams/metric)
parsed *prometheus.CounterVec // parsed ASTs total, partitioned by (success/failure/noop)
shardFactor prometheus.Histogram // per request shard factor
}
func NewShardingMetrics(registerer prometheus.Registerer) *ShardingMetrics {
return &ShardingMetrics{
shards: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
Namespace: "loki",
Name: "query_frontend_shards_total",
}, []string{"type"}),
parsed: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
Namespace: "loki",
Name: "query_frontend_sharding_parsed_queries_total",
}, []string{"type"}),
shardFactor: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{
Namespace: "loki",
Name: "query_frontend_shard_factor",
Help: "Number of shards per request",
Buckets: prometheus.LinearBuckets(0, 16, 4), // 16 is the default shard factor for later schemas
}),
}
}
// shardRecorder constructs a recorder using the underlying metrics.
func (m *ShardingMetrics) shardRecorder() *shardRecorder {
return &shardRecorder{
ShardingMetrics: m,
}
}
// shardRecorder wraps a vector & histogram, providing an easy way to increment sharding counts.
// and unify them into histogram entries.
// NOT SAFE FOR CONCURRENT USE! We avoid introducing mutex locking here
// because AST mapping is single threaded.
type shardRecorder struct {
done bool
total int
*ShardingMetrics
}
// Add increments both the shard count and tracks it for the eventual histogram entry.
func (r *shardRecorder) Add(x int, key string) {
r.total += x
r.shards.WithLabelValues(key).Add(float64(x))
}
// Finish idemptotently records a histogram entry with the total shard factor.
func (r *shardRecorder) Finish() {
if !r.done {
r.done = true
r.shardFactor.Observe(float64(r.total))
}
}
func badASTMapping(expected string, got Expr) error {
return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got)
}
func NewShardMapper(shards int, metrics *ShardingMetrics) (ShardMapper, error) {
if shards < 2 {
return ShardMapper{}, fmt.Errorf("Cannot create ShardMapper with <2 shards. Received %d", shards)
}
return ShardMapper{shards}, nil
return ShardMapper{
shards: shards,
metrics: metrics,
}, nil
}
type ShardMapper struct {
shards int
shards int
metrics *ShardingMetrics
}
func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) {
parsed, err := ParseExpr(query)
if err != nil {
return false, nil, err
}
recorder := m.metrics.shardRecorder()
mapped, err := m.Map(parsed, recorder)
if err != nil {
m.metrics.parsed.WithLabelValues(FailureKey).Inc()
return false, nil, err
}
mappedStr := mapped.String()
originalStr := parsed.String()
noop = originalStr == mappedStr
if noop {
m.metrics.parsed.WithLabelValues(NoopKey).Inc()
} else {
m.metrics.parsed.WithLabelValues(SuccessKey).Inc()
}
recorder.Finish() // only record metrics for successful mappings
return noop, mapped, err
}
func (m ShardMapper) Map(expr Expr) (Expr, error) {
func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) {
switch e := expr.(type) {
case *literalExpr:
return e, nil
case *matchersExpr, *filterExpr:
return m.mapLogSelectorExpr(e.(LogSelectorExpr)), nil
return m.mapLogSelectorExpr(e.(LogSelectorExpr), r), nil
case *vectorAggregationExpr:
return m.mapVectorAggregationExpr(e)
return m.mapVectorAggregationExpr(e, r)
case *rangeAggregationExpr:
return m.mapRangeAggregationExpr(e), nil
return m.mapRangeAggregationExpr(e, r), nil
case *binOpExpr:
lhsMapped, err := m.Map(e.SampleExpr)
lhsMapped, err := m.Map(e.SampleExpr, r)
if err != nil {
return nil, err
}
rhsMapped, err := m.Map(e.RHS)
rhsMapped, err := m.Map(e.RHS, r)
if err != nil {
return nil, err
}
@ -50,15 +156,15 @@ func (m ShardMapper) Map(expr Expr) (Expr, error) {
e.RHS = rhsSampleExpr
return e, nil
default:
return nil, MapperUnsupportedType(expr, m)
return nil, errors.Errorf("unexpected expr type (%T) for ASTMapper type (%T) ", expr, m)
}
}
func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr {
func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *shardRecorder) LogSelectorExpr {
var head *ConcatLogSelectorExpr
for i := m.shards - 1; i >= 0; i-- {
head = &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: i,
Of: m.shards,
@ -68,15 +174,16 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr {
next: head,
}
}
r.Add(m.shards, StreamsKey)
return head
}
func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr {
func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *shardRecorder) SampleExpr {
var head *ConcatSampleExpr
for i := m.shards - 1; i >= 0; i-- {
head = &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: i,
Of: m.shards,
@ -86,18 +193,19 @@ func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr {
next: head,
}
}
r.Add(m.shards, MetricsKey)
return head
}
// technically, std{dev,var} are also parallelizable if there is no cross-shard merging
// in descendent nodes in the AST. This optimization is currently avoided for simplicity.
func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) {
func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr, r *shardRecorder) (SampleExpr, error) {
// if this AST contains unshardable operations, don't shard this at this level,
// but attempt to shard a child node.
if shardable := isShardable(expr.Operations()); !shardable {
subMapped, err := m.Map(expr.left)
subMapped, err := m.Map(expr.left, r)
if err != nil {
return nil, err
}
@ -119,7 +227,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp
case OpTypeSum:
// sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...)
return &vectorAggregationExpr{
left: m.mapSampleExpr(expr),
left: m.mapSampleExpr(expr, r),
grouping: expr.grouping,
params: expr.params,
operation: expr.operation,
@ -131,7 +239,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp
left: expr.left,
grouping: expr.grouping,
operation: OpTypeSum,
})
}, r)
if err != nil {
return nil, err
}
@ -139,7 +247,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp
left: expr.left,
grouping: expr.grouping,
operation: OpTypeCount,
})
}, r)
if err != nil {
return nil, err
}
@ -152,7 +260,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp
case OpTypeCount:
// count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...)
sharded := m.mapSampleExpr(expr)
sharded := m.mapSampleExpr(expr, r)
return &vectorAggregationExpr{
left: sharded,
grouping: expr.grouping,
@ -169,12 +277,12 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp
}
}
func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleExpr {
func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr, r *shardRecorder) SampleExpr {
switch expr.operation {
case OpTypeCountOverTime, OpTypeRate:
// count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)...
// rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)...
return m.mapSampleExpr(expr)
return m.mapSampleExpr(expr, r)
default:
return expr
}

@ -16,7 +16,7 @@ func TestStringer(t *testing.T) {
}{
{
in: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -28,7 +28,7 @@ func TestStringer(t *testing.T) {
},
},
next: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -52,7 +52,7 @@ func TestStringer(t *testing.T) {
}
func TestMapSampleExpr(t *testing.T) {
m, err := NewShardMapper(2)
m, err := NewShardMapper(2, nilMetrics)
require.Nil(t, err)
for _, tc := range []struct {
@ -72,7 +72,7 @@ func TestMapSampleExpr(t *testing.T) {
},
},
out: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -90,7 +90,7 @@ func TestMapSampleExpr(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -113,19 +113,23 @@ func TestMapSampleExpr(t *testing.T) {
},
} {
t.Run(tc.in.String(), func(t *testing.T) {
require.Equal(t, tc.out, m.mapSampleExpr(tc.in))
require.Equal(t, tc.out, m.mapSampleExpr(tc.in, nilMetrics.shardRecorder()))
})
}
}
func TestMappingStrings(t *testing.T) {
m, err := NewShardMapper(2)
m, err := NewShardMapper(2, nilMetrics)
require.Nil(t, err)
for _, tc := range []struct {
in string
out string
}{
{
in: `{foo="bar"}`,
out: `downstream<{foo="bar"}, shard=0_of_2> ++ downstream<{foo="bar"}, shard=1_of_2>`,
},
{
in: `sum(rate({foo="bar"}[1m]))`,
out: `sum(downstream<sum(rate(({foo="bar"})[1m])), shard=0_of_2> ++ downstream<sum(rate(({foo="bar"})[1m])), shard=1_of_2>)`,
@ -155,7 +159,7 @@ func TestMappingStrings(t *testing.T) {
ast, err := ParseExpr(tc.in)
require.Nil(t, err)
mapped, err := m.Map(ast)
mapped, err := m.Map(ast, nilMetrics.shardRecorder())
require.Nil(t, err)
require.Equal(t, tc.out, mapped.String())
@ -165,7 +169,7 @@ func TestMappingStrings(t *testing.T) {
}
func TestMapping(t *testing.T) {
m, err := NewShardMapper(2)
m, err := NewShardMapper(2, nilMetrics)
require.Nil(t, err)
for _, tc := range []struct {
@ -176,7 +180,7 @@ func TestMapping(t *testing.T) {
{
in: `{foo="bar"}`,
expr: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -188,7 +192,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -206,7 +210,7 @@ func TestMapping(t *testing.T) {
{
in: `{foo="bar"} |= "error"`,
expr: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -222,7 +226,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatLogSelectorExpr{
LogSelectorExpr: DownstreamLogSelectorExpr{
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -244,7 +248,7 @@ func TestMapping(t *testing.T) {
{
in: `rate({foo="bar"}[5m])`,
expr: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -262,7 +266,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -286,7 +290,7 @@ func TestMapping(t *testing.T) {
{
in: `count_over_time({foo="bar"}[5m])`,
expr: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -304,7 +308,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -331,7 +335,7 @@ func TestMapping(t *testing.T) {
grouping: &grouping{},
operation: OpTypeSum,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -353,7 +357,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -386,7 +390,7 @@ func TestMapping(t *testing.T) {
params: 3,
operation: OpTypeTopK,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -404,7 +408,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -435,7 +439,7 @@ func TestMapping(t *testing.T) {
},
operation: OpTypeMax,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -453,7 +457,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -481,7 +485,7 @@ func TestMapping(t *testing.T) {
operation: OpTypeSum,
grouping: &grouping{},
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -503,7 +507,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -537,7 +541,7 @@ func TestMapping(t *testing.T) {
grouping: &grouping{},
operation: OpTypeSum,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -559,7 +563,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -588,7 +592,7 @@ func TestMapping(t *testing.T) {
operation: OpTypeSum,
grouping: &grouping{},
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -610,7 +614,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -648,7 +652,7 @@ func TestMapping(t *testing.T) {
},
operation: OpTypeSum,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -672,7 +676,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -711,7 +715,7 @@ func TestMapping(t *testing.T) {
grouping: &grouping{},
operation: OpTypeMax,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -729,7 +733,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -762,7 +766,7 @@ func TestMapping(t *testing.T) {
operation: OpTypeSum,
grouping: &grouping{},
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -784,7 +788,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -824,7 +828,7 @@ func TestMapping(t *testing.T) {
},
operation: OpTypeSum,
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -848,7 +852,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -880,7 +884,7 @@ func TestMapping(t *testing.T) {
operation: OpTypeSum,
grouping: &grouping{},
left: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
@ -902,7 +906,7 @@ func TestMapping(t *testing.T) {
},
},
next: &ConcatSampleExpr{
SampleExpr: DownstreamSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
@ -934,7 +938,7 @@ func TestMapping(t *testing.T) {
ast, err := ParseExpr(tc.in)
require.Equal(t, tc.err, err)
mapped, err := m.Map(ast)
mapped, err := m.Map(ast, nilMetrics.shardRecorder())
require.Equal(t, tc.err, err)
require.Equal(t, tc.expr.String(), mapped.String())

@ -13,7 +13,7 @@ Then you can update statistics by mutating data by using:
Finally to get a snapshot of the current query statistic use
stats.Snapshot(ctx,time.Since(start))
stats.Snapshot(ctx, time.Since(start))
Ingester statistics are sent across the GRPC stream using Trailers
see https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md
@ -22,6 +22,9 @@ package stats
import (
"context"
"errors"
fmt "fmt"
"sync"
"time"
"github.com/dustin/go-humanize"
@ -35,6 +38,8 @@ const (
chunksKey ctxKeyType = "chunks"
ingesterKey ctxKeyType = "ingester"
storeKey ctxKeyType = "store"
resultKey ctxKeyType = "result" // key for pre-computed results to be merged in `Snapshot`
lockKey ctxKeyType = "lock" // key for locking a context when stats is used concurrently
)
// Log logs a query statistics result.
@ -82,6 +87,8 @@ func NewContext(ctx context.Context) context.Context {
ctx = context.WithValue(ctx, storeKey, &StoreData{})
ctx = context.WithValue(ctx, chunksKey, &ChunkData{})
ctx = context.WithValue(ctx, ingesterKey, &IngesterData{})
ctx = context.WithValue(ctx, resultKey, &Result{})
ctx = context.WithValue(ctx, lockKey, &sync.Mutex{})
return ctx
}
@ -157,8 +164,17 @@ func Snapshot(ctx context.Context, execTime time.Duration) Result {
res.Store.CompressedBytes = c.CompressedBytes
res.Store.TotalDuplicates = c.TotalDuplicates
}
res.ComputeSummary(execTime)
return res
existing, err := GetResult(ctx)
if err != nil {
res.ComputeSummary(execTime)
return res
}
existing.Merge(res)
existing.ComputeSummary(execTime)
return *existing
}
// ComputeSummary calculates the summary based on store and ingester data.
@ -204,3 +220,37 @@ func (r *Result) Merge(m Result) {
r.ComputeSummary(time.Duration(int64((r.Summary.ExecTime + m.Summary.ExecTime) * float64(time.Second))))
}
// JoinResults merges a Result with the embedded Result in a context in a concurrency-safe manner.
func JoinResults(ctx context.Context, res Result) error {
mtx, err := GetMutex(ctx)
if err != nil {
return err
}
mtx.Lock()
defer mtx.Unlock()
v, err := GetResult(ctx)
if err != nil {
return err
}
v.Merge(res)
return nil
}
func GetResult(ctx context.Context) (*Result, error) {
v, ok := ctx.Value(resultKey).(*Result)
if !ok {
return nil, errors.New("unpopulated Results key")
}
return v, nil
}
// GetChunkData returns the chunks statistics data from the current context.
func GetMutex(ctx context.Context) (*sync.Mutex, error) {
res, ok := ctx.Value(lockKey).(*sync.Mutex)
if !ok {
return nil, fmt.Errorf("no mutex available under %s", string(lockKey))
}
return res, nil
}

@ -64,6 +64,53 @@ func TestSnapshot(t *testing.T) {
require.Equal(t, expected, res)
}
func TestSnapshot_MergesResults(t *testing.T) {
ctx := NewContext(context.Background())
expected := Result{
Ingester: Ingester{
TotalChunksMatched: 200,
TotalBatches: 50,
TotalLinesSent: 60,
HeadChunkBytes: 10,
HeadChunkLines: 20,
DecompressedBytes: 24,
DecompressedLines: 40,
CompressedBytes: 60,
TotalDuplicates: 2,
TotalReached: 2,
},
Store: Store{
TotalChunksRef: 50,
TotalChunksDownloaded: 60,
ChunksDownloadTime: time.Second.Seconds(),
HeadChunkBytes: 10,
HeadChunkLines: 20,
DecompressedBytes: 40,
DecompressedLines: 20,
CompressedBytes: 30,
TotalDuplicates: 10,
},
Summary: Summary{
ExecTime: 2 * time.Second.Seconds(),
BytesProcessedPerSecond: int64(42),
LinesProcessedPerSecond: int64(50),
TotalBytesProcessed: int64(84),
TotalLinesProcessed: int64(100),
},
}
err := JoinResults(ctx, expected)
require.Nil(t, err)
res := Snapshot(ctx, 2*time.Second)
require.Equal(t, expected, res)
}
func TestGetResult_ErrsNonexistant(t *testing.T) {
out, err := GetResult(context.Background())
require.NotNil(t, err)
require.Nil(t, out)
}
func fakeIngesterQuery(ctx context.Context) {
d, _ := ctx.Value(trailersKey).(*trailerCollector)
meta := d.addTrailer()

@ -0,0 +1,172 @@
package logql
import (
"context"
"fmt"
"log"
"time"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
)
func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier {
return MockQuerier{
shards: shards,
streams: streams,
}
}
// Shard aware mock querier
type MockQuerier struct {
shards int
streams []logproto.Stream
}
func (q MockQuerier) Select(_ context.Context, req SelectParams) (iter.EntryIterator, error) {
expr, err := req.LogSelector()
if err != nil {
return nil, err
}
filter, err := expr.Filter()
if err != nil {
return nil, err
}
matchers := expr.Matchers()
var shard *astmapper.ShardAnnotation
if len(req.Shards) > 0 {
shards, err := ParseShards(req.Shards)
if err != nil {
return nil, err
}
shard = &shards[0]
}
var matched []logproto.Stream
outer:
for _, stream := range q.streams {
ls := mustParseLabels(stream.Labels)
// filter by shard if requested
if shard != nil && ls.Hash()%uint64(shard.Of) != uint64(shard.Shard) {
continue
}
for _, matcher := range matchers {
if !matcher.Matches(ls.Get(matcher.Name)) {
continue outer
}
}
matched = append(matched, stream)
}
// apply the LineFilter
filtered := make([]logproto.Stream, 0, len(matched))
if filter == nil || filter == TrueFilter {
filtered = matched
} else {
for _, s := range matched {
var entries []logproto.Entry
for _, entry := range s.Entries {
if filter.Filter([]byte(entry.Line)) {
entries = append(entries, entry)
}
}
if len(entries) > 0 {
filtered = append(filtered, logproto.Stream{
Labels: s.Labels,
Entries: entries,
})
}
}
}
return iter.NewTimeRangedIterator(
iter.NewStreamsIterator(context.Background(), filtered, req.Direction),
req.Start,
req.End,
), nil
}
type MockDownstreamer struct {
*Engine
}
func (m MockDownstreamer) Downstreamer() Downstreamer { return m }
func (d MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery) ([]Result, error) {
results := make([]Result, 0, len(queries))
for _, query := range queries {
params := NewLiteralParams(
query.Expr.String(),
query.Params.Start(),
query.Params.End(),
query.Params.Step(),
query.Params.Interval(),
query.Params.Direction(),
query.Params.Limit(),
query.Shards.Encode(),
)
res, err := d.Query(params).Exec(ctx)
if err != nil {
return nil, err
}
results = append(results, res)
}
return results, nil
}
// create nStreams of nEntries with labelNames each where each label value
// with the exception of the "index" label is modulo'd into a shard
func randomStreams(nStreams, nEntries, nShards int, labelNames []string) (streams []logproto.Stream) {
for i := 0; i < nStreams; i++ {
// labels
stream := logproto.Stream{}
ls := labels.Labels{{Name: "index", Value: fmt.Sprintf("%d", i)}}
for _, lName := range labelNames {
// I needed a way to hash something to uint64
// in order to get some form of random label distribution
shard := append(ls, labels.Label{
Name: lName,
Value: fmt.Sprintf("%d", i),
}).Hash() % uint64(nShards)
ls = append(ls, labels.Label{
Name: lName,
Value: fmt.Sprintf("%d", shard),
})
}
for j := 0; j < nEntries; j++ {
stream.Entries = append(stream.Entries, logproto.Entry{
Timestamp: time.Unix(0, int64(j*int(time.Second))),
Line: fmt.Sprintf("line number: %d", j),
})
}
stream.Labels = ls.String()
streams = append(streams, stream)
}
return streams
}
func mustParseLabels(s string) labels.Labels {
labels, err := promql.ParseMetric(s)
if err != nil {
log.Fatalf("Failed to parse %s", s)
}
return labels
}

@ -298,7 +298,14 @@ func (t *Loki) initQueryFrontend() (_ services.Service, err error) {
"config", fmt.Sprintf("%+v", t.cfg.QueryRange),
"limits", fmt.Sprintf("%+v", t.cfg.LimitsConfig),
)
tripperware, stopper, err := queryrange.NewTripperware(t.cfg.QueryRange, util.Logger, t.overrides, prometheus.DefaultRegisterer)
tripperware, stopper, err := queryrange.NewTripperware(
t.cfg.QueryRange,
util.Logger,
t.overrides,
t.cfg.SchemaConfig,
t.cfg.Querier.QueryIngestersWithin,
prometheus.DefaultRegisterer,
)
if err != nil {
return
}

@ -47,7 +47,17 @@ func (q *Querier) RangeQueryHandler(w http.ResponseWriter, r *http.Request) {
return
}
query := q.engine.NewRangeQuery(request.Query, request.Start, request.End, request.Step, request.Interval, request.Direction, request.Limit)
params := logql.NewLiteralParams(
request.Query,
request.Start,
request.End,
request.Step,
request.Interval,
request.Direction,
request.Limit,
request.Shards,
)
query := q.engine.Query(params)
result, err := query.Exec(ctx)
if err != nil {
serverutil.WriteError(err, w)
@ -77,7 +87,17 @@ func (q *Querier) InstantQueryHandler(w http.ResponseWriter, r *http.Request) {
return
}
query := q.engine.NewInstantQuery(request.Query, request.Ts, request.Direction, request.Limit)
params := logql.NewLiteralParams(
request.Query,
request.Ts,
request.Ts,
0,
0,
request.Direction,
request.Limit,
nil,
)
query := q.engine.Query(params)
result, err := query.Exec(ctx)
if err != nil {
serverutil.WriteError(err, w)
@ -124,7 +144,18 @@ func (q *Querier) LogQueryHandler(w http.ResponseWriter, r *http.Request) {
return
}
query := q.engine.NewRangeQuery(request.Query, request.Start, request.End, request.Step, request.Interval, request.Direction, request.Limit)
params := logql.NewLiteralParams(
request.Query,
request.Start,
request.End,
request.Step,
request.Interval,
request.Direction,
request.Limit,
request.Shards,
)
query := q.engine.Query(params)
result, err := query.Exec(ctx)
if err != nil {
serverutil.WriteError(err, w)

@ -63,7 +63,7 @@ type Querier struct {
ring ring.ReadRing
pool *ring_client.Pool
store storage.Store
engine logql.Engine
engine *logql.Engine
limits *validation.Overrides
}
@ -86,6 +86,7 @@ func newQuerier(cfg Config, clientCfg client.Config, clientFactory ring_client.P
store: store,
limits: limits,
}
querier.engine = logql.NewEngine(cfg.Engine, &querier)
err := services.StartAndAwaitRunning(context.Background(), querier.pool)
if err != nil {

@ -52,6 +52,12 @@ func (r *LokiRequest) WithQuery(query string) queryrange.Request {
return &new
}
func (r *LokiRequest) WithShards(shards logql.Shards) *LokiRequest {
new := *r
new.Shards = shards.Encode()
return &new
}
func (codec) DecodeRequest(_ context.Context, r *http.Request) (queryrange.Request, error) {
if err := r.ParseForm(); err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
@ -67,8 +73,9 @@ func (codec) DecodeRequest(_ context.Context, r *http.Request) (queryrange.Reque
StartTs: req.Start.UTC(),
EndTs: req.End.UTC(),
// GetStep must return milliseconds
Step: int64(req.Step) / 1e6,
Path: r.URL.Path,
Step: int64(req.Step) / 1e6,
Path: r.URL.Path,
Shards: req.Shards,
}, nil
}
@ -84,6 +91,9 @@ func (codec) EncodeRequest(ctx context.Context, r queryrange.Request) (*http.Req
"direction": []string{lokiReq.Direction.String()},
"limit": []string{fmt.Sprintf("%d", lokiReq.Limit)},
}
if len(lokiReq.Shards) > 0 {
params["shards"] = lokiReq.Shards
}
if lokiReq.Step != 0 {
params["step"] = []string{fmt.Sprintf("%f", float64(lokiReq.Step)/float64(1e3))}
}
@ -397,9 +407,11 @@ func (p paramsWrapper) End() time.Time {
func (p paramsWrapper) Step() time.Duration {
return time.Duration(p.LokiRequest.Step * 1e6)
}
func (p paramsWrapper) Limit() uint32 {
return p.LokiRequest.Limit
}
func (p paramsWrapper) Interval() time.Duration { return 0 }
func (p paramsWrapper) Direction() logproto.Direction {
return p.LokiRequest.Direction
}
func (p paramsWrapper) Limit() uint32 { return p.LokiRequest.Limit }
func (p paramsWrapper) Shards() []string {
return p.LokiRequest.Shards
}

@ -0,0 +1,185 @@
package queryrange
import (
"context"
"fmt"
"time"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
"github.com/cortexproject/cortex/pkg/util/spanlogger"
"github.com/go-kit/kit/log/level"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/logql"
)
const (
DefaultDownstreamConcurrency = 32
)
type DownstreamHandler struct {
next queryrange.Handler
}
func ParamsToLokiRequest(params logql.Params) *LokiRequest {
return &LokiRequest{
Query: params.Query(),
Limit: params.Limit(),
Step: int64(params.Step() / time.Millisecond),
StartTs: params.Start(),
EndTs: params.End(),
Direction: params.Direction(),
Path: "/loki/api/v1/query_range", // TODO(owen-d): make this derivable
}
}
func (h DownstreamHandler) Downstreamer() logql.Downstreamer {
p := DefaultDownstreamConcurrency
locks := make(chan struct{}, p)
for i := 0; i < p; i++ {
locks <- struct{}{}
}
return &instance{
parallelism: p,
locks: locks,
handler: h.next,
}
}
// instance is an intermediate struct for controlling concurrency across a single query
type instance struct {
parallelism int
locks chan struct{}
handler queryrange.Handler
}
func (i instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery) ([]logql.Result, error) {
return i.For(queries, func(qry logql.DownstreamQuery) (logql.Result, error) {
req := ParamsToLokiRequest(qry.Params).WithShards(qry.Shards).WithQuery(qry.Expr.String()).(*LokiRequest)
logger, ctx := spanlogger.New(ctx, "DownstreamHandler.instance")
defer logger.Finish()
level.Debug(logger).Log("shards", req.Shards, "query", req.Query)
res, err := i.handler.Do(ctx, req)
if err != nil {
return logql.Result{}, err
}
return ResponseToResult(res)
})
}
// For runs a function against a list of queries, collecting the results or returning an error. The indices are preserved such that input[i] maps to output[i].
func (in instance) For(
queries []logql.DownstreamQuery,
fn func(logql.DownstreamQuery) (logql.Result, error),
) ([]logql.Result, error) {
type resp struct {
i int
res logql.Result
err error
}
done := make(chan struct{})
defer close(done)
ch := make(chan resp)
// Make one goroutine to dispatch the other goroutines, bounded by instance parallelism
go func() {
for i := 0; i < len(queries); i++ {
select {
case <-done:
break
case <-in.locks:
go func(i int) {
// release lock back into pool
defer func() {
in.locks <- struct{}{}
}()
res, err := fn(queries[i])
response := resp{
i: i,
res: res,
err: err,
}
// Feed the result into the channel unless the work has completed.
select {
case <-done:
case ch <- response:
}
}(i)
}
}
}()
results := make([]logql.Result, len(queries))
for i := 0; i < len(queries); i++ {
resp := <-ch
if resp.err != nil {
return nil, resp.err
}
results[resp.i] = resp.res
}
return results, nil
}
// convert to matrix
func sampleStreamToMatrix(streams []queryrange.SampleStream) promql.Value {
xs := make(promql.Matrix, 0, len(streams))
for _, stream := range streams {
x := promql.Series{}
x.Metric = make(labels.Labels, 0, len(stream.Labels))
for _, l := range stream.Labels {
x.Metric = append(x.Metric, labels.Label(l))
}
x.Points = make([]promql.Point, 0, len(stream.Samples))
for _, sample := range stream.Samples {
x.Points = append(x.Points, promql.Point{
T: sample.TimestampMs,
V: sample.Value,
})
}
xs = append(xs, x)
}
return xs
}
func ResponseToResult(resp queryrange.Response) (logql.Result, error) {
switch r := resp.(type) {
case *LokiResponse:
if r.Error != "" {
return logql.Result{}, fmt.Errorf("%s: %s", r.ErrorType, r.Error)
}
streams := make(logql.Streams, 0, len(r.Data.Result))
for _, stream := range r.Data.Result {
streams = append(streams, stream)
}
return logql.Result{
Statistics: r.Statistics,
Data: streams,
}, nil
case *LokiPromResponse:
if r.Response.Error != "" {
return logql.Result{}, fmt.Errorf("%s: %s", r.Response.ErrorType, r.Response.Error)
}
return logql.Result{
Statistics: r.Statistics,
Data: sampleStreamToMatrix(r.Response.Data.Result),
}, nil
default:
return logql.Result{}, fmt.Errorf("cannot decode (%T)", resp)
}
}

@ -0,0 +1,344 @@
package queryrange
import (
"context"
"errors"
"sync"
"testing"
"time"
"github.com/cortexproject/cortex/pkg/ingester/client"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/stats"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
)
func testSampleStreams() []queryrange.SampleStream {
return []queryrange.SampleStream{
{
Labels: []client.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []client.Sample{
{
Value: 0,
TimestampMs: 0,
},
{
Value: 1,
TimestampMs: 1,
},
{
Value: 2,
TimestampMs: 2,
},
},
},
{
Labels: []client.LabelAdapter{{Name: "bazz", Value: "buzz"}},
Samples: []client.Sample{
{
Value: 4,
TimestampMs: 4,
},
{
Value: 5,
TimestampMs: 5,
},
{
Value: 6,
TimestampMs: 6,
},
},
},
}
}
func TestSampleStreamToMatrix(t *testing.T) {
input := testSampleStreams()
expected := promql.Matrix{
{
Metric: labels.FromMap(map[string]string{
"foo": "bar",
}),
Points: []promql.Point{
{
V: 0,
T: 0,
},
{
V: 1,
T: 1,
},
{
V: 2,
T: 2,
},
},
},
{
Metric: labels.FromMap(map[string]string{
"bazz": "buzz",
}),
Points: []promql.Point{
{
V: 4,
T: 4,
},
{
V: 5,
T: 5,
},
{
V: 6,
T: 6,
},
},
},
}
require.Equal(t, expected, sampleStreamToMatrix(input))
}
func TestResponseToResult(t *testing.T) {
for _, tc := range []struct {
desc string
input queryrange.Response
err bool
expected logql.Result
}{
{
desc: "LokiResponse",
input: &LokiResponse{
Data: LokiData{
Result: []logproto.Stream{{
Labels: `{foo="bar"}`,
}},
},
Statistics: stats.Result{
Summary: stats.Summary{ExecTime: 1},
},
},
expected: logql.Result{
Statistics: stats.Result{
Summary: stats.Summary{ExecTime: 1},
},
Data: logql.Streams{{
Labels: `{foo="bar"}`,
}},
},
},
{
desc: "LokiResponseError",
input: &LokiResponse{
Error: "foo",
ErrorType: "bar",
},
err: true,
},
{
desc: "LokiPromResponse",
input: &LokiPromResponse{
Statistics: stats.Result{
Summary: stats.Summary{ExecTime: 1},
},
Response: &queryrange.PrometheusResponse{
Data: queryrange.PrometheusData{
Result: testSampleStreams(),
},
},
},
expected: logql.Result{
Statistics: stats.Result{
Summary: stats.Summary{ExecTime: 1},
},
Data: sampleStreamToMatrix(testSampleStreams()),
},
},
{
desc: "LokiPromResponseError",
input: &LokiPromResponse{
Response: &queryrange.PrometheusResponse{
Error: "foo",
ErrorType: "bar",
},
},
err: true,
},
{
desc: "UnexpectedTypeError",
input: nil,
err: true,
},
} {
t.Run(tc.desc, func(t *testing.T) {
out, err := ResponseToResult(tc.input)
if tc.err {
require.NotNil(t, err)
}
require.Equal(t, tc.expected, out)
})
}
}
func TestDownstreamHandler(t *testing.T) {
// Pretty poor test, but this is just a passthrough struct, so ensure we create locks
// and can consume them
h := DownstreamHandler{nil}
in := h.Downstreamer().(*instance)
require.Equal(t, DefaultDownstreamConcurrency, in.parallelism)
require.NotNil(t, in.locks)
ensureParallelism(t, in, in.parallelism)
}
// Consumes the locks in an instance, making sure they're all available. Does not replace them and thus instance is unusuable after. This is a cleanup test to ensure internal state
func ensureParallelism(t *testing.T, in *instance, n int) {
for i := 0; i < n; i++ {
select {
case <-in.locks:
case <-time.After(time.Millisecond):
require.FailNow(t, "lock couldn't be acquired")
}
}
// ensure no more locks available
select {
case <-in.locks:
require.FailNow(t, "unexpected lock acquisition")
default:
}
}
func TestInstanceFor(t *testing.T) {
mkIn := func() *instance { return DownstreamHandler{nil}.Downstreamer().(*instance) }
in := mkIn()
queries := make([]logql.DownstreamQuery, in.parallelism+1)
var mtx sync.Mutex
var ct int
// ensure we can execute queries that number more than the parallelism parameter
_, err := in.For(queries, func(_ logql.DownstreamQuery) (logql.Result, error) {
mtx.Lock()
defer mtx.Unlock()
ct++
return logql.Result{}, nil
})
require.Nil(t, err)
require.Equal(t, len(queries), ct)
ensureParallelism(t, in, in.parallelism)
// ensure an early error abandons the other queues queries
in = mkIn()
ct = 0
_, err = in.For(queries, func(_ logql.DownstreamQuery) (logql.Result, error) {
mtx.Lock()
defer mtx.Unlock()
ct++
return logql.Result{}, errors.New("testerr")
})
require.NotNil(t, err)
// Ensure no more than the initial batch was parallelized.
require.LessOrEqual(t, ct, in.parallelism)
ensureParallelism(t, in, in.parallelism)
in = mkIn()
results, err := in.For(
[]logql.DownstreamQuery{
{
Shards: logql.Shards{
{Shard: 0, Of: 2},
},
},
{
Shards: logql.Shards{
{Shard: 1, Of: 2},
},
},
},
func(qry logql.DownstreamQuery) (logql.Result, error) {
return logql.Result{
Data: logql.Streams{{
Labels: qry.Shards[0].String(),
}},
}, nil
},
)
require.Nil(t, err)
require.Equal(
t,
[]logql.Result{
logql.Result{
Data: logql.Streams{{Labels: "0_of_2"}},
},
logql.Result{
Data: logql.Streams{{Labels: "1_of_2"}},
},
},
results,
)
ensureParallelism(t, in, in.parallelism)
}
func TestInstanceDownstream(t *testing.T) {
params := logql.NewLiteralParams(
"",
time.Now(),
time.Now(),
0,
0,
logproto.BACKWARD,
1000,
nil,
)
expr, err := logql.ParseExpr(`{foo="bar"}`)
require.Nil(t, err)
expectedResp := func() *LokiResponse {
return &LokiResponse{
Data: LokiData{
Result: []logproto.Stream{{
Labels: `{foo="bar"}`,
}},
},
Statistics: stats.Result{
Summary: stats.Summary{ExecTime: 1},
},
}
}
queries := []logql.DownstreamQuery{
{
Expr: expr,
Params: params,
Shards: logql.Shards{{Shard: 0, Of: 2}},
},
}
var got queryrange.Request
var want queryrange.Request
handler := queryrange.HandlerFunc(
func(_ context.Context, req queryrange.Request) (queryrange.Response, error) {
// for some reason these seemingly can't be checked in their own goroutines,
// so we assign them to scoped variables for later comparison.
got = req
want = ParamsToLokiRequest(params).WithShards(logql.Shards{{Shard: 0, Of: 2}}).WithQuery(expr.String())
return expectedResp(), nil
},
)
expected, err := ResponseToResult(expectedResp())
require.Nil(t, err)
results, err := DownstreamHandler{handler}.Downstreamer().Downstream(context.Background(), queries)
require.Equal(t, want, got)
require.Nil(t, err)
require.Equal(t, []logql.Result{expected}, results)
}

@ -40,6 +40,7 @@ type LokiRequest struct {
EndTs time.Time `protobuf:"bytes,5,opt,name=endTs,proto3,stdtime" json:"endTs"`
Direction logproto.Direction `protobuf:"varint,6,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"`
Path string `protobuf:"bytes,7,opt,name=path,proto3" json:"path,omitempty"`
Shards []string `protobuf:"bytes,8,rep,name=shards,proto3" json:"shards"`
}
func (m *LokiRequest) Reset() { *m = LokiRequest{} }
@ -123,6 +124,13 @@ func (m *LokiRequest) GetPath() string {
return ""
}
func (m *LokiRequest) GetShards() []string {
if m != nil {
return m.Shards
}
return nil
}
type LokiResponse struct {
Status string `protobuf:"bytes,1,opt,name=Status,json=status,proto3" json:"status"`
Data LokiData `protobuf:"bytes,2,opt,name=Data,json=data,proto3" json:"data,omitempty"`
@ -330,49 +338,50 @@ func init() {
}
var fileDescriptor_51b9d53b40d11902 = []byte{
// 669 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x93, 0x4d, 0x6f, 0xd3, 0x30,
0x18, 0xc7, 0xe3, 0xae, 0xaf, 0x2e, 0x1b, 0xc8, 0x9b, 0x58, 0x54, 0x24, 0xa7, 0xea, 0x85, 0x22,
0x20, 0x15, 0x1d, 0x5c, 0x90, 0x40, 0x5b, 0x34, 0xc4, 0x85, 0x03, 0xca, 0xfa, 0x05, 0xbc, 0xd6,
0xcb, 0xc2, 0x9a, 0x3a, 0xb3, 0x1d, 0xc4, 0x6e, 0x5c, 0xb9, 0xed, 0x63, 0x20, 0x3e, 0xc9, 0x8e,
0x3b, 0x4e, 0x1c, 0x02, 0xcb, 0x2e, 0xa8, 0xa7, 0x7d, 0x00, 0x0e, 0xc8, 0x76, 0xd2, 0x66, 0x88,
0xc3, 0xb8, 0xc4, 0xcf, 0xdb, 0xdf, 0x7e, 0xfc, 0x7b, 0x1c, 0xf8, 0x30, 0x3e, 0x0a, 0x06, 0xc7,
0x09, 0xe5, 0x21, 0xe5, 0x7a, 0x3d, 0xe1, 0x64, 0x16, 0xd0, 0x92, 0xe9, 0xc6, 0x9c, 0x49, 0x86,
0xe0, 0x32, 0xd2, 0x79, 0x1a, 0x84, 0xf2, 0x30, 0xd9, 0x77, 0xc7, 0x2c, 0x1a, 0x04, 0x2c, 0x60,
0x03, 0x5d, 0xb2, 0x9f, 0x1c, 0x68, 0x4f, 0x3b, 0xda, 0x32, 0xd2, 0xce, 0x03, 0x75, 0xc6, 0x94,
0x05, 0x26, 0x51, 0x18, 0x7f, 0x25, 0x8f, 0xa7, 0x03, 0x21, 0x89, 0x14, 0xe6, 0x9b, 0x27, 0xdf,
0x96, 0x0e, 0x1a, 0x33, 0x2e, 0xe9, 0xa7, 0x98, 0xb3, 0x0f, 0x74, 0x2c, 0x73, 0x6f, 0x70, 0xcb,
0xee, 0x3b, 0x4e, 0xc0, 0x58, 0x30, 0xa5, 0xcb, 0x46, 0x65, 0x18, 0x51, 0x21, 0x49, 0x14, 0x9b,
0x82, 0xde, 0x69, 0x05, 0xb6, 0xdf, 0xb1, 0xa3, 0xd0, 0xa7, 0xc7, 0x09, 0x15, 0x12, 0x6d, 0xc0,
0x9a, 0xde, 0xc4, 0x06, 0x5d, 0xd0, 0x6f, 0xf9, 0xc6, 0x51, 0xd1, 0x69, 0x18, 0x85, 0xd2, 0xae,
0x74, 0x41, 0x7f, 0xd5, 0x37, 0x0e, 0x42, 0xb0, 0x2a, 0x24, 0x8d, 0xed, 0x95, 0x2e, 0xe8, 0xaf,
0xf8, 0xda, 0x46, 0xaf, 0x61, 0x43, 0x48, 0xc2, 0xe5, 0x48, 0xd8, 0xd5, 0x2e, 0xe8, 0xb7, 0x87,
0x1d, 0xd7, 0xb4, 0xe0, 0x16, 0x2d, 0xb8, 0xa3, 0xa2, 0x05, 0xaf, 0x79, 0x96, 0x3a, 0xd6, 0xe9,
0x0f, 0x07, 0xf8, 0x85, 0x08, 0xbd, 0x84, 0x35, 0x3a, 0x9b, 0x8c, 0x84, 0x5d, 0xfb, 0x0f, 0xb5,
0x91, 0xa0, 0x67, 0xb0, 0x35, 0x09, 0x39, 0x1d, 0xcb, 0x90, 0xcd, 0xec, 0x7a, 0x17, 0xf4, 0xd7,
0x86, 0xeb, 0xee, 0x02, 0xfb, 0x6e, 0x91, 0xf2, 0x97, 0x55, 0xea, 0x0a, 0x31, 0x91, 0x87, 0x76,
0x43, 0xdf, 0x56, 0xdb, 0xbd, 0xdf, 0x15, 0x78, 0xc7, 0x20, 0x11, 0x31, 0x9b, 0x09, 0x8a, 0x7a,
0xb0, 0xbe, 0x27, 0x89, 0x4c, 0x84, 0x81, 0xe2, 0xc1, 0x79, 0xea, 0xd4, 0x85, 0x8e, 0xf8, 0xf9,
0x8a, 0xb6, 0x61, 0x75, 0x97, 0x48, 0xa2, 0x01, 0xb5, 0x87, 0x1b, 0x6e, 0x69, 0x12, 0x6a, 0x2f,
0x95, 0xf3, 0xee, 0xab, 0x86, 0xe7, 0xa9, 0xb3, 0x36, 0x21, 0x92, 0x3c, 0x61, 0x51, 0x28, 0x69,
0x14, 0xcb, 0x13, 0xbf, 0xaa, 0x7c, 0xf4, 0x02, 0xb6, 0xde, 0x70, 0xce, 0xf8, 0xe8, 0x24, 0xa6,
0x1a, 0x69, 0xcb, 0xdb, 0x9c, 0xa7, 0xce, 0x3a, 0x2d, 0x82, 0x25, 0x45, 0x6b, 0x11, 0x44, 0x8f,
0x60, 0x4d, 0xcb, 0x34, 0xee, 0x96, 0xb7, 0x3e, 0x4f, 0x9d, 0xbb, 0x3a, 0x5b, 0x2a, 0xaf, 0xe9,
0xc0, 0x4d, 0x3e, 0xb5, 0x5b, 0xf1, 0x59, 0x0c, 0xbe, 0x5e, 0x1e, 0xbc, 0x0d, 0x1b, 0x1f, 0x29,
0x17, 0x6a, 0x9b, 0x86, 0x8e, 0x17, 0x2e, 0xda, 0x81, 0x50, 0x01, 0x09, 0x85, 0x0c, 0xc7, 0xc2,
0x6e, 0x6a, 0x18, 0xab, 0xae, 0x79, 0xda, 0x3e, 0x15, 0xc9, 0x54, 0x7a, 0x28, 0xa7, 0x50, 0x2a,
0xf4, 0x4b, 0x76, 0xef, 0x1b, 0x80, 0xcd, 0x02, 0x19, 0x72, 0x21, 0x34, 0x32, 0x4d, 0xc5, 0xe0,
0x5f, 0x53, 0x62, 0xbe, 0x88, 0xfa, 0x25, 0x1b, 0xcd, 0x60, 0xdd, 0xd4, 0xdb, 0x95, 0xee, 0x4a,
0xbf, 0x3d, 0xdc, 0x5c, 0xde, 0x6f, 0x4f, 0x72, 0x4a, 0xa2, 0x9d, 0x09, 0x89, 0x25, 0xe5, 0xde,
0x2b, 0xd5, 0xc5, 0xf7, 0xd4, 0x79, 0x5c, 0xfe, 0xa5, 0x39, 0x39, 0x20, 0x33, 0x32, 0x98, 0xb2,
0xa3, 0x70, 0x50, 0xfe, 0x77, 0x73, 0xad, 0x1a, 0xbb, 0x39, 0xcb, 0xcf, 0xd7, 0xde, 0x17, 0x00,
0xef, 0xa9, 0x66, 0xdf, 0x73, 0x16, 0x2d, 0xde, 0xcb, 0x36, 0x6c, 0xf2, 0xdc, 0xd6, 0x2d, 0xb7,
0x87, 0xb8, 0xfc, 0x1e, 0x54, 0x2d, 0x95, 0x87, 0x34, 0x11, 0x85, 0xc2, 0xab, 0x9e, 0xa5, 0x0e,
0xf0, 0x17, 0x2a, 0xb4, 0x75, 0x03, 0x63, 0xe5, 0x5f, 0x18, 0x95, 0xc4, 0x2a, 0x83, 0xf3, 0x9e,
0x9f, 0x5f, 0x62, 0xeb, 0xe2, 0x12, 0x5b, 0xd7, 0x97, 0x18, 0x7c, 0xce, 0x30, 0xf8, 0x9a, 0x61,
0x70, 0x96, 0x61, 0x70, 0x9e, 0x61, 0xf0, 0x33, 0xc3, 0xe0, 0x57, 0x86, 0xad, 0xeb, 0x0c, 0x83,
0xd3, 0x2b, 0x6c, 0x9d, 0x5f, 0x61, 0xeb, 0xe2, 0x0a, 0x5b, 0xfb, 0x75, 0x7d, 0xc3, 0xad, 0x3f,
0x01, 0x00, 0x00, 0xff, 0xff, 0x3b, 0xa1, 0x1a, 0x82, 0x11, 0x05, 0x00, 0x00,
// 687 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0xcf, 0x6e, 0x13, 0x3f,
0x10, 0x5e, 0xe7, 0x7f, 0x9c, 0x5f, 0xfb, 0x43, 0x6e, 0x45, 0x57, 0x41, 0xf2, 0x46, 0xb9, 0x10,
0x04, 0x6c, 0x44, 0x0a, 0x17, 0x24, 0x50, 0xbb, 0x2a, 0xe2, 0xc2, 0x01, 0xb9, 0x79, 0x01, 0x37,
0x71, 0x37, 0x4b, 0xb3, 0xf1, 0xd6, 0x76, 0x10, 0xbd, 0x71, 0xe5, 0xd6, 0xc7, 0x40, 0x9c, 0x79,
0x88, 0x1e, 0x7b, 0xac, 0x38, 0x04, 0x9a, 0x5e, 0x50, 0x4e, 0x7d, 0x00, 0x0e, 0xc8, 0xf6, 0x6e,
0xb2, 0x45, 0x1c, 0xca, 0x65, 0x67, 0xe6, 0x9b, 0xf9, 0xec, 0x99, 0xcf, 0xb3, 0xf0, 0x7e, 0x72,
0x14, 0x76, 0x8f, 0xa7, 0x4c, 0x44, 0x4c, 0x18, 0x7b, 0x22, 0xe8, 0x24, 0x64, 0x39, 0xd7, 0x4f,
0x04, 0x57, 0x1c, 0xc1, 0x15, 0xd2, 0x7c, 0x1c, 0x46, 0x6a, 0x34, 0x3d, 0xf0, 0x07, 0x3c, 0xee,
0x86, 0x3c, 0xe4, 0x5d, 0x53, 0x72, 0x30, 0x3d, 0x34, 0x91, 0x09, 0x8c, 0x67, 0xa9, 0xcd, 0x7b,
0xfa, 0x8e, 0x31, 0x0f, 0x6d, 0x22, 0x73, 0xfe, 0x48, 0x1e, 0x8f, 0xbb, 0x52, 0x51, 0x25, 0xed,
0x37, 0x4d, 0xbe, 0xce, 0x5d, 0x34, 0xe0, 0x42, 0xb1, 0x0f, 0x89, 0xe0, 0xef, 0xd8, 0x40, 0xa5,
0x51, 0xf7, 0x96, 0xdd, 0x37, 0xbd, 0x90, 0xf3, 0x70, 0xcc, 0x56, 0x8d, 0xaa, 0x28, 0x66, 0x52,
0xd1, 0x38, 0xb1, 0x05, 0xed, 0xaf, 0x05, 0xd8, 0x78, 0xc3, 0x8f, 0x22, 0xc2, 0x8e, 0xa7, 0x4c,
0x2a, 0xb4, 0x09, 0xcb, 0xe6, 0x10, 0x17, 0xb4, 0x40, 0xa7, 0x4e, 0x6c, 0xa0, 0xd1, 0x71, 0x14,
0x47, 0xca, 0x2d, 0xb4, 0x40, 0x67, 0x8d, 0xd8, 0x00, 0x21, 0x58, 0x92, 0x8a, 0x25, 0x6e, 0xb1,
0x05, 0x3a, 0x45, 0x62, 0x7c, 0xf4, 0x12, 0x56, 0xa5, 0xa2, 0x42, 0xf5, 0xa5, 0x5b, 0x6a, 0x81,
0x4e, 0xa3, 0xd7, 0xf4, 0x6d, 0x0b, 0x7e, 0xd6, 0x82, 0xdf, 0xcf, 0x5a, 0x08, 0x6a, 0x67, 0x33,
0xcf, 0x39, 0xfd, 0xee, 0x01, 0x92, 0x91, 0xd0, 0x73, 0x58, 0x66, 0x93, 0x61, 0x5f, 0xba, 0xe5,
0x7f, 0x60, 0x5b, 0x0a, 0x7a, 0x02, 0xeb, 0xc3, 0x48, 0xb0, 0x81, 0x8a, 0xf8, 0xc4, 0xad, 0xb4,
0x40, 0x67, 0xbd, 0xb7, 0xe1, 0x2f, 0x65, 0xdf, 0xcb, 0x52, 0x64, 0x55, 0xa5, 0x47, 0x48, 0xa8,
0x1a, 0xb9, 0x55, 0x33, 0xad, 0xf1, 0x51, 0x1b, 0x56, 0xe4, 0x88, 0x8a, 0xa1, 0x74, 0x6b, 0xad,
0x62, 0xa7, 0x1e, 0xc0, 0xc5, 0xcc, 0x4b, 0x11, 0x92, 0xda, 0xf6, 0xaf, 0x02, 0xfc, 0xcf, 0xca,
0x26, 0x13, 0x3e, 0x91, 0x4c, 0x93, 0xf6, 0x15, 0x55, 0x53, 0x69, 0x85, 0x4b, 0x49, 0x06, 0x21,
0xa9, 0x45, 0x3b, 0xb0, 0xb4, 0x47, 0x15, 0x35, 0x22, 0x36, 0x7a, 0x9b, 0x7e, 0xee, 0xb5, 0xf4,
0x59, 0x3a, 0x17, 0xdc, 0xd5, 0x43, 0x2d, 0x66, 0xde, 0xfa, 0x90, 0x2a, 0xfa, 0x88, 0xc7, 0x91,
0x62, 0x71, 0xa2, 0x4e, 0x48, 0x49, 0xc7, 0xe8, 0x19, 0xac, 0xbf, 0x12, 0x82, 0x8b, 0xfe, 0x49,
0xc2, 0x8c, 0xec, 0xf5, 0x60, 0x6b, 0x31, 0xf3, 0x36, 0x58, 0x06, 0xe6, 0x18, 0xf5, 0x25, 0x88,
0x1e, 0xc0, 0xb2, 0xa1, 0x99, 0x27, 0xa9, 0x07, 0x1b, 0x8b, 0x99, 0xf7, 0xbf, 0xc9, 0xe6, 0xca,
0xcb, 0x06, 0xb8, 0xa9, 0x61, 0xf9, 0x56, 0x1a, 0x2e, 0x97, 0xa3, 0x92, 0x5f, 0x0e, 0x17, 0x56,
0xdf, 0x33, 0x21, 0xf5, 0x31, 0x55, 0x83, 0x67, 0x21, 0xda, 0x85, 0x50, 0x0b, 0x12, 0x49, 0x15,
0x0d, 0xb4, 0xc6, 0x5a, 0x8c, 0x35, 0xdf, 0xae, 0x3f, 0x61, 0x72, 0x3a, 0x56, 0x01, 0x4a, 0x55,
0xc8, 0x15, 0x92, 0x9c, 0xdf, 0xfe, 0x02, 0x60, 0x2d, 0x93, 0x0c, 0xf9, 0x10, 0x5a, 0x9a, 0x51,
0xc5, 0xca, 0xbf, 0xae, 0xc9, 0x62, 0x89, 0x92, 0x9c, 0x8f, 0x26, 0xb0, 0x62, 0xeb, 0xdd, 0x42,
0xab, 0xd8, 0x69, 0xf4, 0xb6, 0x56, 0xf3, 0xed, 0x2b, 0xc1, 0x68, 0xbc, 0x3b, 0xa4, 0x89, 0x62,
0x22, 0x78, 0xa1, 0xbb, 0xf8, 0x36, 0xf3, 0x1e, 0xe6, 0x7f, 0x7b, 0x41, 0x0f, 0xe9, 0x84, 0x76,
0xc7, 0xfc, 0x28, 0xea, 0xe6, 0xff, 0xef, 0x94, 0xab, 0x9f, 0xdd, 0xde, 0x45, 0x52, 0xdb, 0xfe,
0x04, 0xe0, 0x1d, 0xdd, 0xec, 0x5b, 0xc1, 0xe3, 0xe5, 0xbe, 0xec, 0xc0, 0x9a, 0x48, 0x7d, 0xd3,
0x72, 0xa3, 0x87, 0xf3, 0xfb, 0xa0, 0x6b, 0x99, 0x1a, 0xb1, 0xa9, 0xcc, 0x18, 0x41, 0xe9, 0x6c,
0xe6, 0x01, 0xb2, 0x64, 0xa1, 0xed, 0x1b, 0x32, 0x16, 0xfe, 0x26, 0xa3, 0xa6, 0x38, 0x79, 0xe1,
0x82, 0xa7, 0xe7, 0x97, 0xd8, 0xb9, 0xb8, 0xc4, 0xce, 0xf5, 0x25, 0x06, 0x1f, 0xe7, 0x18, 0x7c,
0x9e, 0x63, 0x70, 0x36, 0xc7, 0xe0, 0x7c, 0x8e, 0xc1, 0x8f, 0x39, 0x06, 0x3f, 0xe7, 0xd8, 0xb9,
0x9e, 0x63, 0x70, 0x7a, 0x85, 0x9d, 0xf3, 0x2b, 0xec, 0x5c, 0x5c, 0x61, 0xe7, 0xa0, 0x62, 0x26,
0xdc, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x80, 0x97, 0x84, 0xc9, 0x35, 0x05, 0x00, 0x00,
}
func (this *LokiRequest) Equal(that interface{}) bool {
@ -415,6 +424,14 @@ func (this *LokiRequest) Equal(that interface{}) bool {
if this.Path != that1.Path {
return false
}
if len(this.Shards) != len(that1.Shards) {
return false
}
for i := range this.Shards {
if this.Shards[i] != that1.Shards[i] {
return false
}
}
return true
}
func (this *LokiResponse) Equal(that interface{}) bool {
@ -525,7 +542,7 @@ func (this *LokiRequest) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 11)
s := make([]string, 0, 12)
s = append(s, "&queryrange.LokiRequest{")
s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n")
s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n")
@ -534,6 +551,7 @@ func (this *LokiRequest) GoString() string {
s = append(s, "EndTs: "+fmt.Sprintf("%#v", this.EndTs)+",\n")
s = append(s, "Direction: "+fmt.Sprintf("%#v", this.Direction)+",\n")
s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n")
s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n")
s = append(s, "}")
return strings.Join(s, "")
}
@ -644,6 +662,21 @@ func (m *LokiRequest) MarshalTo(dAtA []byte) (int, error) {
i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Path)))
i += copy(dAtA[i:], m.Path)
}
if len(m.Shards) > 0 {
for _, s := range m.Shards {
dAtA[i] = 0x42
i++
l = len(s)
for l >= 1<<7 {
dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
l >>= 7
i++
}
dAtA[i] = uint8(l)
i++
i += copy(dAtA[i:], s)
}
}
return i, nil
}
@ -822,6 +855,12 @@ func (m *LokiRequest) Size() (n int) {
if l > 0 {
n += 1 + l + sovQueryrange(uint64(l))
}
if len(m.Shards) > 0 {
for _, s := range m.Shards {
l = len(s)
n += 1 + l + sovQueryrange(uint64(l))
}
}
return n
}
@ -918,6 +957,7 @@ func (this *LokiRequest) String() string {
`EndTs:` + strings.Replace(strings.Replace(this.EndTs.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`,
`Direction:` + fmt.Sprintf("%v", this.Direction) + `,`,
`Path:` + fmt.Sprintf("%v", this.Path) + `,`,
`Shards:` + fmt.Sprintf("%v", this.Shards) + `,`,
`}`,
}, "")
return s
@ -1185,6 +1225,38 @@ func (m *LokiRequest) Unmarshal(dAtA []byte) error {
}
m.Path = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 8:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthQueryrange
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthQueryrange
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipQueryrange(dAtA[iNdEx:])

@ -19,6 +19,7 @@ message LokiRequest {
google.protobuf.Timestamp endTs = 5 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false];
logproto.Direction direction = 6;
string path = 7;
repeated string shards = 8 [(gogoproto.jsontag) = "shards"];
}
message LokiResponse {

@ -0,0 +1,239 @@
package queryrange
import (
"context"
"fmt"
"time"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
"github.com/cortexproject/cortex/pkg/util/spanlogger"
"github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/marshal"
)
var nanosecondsInMillisecond = int64(time.Millisecond / time.Nanosecond)
// NewQueryShardMiddleware creates a middleware which downstreams queries after AST mapping and query encoding.
func NewQueryShardMiddleware(
logger log.Logger,
confs queryrange.ShardingConfigs,
minShardingLookback time.Duration,
middlewareMetrics *queryrange.InstrumentMiddlewareMetrics,
shardingMetrics *logql.ShardingMetrics,
) queryrange.Middleware {
noshards := !hasShards(confs)
if noshards {
level.Warn(logger).Log(
"middleware", "QueryShard",
"msg", "no configuration with shard found",
"confs", fmt.Sprintf("%+v", confs),
)
return queryrange.PassthroughMiddleware
}
mapperware := queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler {
return newASTMapperware(confs, next, logger, shardingMetrics)
})
return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler {
return &shardSplitter{
MinShardingLookback: minShardingLookback,
shardingware: queryrange.MergeMiddlewares(
queryrange.InstrumentMiddleware("shardingware", middlewareMetrics),
mapperware,
).Wrap(next),
now: time.Now,
next: queryrange.InstrumentMiddleware("sharding-bypass", middlewareMetrics).Wrap(next),
}
})
}
func newASTMapperware(
confs queryrange.ShardingConfigs,
next queryrange.Handler,
logger log.Logger,
metrics *logql.ShardingMetrics,
) *astMapperware {
return &astMapperware{
confs: confs,
logger: log.With(logger, "middleware", "QueryShard.astMapperware"),
next: next,
ng: logql.NewShardedEngine(logql.EngineOpts{}, DownstreamHandler{next}, metrics),
}
}
type astMapperware struct {
confs queryrange.ShardingConfigs
logger log.Logger
next queryrange.Handler
ng *logql.ShardedEngine
}
func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) {
conf, err := ast.confs.GetConf(r)
// cannot shard with this timerange
if err != nil {
level.Warn(ast.logger).Log("err", err.Error(), "msg", "skipped AST mapper for request")
return ast.next.Do(ctx, r)
}
shardedLog, ctx := spanlogger.New(ctx, "shardedEngine")
defer shardedLog.Finish()
req, ok := r.(*LokiRequest)
if !ok {
return nil, fmt.Errorf("expected *LokiRequest, got (%T)", r)
}
params := paramsFromRequest(req)
query := ast.ng.Query(params, int(conf.RowShards))
res, err := query.Exec(ctx)
if err != nil {
return nil, err
}
value, err := marshal.NewResultValue(res.Data)
if err != nil {
return nil, err
}
switch res.Data.Type() {
case promql.ValueTypeMatrix:
return &LokiPromResponse{
Response: &queryrange.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrange.PrometheusData{
ResultType: loghttp.ResultTypeMatrix,
Result: toProto(value.(loghttp.Matrix)),
},
},
Statistics: res.Statistics,
}, nil
case logql.ValueTypeStreams:
return &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: req.Direction,
Limit: req.Limit,
Version: uint32(loghttp.GetVersion(req.Path)),
Statistics: res.Statistics,
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: value.(loghttp.Streams).ToProto(),
},
}, nil
default:
return nil, fmt.Errorf("unexpected downstream response type (%T)", res.Data)
}
}
// shardSplitter middleware will only shard appropriate requests that do not extend past the MinShardingLookback interval.
// This is used to send nonsharded requests to the ingesters in order to not overload them.
// TODO(owen-d): export in cortex so we don't duplicate code
type shardSplitter struct {
MinShardingLookback time.Duration // delimiter for splitting sharded vs non-sharded queries
shardingware queryrange.Handler // handler for sharded queries
next queryrange.Handler // handler for non-sharded queries
now func() time.Time // injectable time.Now
}
func (splitter *shardSplitter) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) {
cutoff := splitter.now().Add(-splitter.MinShardingLookback)
sharded, nonsharded := partitionRequest(r, cutoff)
return splitter.parallel(ctx, sharded, nonsharded)
}
func (splitter *shardSplitter) parallel(ctx context.Context, sharded, nonsharded queryrange.Request) (queryrange.Response, error) {
if sharded == nil {
return splitter.next.Do(ctx, nonsharded)
}
if nonsharded == nil {
return splitter.shardingware.Do(ctx, sharded)
}
nonshardCh := make(chan queryrange.Response, 1)
shardCh := make(chan queryrange.Response, 1)
errCh := make(chan error, 2)
go func() {
res, err := splitter.next.Do(ctx, nonsharded)
if err != nil {
errCh <- err
return
}
nonshardCh <- res
}()
go func() {
res, err := splitter.shardingware.Do(ctx, sharded)
if err != nil {
errCh <- err
return
}
shardCh <- res
}()
resps := make([]queryrange.Response, 0, 2)
for i := 0; i < 2; i++ {
select {
case r := <-nonshardCh:
resps = append(resps, r)
case r := <-shardCh:
resps = append(resps, r)
case err := <-errCh:
return nil, err
case <-ctx.Done():
return nil, ctx.Err()
}
}
return lokiCodec.MergeResponse(resps...)
}
// TODO(owen-d): export in cortex so we don't duplicate code
func hasShards(confs queryrange.ShardingConfigs) bool {
for _, conf := range confs {
if conf.RowShards > 0 {
return true
}
}
return false
}
// partitionRequet splits a request into potentially multiple requests, one including the request's time range
// [0,t). The other will include [t,inf)
// TODO(owen-d): export in cortex so we don't duplicate code
func partitionRequest(r queryrange.Request, t time.Time) (before queryrange.Request, after queryrange.Request) {
boundary := TimeToMillis(t)
if r.GetStart() >= boundary {
return nil, r
}
if r.GetEnd() < boundary {
return r, nil
}
return r.WithStartEnd(r.GetStart(), boundary), r.WithStartEnd(boundary, r.GetEnd())
}
// TimeFromMillis is a helper to turn milliseconds -> time.Time
func TimeFromMillis(ms int64) time.Time {
return time.Unix(0, ms*nanosecondsInMillisecond)
}
func TimeToMillis(t time.Time) int64 {
return t.UnixNano() / nanosecondsInMillisecond
}

@ -0,0 +1,189 @@
package queryrange
import (
"context"
"fmt"
"sort"
"testing"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
"github.com/go-kit/kit/log"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/stretchr/testify/require"
)
var (
nilShardingMetrics = logql.NewShardingMetrics(nil)
defaultReq = func() *LokiRequest {
return &LokiRequest{
Limit: 100,
StartTs: start,
EndTs: end,
Direction: logproto.BACKWARD,
Path: "/loki/api/v1/query_range",
}
}
lokiResps = []queryrange.Response{
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: defaultReq().Limit,
Version: 1,
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 6), Line: "6"},
{Timestamp: time.Unix(0, 5), Line: "5"},
},
},
},
},
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: 100,
Version: 1,
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="error"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 2), Line: "2"},
{Timestamp: time.Unix(0, 1), Line: "1"},
},
},
},
},
},
}
)
func Test_PartitionRequest(t *testing.T) {
midpt := time.Unix(0, 0).Add(500 * time.Millisecond)
cutoff := TimeToMillis(midpt)
// test split
req := defaultReq().WithStartEnd(0, cutoff*2)
before, after := partitionRequest(req, midpt)
require.Equal(t, req.WithStartEnd(0, cutoff), before)
require.Equal(t, req.WithStartEnd(cutoff, 2*cutoff), after)
// test all before cutoff
before, after = partitionRequest(req, midpt.Add(1000*time.Millisecond))
require.Equal(t, req, before)
require.Nil(t, after)
// test after cutoff
before, after = partitionRequest(req, time.Unix(0, 0))
require.Nil(t, before)
require.Equal(t, req, after)
}
func Test_shardSplitter(t *testing.T) {
splitter := &shardSplitter{
shardingware: mockHandler(lokiResps[0], nil),
next: mockHandler(lokiResps[1], nil),
now: time.Now,
MinShardingLookback: 0,
}
req := defaultReq().WithStartEnd(
TimeToMillis(time.Now().Add(-time.Hour)),
TimeToMillis(time.Now().Add(time.Hour)),
)
resp, err := splitter.Do(context.Background(), req)
require.Nil(t, err)
expected, err := lokiCodec.MergeResponse(lokiResps...)
require.Nil(t, err)
require.Equal(t, expected, resp)
}
func Test_astMapper(t *testing.T) {
called := 0
handler := queryrange.HandlerFunc(func(ctx context.Context, req queryrange.Request) (queryrange.Response, error) {
resp := lokiResps[called]
called++
return resp, nil
})
mware := newASTMapperware(
queryrange.ShardingConfigs{
chunk.PeriodConfig{
RowShards: 2,
},
},
handler,
log.NewNopLogger(),
nilShardingMetrics,
)
resp, err := mware.Do(context.Background(), defaultReq().WithQuery(`{food="bar"}`))
require.Nil(t, err)
expected, err := lokiCodec.MergeResponse(lokiResps...)
sort.Sort(logproto.Streams(expected.(*LokiResponse).Data.Result))
require.Nil(t, err)
require.Equal(t, called, 2)
require.Equal(t, expected.(*LokiResponse).Data, resp.(*LokiResponse).Data)
}
func Test_hasShards(t *testing.T) {
for i, tc := range []struct {
input queryrange.ShardingConfigs
expected bool
}{
{
input: queryrange.ShardingConfigs{
{},
},
expected: false,
},
{
input: queryrange.ShardingConfigs{
{RowShards: 16},
},
expected: true,
},
{
input: queryrange.ShardingConfigs{
{},
{RowShards: 16},
{},
},
expected: true,
},
{
input: nil,
expected: false,
},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
require.Equal(t, tc.expected, hasShards(tc.input))
})
}
}
// astmapper successful stream & prom conversion
func mockHandler(resp queryrange.Response, err error) queryrange.Handler {
return queryrange.HandlerFunc(func(ctx context.Context, req queryrange.Request) (queryrange.Response, error) {
if expired := ctx.Err(); expired != nil {
return nil, expired
}
return resp, err
})
}

@ -1,10 +1,13 @@
package queryrange
import (
"errors"
"flag"
"net/http"
"strings"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/chunk/cache"
"github.com/cortexproject/cortex/pkg/querier/frontend"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
@ -35,22 +38,32 @@ type Stopper interface {
}
// NewTripperware returns a Tripperware configured with middlewares to align, split and cache requests.
func NewTripperware(cfg Config, log log.Logger, limits Limits, registerer prometheus.Registerer) (frontend.Tripperware, Stopper, error) {
func NewTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema chunk.SchemaConfig,
minShardingLookback time.Duration,
registerer prometheus.Registerer,
) (frontend.Tripperware, Stopper, error) {
// Ensure that QuerySplitDuration uses configuration defaults.
// This avoids divide by zero errors when determining cache keys where user specific overrides don't exist.
limits = WithDefaultLimits(limits, cfg.Config)
instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer)
retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer)
shardingMetrics := logql.NewShardingMetrics(registerer)
splitByMetrics := NewSplitByMetrics(registerer)
metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, lokiCodec, PrometheusExtractor{}, instrumentMetrics, retryMetrics)
metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, PrometheusExtractor{}, instrumentMetrics, retryMetrics, shardingMetrics, splitByMetrics)
if err != nil {
return nil, nil, err
}
logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, lokiCodec, instrumentMetrics, retryMetrics)
logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, shardingMetrics, splitByMetrics)
if err != nil {
return nil, nil, err
}
return func(next http.RoundTripper) http.RoundTripper {
metricRT := metricsTripperware(next)
logFilterRT := logFilterTripperware(next)
@ -146,14 +159,34 @@ func NewLogFilterTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema chunk.SchemaConfig,
minShardingLookback time.Duration,
codec queryrange.Codec,
instrumentMetrics *queryrange.InstrumentMiddlewareMetrics,
retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics,
shardingMetrics *logql.ShardingMetrics,
splitByMetrics *SplitByMetrics,
) (frontend.Tripperware, error) {
queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)}
if cfg.SplitQueriesByInterval != 0 {
queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec))
queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec, splitByMetrics))
}
if cfg.ShardedQueries {
if minShardingLookback == 0 {
return nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled")
}
queryRangeMiddleware = append(queryRangeMiddleware,
NewQueryShardMiddleware(
log,
schema.Configs,
minShardingLookback,
instrumentMetrics, // instrumentation is included in the sharding middleware
shardingMetrics,
),
)
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("retry", instrumentMetrics), queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMiddlewareMetrics))
}
@ -171,10 +204,14 @@ func NewMetricTripperware(
cfg Config,
log log.Logger,
limits Limits,
schema chunk.SchemaConfig,
minShardingLookback time.Duration,
codec queryrange.Codec,
extractor queryrange.Extractor,
instrumentMetrics *queryrange.InstrumentMiddlewareMetrics,
retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics,
shardingMetrics *logql.ShardingMetrics,
splitByMetrics *SplitByMetrics,
) (frontend.Tripperware, Stopper, error) {
queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)}
if cfg.AlignQueriesWithStep {
@ -193,7 +230,7 @@ func NewMetricTripperware(
queryRangeMiddleware = append(
queryRangeMiddleware,
queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics),
SplitByIntervalMiddleware(limits, codec),
SplitByIntervalMiddleware(limits, codec, splitByMetrics),
)
var c cache.Cache
@ -218,6 +255,21 @@ func NewMetricTripperware(
)
}
if cfg.ShardedQueries {
if minShardingLookback == 0 {
return nil, nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled")
}
queryRangeMiddleware = append(queryRangeMiddleware,
NewQueryShardMiddleware(
log,
schema.Configs,
minShardingLookback,
instrumentMetrics, // instrumentation is included in the sharding middleware
shardingMetrics,
),
)
}
if cfg.MaxRetries > 0 {
queryRangeMiddleware = append(
queryRangeMiddleware,

@ -12,6 +12,7 @@ import (
"testing"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/chunk/cache"
"github.com/cortexproject/cortex/pkg/querier/frontend"
"github.com/cortexproject/cortex/pkg/querier/queryrange"
@ -80,7 +81,7 @@ var (
// those tests are mostly for testing the glue between all component and make sure they activate correctly.
func TestMetricsTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -144,7 +145,7 @@ func TestMetricsTripperware(t *testing.T) {
func TestLogFilterTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -191,7 +192,7 @@ func TestLogFilterTripperware(t *testing.T) {
}
func TestLogNoRegex(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -225,7 +226,7 @@ func TestLogNoRegex(t *testing.T) {
}
func TestUnhandledPath(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -249,7 +250,7 @@ func TestUnhandledPath(t *testing.T) {
}
func TestRegexpParamsSupport(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -320,7 +321,7 @@ func TestPostQueries(t *testing.T) {
}
func TestEntriesLimitsTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -351,7 +352,7 @@ func TestEntriesLimitsTripperware(t *testing.T) {
}
func TestEntriesLimitWithZeroTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil)
tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil)
if stopper != nil {
defer stopper.Stop()
}

@ -8,23 +8,14 @@ import (
"github.com/cortexproject/cortex/pkg/querier/queryrange"
"github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/weaveworks/common/httpgrpc"
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/logproto"
)
// SplitByIntervalMiddleware creates a new Middleware that splits log requests by a given interval.
func SplitByIntervalMiddleware(limits Limits, merger queryrange.Merger) queryrange.Middleware {
return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler {
return &splitByInterval{
next: next,
limits: limits,
merger: merger,
}
})
}
type lokiResult struct {
req queryrange.Request
ch chan *packedResp
@ -35,10 +26,38 @@ type packedResp struct {
err error
}
type SplitByMetrics struct {
splits prometheus.Histogram
}
func NewSplitByMetrics(r prometheus.Registerer) *SplitByMetrics {
return &SplitByMetrics{
splits: promauto.With(r).NewHistogram(prometheus.HistogramOpts{
Namespace: "loki",
Name: "query_frontend_partitions",
Help: "Number of time-based partitions (sub-requests) per request",
Buckets: prometheus.ExponentialBuckets(1, 4, 5), // 1 -> 1024
}),
}
}
type splitByInterval struct {
next queryrange.Handler
limits Limits
merger queryrange.Merger
next queryrange.Handler
limits Limits
merger queryrange.Merger
metrics *SplitByMetrics
}
// SplitByIntervalMiddleware creates a new Middleware that splits log requests by a given interval.
func SplitByIntervalMiddleware(limits Limits, merger queryrange.Merger, metrics *SplitByMetrics) queryrange.Middleware {
return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler {
return &splitByInterval{
next: next,
limits: limits,
merger: merger,
metrics: metrics,
}
})
}
func (h *splitByInterval) Feed(ctx context.Context, input []*lokiResult) chan *lokiResult {
@ -148,6 +167,7 @@ func (h *splitByInterval) Do(ctx context.Context, r queryrange.Request) (queryra
}
intervals := splitByTime(lokiRequest, interval)
h.metrics.splits.Observe(float64(len(intervals)))
// no interval should not be processed by the frontend.
if len(intervals) == 0 {

@ -16,6 +16,8 @@ import (
"github.com/grafana/loki/pkg/logproto"
)
var nilMetrics = NewSplitByMetrics(nil)
func Test_splitQuery(t *testing.T) {
tests := []struct {
@ -105,6 +107,7 @@ func Test_splitByInterval_Do(t *testing.T) {
split := SplitByIntervalMiddleware(
l,
lokiCodec,
nilMetrics,
).Wrap(next)
tests := []struct {
@ -288,6 +291,7 @@ func Test_ExitEarly(t *testing.T) {
split := SplitByIntervalMiddleware(
l,
lokiCodec,
nilMetrics,
).Wrap(next)
req := &LokiRequest{
@ -366,6 +370,7 @@ func Test_DoesntDeadlock(t *testing.T) {
split := SplitByIntervalMiddleware(
l,
lokiCodec,
nilMetrics,
).Wrap(next)
// split into n requests w/ n/2 limit, ensuring unused responses are cleaned up properly

@ -6,6 +6,7 @@ import (
"time"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/cortexproject/cortex/pkg/util"
"github.com/cortexproject/cortex/pkg/util/spanlogger"
"github.com/go-kit/kit/log/level"
@ -87,14 +88,18 @@ type batchChunkIterator struct {
// newBatchChunkIterator creates a new batch iterator with the given batchSize.
func newBatchChunkIterator(ctx context.Context, chunks []*chunkenc.LazyChunk, batchSize int, matchers []*labels.Matcher, filter logql.LineFilter, req *logproto.QueryRequest) *batchChunkIterator {
// __name__ is not something we filter by because it's a constant in loki and only used for upstream compatibility.
// Therefore remove it
for i := range matchers {
if matchers[i].Name == labels.MetricName {
matchers = append(matchers[:i], matchers[i+1:]...)
break
// __name__ is not something we filter by because it's a constant in loki
// and only used for upstream compatibility; therefore remove it.
// The same applies to the sharding label which is injected by the cortex storage code.
for _, omit := range []string{labels.MetricName, astmapper.ShardLabel} {
for i := range matchers {
if matchers[i].Name == omit {
matchers = append(matchers[:i], matchers[i+1:]...)
break
}
}
}
ctx, cancel := context.WithCancel(ctx)
res := &batchChunkIterator{
batchSize: batchSize,

@ -549,7 +549,7 @@ func Test_newBatchChunkIterator(t *testing.T) {
for name, tt := range tests {
tt := tt
t.Run(name, func(t *testing.T) {
it := newBatchChunkIterator(context.Background(), tt.chunks, tt.batchSize, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction))
it := newBatchChunkIterator(context.Background(), tt.chunks, tt.batchSize, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction, nil))
streams, _, err := iter.ReadBatch(it, 1000)
_ = it.Close()
if err != nil {
@ -756,7 +756,7 @@ func TestBuildHeapIterator(t *testing.T) {
t.Errorf("buildHeapIterator error = %v", err)
return
}
req := newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD)
req := newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil)
streams, _, err := iter.ReadBatch(it, req.Limit)
_ = it.Close()
if err != nil {

@ -8,6 +8,7 @@ import (
"github.com/cortexproject/cortex/pkg/chunk"
cortex_local "github.com/cortexproject/cortex/pkg/chunk/local"
"github.com/cortexproject/cortex/pkg/chunk/storage"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
@ -60,7 +61,18 @@ func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConf
}, nil
}
// decodeReq sanitizes an incoming request, rounds bounds, and appends the __name__ matcher
// NewTableClient creates a TableClient for managing tables for index/chunk store.
// ToDo: Add support in Cortex for registering custom table client like index client.
func NewTableClient(name string, cfg Config) (chunk.TableClient, error) {
if name == local.BoltDBShipperType {
name = "boltdb"
cfg.FSConfig = cortex_local.FSConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory}
}
return storage.NewTableClient(name, cfg.Config)
}
// decodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher,
// and adds the "__cortex_shard__" label if this is a sharded query.
func decodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) {
expr, err := req.LogSelector()
if err != nil {
@ -79,6 +91,29 @@ func decodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, mod
}
matchers = append(matchers, nameLabelMatcher)
if shards := req.GetShards(); shards != nil {
parsed, err := logql.ParseShards(shards)
if err != nil {
return nil, nil, 0, 0, err
}
for _, s := range parsed {
shardMatcher, err := labels.NewMatcher(
labels.MatchEqual,
astmapper.ShardLabel,
s.String(),
)
if err != nil {
return nil, nil, 0, 0, err
}
matchers = append(matchers, shardMatcher)
// TODO(owen-d): passing more than one shard will require
// a refactor to cortex to support it. We're leaving this codepath in
// preparation of that but will not pass more than one until it's supported.
break // nolint:staticcheck
}
}
from, through := util.RoundToMilliseconds(req.Start, req.End)
return matchers, filter, from, through, nil
}

@ -13,12 +13,14 @@ import (
"time"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"github.com/cortexproject/cortex/pkg/chunk"
cortex_local "github.com/cortexproject/cortex/pkg/chunk/local"
"github.com/cortexproject/cortex/pkg/chunk/storage"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/cortexproject/cortex/pkg/util/flagext"
"github.com/grafana/loki/pkg/iter"
@ -187,7 +189,7 @@ func Test_store_LazyQuery(t *testing.T) {
}{
{
"all",
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
@ -255,7 +257,7 @@ func Test_store_LazyQuery(t *testing.T) {
},
{
"filter regex",
newQuery("{foo=~\"ba.*\"} |~ \"1|2|3\" !~ \"2|3\"", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"ba.*\"} |~ \"1|2|3\" !~ \"2|3\"", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
@ -279,7 +281,7 @@ func Test_store_LazyQuery(t *testing.T) {
},
{
"filter matcher",
newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
@ -316,7 +318,7 @@ func Test_store_LazyQuery(t *testing.T) {
},
{
"filter time",
newQuery("{foo=~\"ba.*\"}", from, from.Add(time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"ba.*\"}", from, from.Add(time.Millisecond), logproto.FORWARD, nil),
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
@ -375,7 +377,7 @@ func Test_store_GetSeries(t *testing.T) {
}{
{
"all",
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.SeriesIdentifier{
{Labels: mustParseLabels("{foo=\"bar\"}")},
{Labels: mustParseLabels("{foo=\"bazz\"}")},
@ -384,7 +386,7 @@ func Test_store_GetSeries(t *testing.T) {
},
{
"all-single-batch",
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.SeriesIdentifier{
{Labels: mustParseLabels("{foo=\"bar\"}")},
{Labels: mustParseLabels("{foo=\"bazz\"}")},
@ -393,7 +395,7 @@ func Test_store_GetSeries(t *testing.T) {
},
{
"regexp filter (post chunk fetching)",
newQuery("{foo=~\"bar.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=~\"bar.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.SeriesIdentifier{
{Labels: mustParseLabels("{foo=\"bar\"}")},
},
@ -401,7 +403,7 @@ func Test_store_GetSeries(t *testing.T) {
},
{
"filter matcher",
newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD),
newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]logproto.SeriesIdentifier{
{Labels: mustParseLabels("{foo=\"bar\"}")},
},
@ -427,6 +429,51 @@ func Test_store_GetSeries(t *testing.T) {
}
}
func Test_store_decodeReq_Matchers(t *testing.T) {
tests := []struct {
name string
req *logproto.QueryRequest
matchers []*labels.Matcher
}{
{
"unsharded",
newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil),
[]*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.*"),
labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "logs"),
},
},
{
"unsharded",
newQuery(
"{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD,
[]astmapper.ShardAnnotation{
{Shard: 1, Of: 2},
},
),
[]*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.*"),
labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "logs"),
labels.MustNewMatcher(
labels.MatchEqual,
astmapper.ShardLabel,
astmapper.ShardAnnotation{Shard: 1, Of: 2}.String(),
),
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
ms, _, _, _, err := decodeReq(logql.SelectParams{QueryRequest: tt.req})
if err != nil {
t.Errorf("store.GetSeries() error = %v", err)
return
}
require.Equal(t, tt.matchers, ms)
})
}
}
type timeRange struct {
from, to time.Time
}

@ -9,6 +9,7 @@ import (
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/chunk/cache"
"github.com/cortexproject/cortex/pkg/ingester/client"
"github.com/cortexproject/cortex/pkg/querier/astmapper"
"github.com/davecgh/go-spew/spew"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
@ -101,14 +102,18 @@ func newMatchers(matchers string) []*labels.Matcher {
return res
}
func newQuery(query string, start, end time.Time, direction logproto.Direction) *logproto.QueryRequest {
return &logproto.QueryRequest{
func newQuery(query string, start, end time.Time, direction logproto.Direction, shards []astmapper.ShardAnnotation) *logproto.QueryRequest {
req := &logproto.QueryRequest{
Selector: query,
Start: start,
Limit: 1000,
End: end,
Direction: direction,
}
for _, shard := range shards {
req.Shards = append(req.Shards, shard.String())
}
return req
}
type mockChunkStore struct {

@ -7,9 +7,27 @@
replication_factor: 3,
memcached_replicas: 3,
querierConcurrency: 16,
grpc_server_max_msg_size: 100 << 20, // 100MB
querier: {
concurrency: 32,
},
queryFrontend: {
replicas: 2,
shard_factor: 16, // v10 schema shard factor
sharded_queries_enabled: false,
// Queries can technically be sharded an arbitrary number of times. Thus query_split_factor is used
// as a coefficient to multiply the frontend tenant queues by. The idea is that this
// yields a bit of headroom so tenant queues aren't underprovisioned. Therefore the split factor
// should represent the highest reasonable split factor for a query. If too low, a long query
// (i.e. 30d) with a high split factor (i.e. 5) would result in
// (day_splits * shard_factor * split_factor) or 30 * 16 * 5 = 2400 sharded queries, which may be
// more than the max queue size and thus would always error.
query_split_factor:: 3,
},
// Default to GCS and Bigtable for chunk and index store
storage_backend: 'bigtable,gcs',
@ -104,13 +122,20 @@
},
frontend: {
compress_responses: true,
} + if $._config.queryFrontend.sharded_queries_enabled then {
// In process tenant queues on frontends. We divide by the number of frontends;
// 2 in this case in order to apply the global limit in aggregate.
// This is basically base * shard_factor * query_split_factor / num_frontends where
max_outstanding_per_tenant: std.floor(200 * $._config.queryFrontend.shard_factor * $._config.queryFrontend.query_split_factor / $._config.queryFrontend.replicas),
}
else {
max_outstanding_per_tenant: 200,
log_queries_longer_than: '5s',
},
frontend_worker: {
frontend_address: 'query-frontend.%s.svc.cluster.local:9095' % $._config.namespace,
// Limit to N/2 worker threads per frontend, as we have two frontends.
parallelism: $._config.querierConcurrency / 2,
parallelism: std.floor($._config.querier.concurrency / $._config.queryFrontend.replicas),
grpc_client_config: {
max_send_msg_size: $._config.grpc_server_max_msg_size,
},
@ -133,10 +158,16 @@
},
},
},
} + if $._config.queryFrontend.sharded_queries_enabled then {
parallelise_shardable_queries: true,
} else {},
querier: {
query_ingesters_within: '2h', // twice the max-chunk age (1h default) for safety buffer
},
limits_config: {
enforce_metric_name: false,
max_query_parallelism: 32,
// align middleware parallelism with shard factor to optimize one-legged sharded queries.
max_query_parallelism: $._config.queryFrontend.shard_factor,
reject_old_samples: true,
reject_old_samples_max_age: '168h',
max_query_length: '12000h', // 500 days

@ -13,7 +13,8 @@
container.mixin.readinessProbe.httpGet.withPath('/ready') +
container.mixin.readinessProbe.httpGet.withPort($._config.http_listen_port) +
container.mixin.readinessProbe.withInitialDelaySeconds(15) +
container.mixin.readinessProbe.withTimeoutSeconds(1),
container.mixin.readinessProbe.withTimeoutSeconds(1) +
$.util.resourcesRequests('4', '2Gi'),
local deployment = $.apps.v1.deployment,

@ -11,14 +11,20 @@
container.new('query-frontend', $._images.query_frontend) +
container.withPorts($.util.defaultPorts) +
container.withArgsMixin($.util.mapToFlags($.query_frontend_args)) +
$.util.resourcesRequests('2', '600Mi') +
$.util.resourcesLimits(null, '1200Mi') +
$.jaeger_mixin,
$.jaeger_mixin +
if $._config.queryFrontend.sharded_queries_enabled then
$.util.resourcesRequests('2', '2Gi') +
$.util.resourcesLimits(null, '6Gi') +
container.withEnvMap({
JAEGER_REPORTER_MAX_QUEUE_SIZE: '5000',
})
else $.util.resourcesRequests('2', '600Mi') +
$.util.resourcesLimits(null, '1200Mi'),
local deployment = $.apps.v1.deployment,
query_frontend_deployment:
deployment.new('query-frontend', 2, [$.query_frontend_container]) +
deployment.new('query-frontend', $._config.queryFrontend.replicas, [$.query_frontend_container]) +
$.config_hash_mixin +
$.util.configVolumeMount('loki', '/etc/loki/config') +
$.util.configVolumeMount('overrides', '/etc/loki/overrides') +

@ -142,6 +142,24 @@ local utils = import 'mixin-utils/utils.libsonnet';
g.panel('Flush Rate') +
g.qpsPanel('loki_ingester_chunk_age_seconds_count{cluster="$cluster", job="$namespace/ingester"}'),
),
)
.addRow(
g.row('Duration')
.addPanel(
g.panel('Chunk Duration hours (end-start)') +
g.queryPanel(
[
'histogram_quantile(0.5, sum(rate(loki_ingester_chunk_bounds_hours_bucket{cluster="$cluster", job="$namespace/ingester"}[5m])) by (le))',
'histogram_quantile(0.99, sum(rate(loki_ingester_chunk_bounds_hours_bucket{cluster="$cluster", job="$namespace/ingester"}[5m])) by (le))',
'sum(rate(loki_ingester_chunk_bounds_hours_sum{cluster="$cluster", job="$namespace/ingester"}[5m])) / sum(rate(loki_ingester_chunk_bounds_hours_count{cluster="$cluster", job="$namespace/ingester"}[5m]))',
],
[
'p50',
'p99',
'avg',
],
),
)
),
},
}

Loading…
Cancel
Save