feat: Introduce shardable probabilistic topk for instant queries. (backport k227) (#14765)

Co-authored-by: Karsten Jeschkies <karsten.jeschkies@grafana.com>
pull/14816/head
loki-gh-app[bot] 1 year ago committed by GitHub
parent 4309f47681
commit 02eb02458e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 13
      docs/sources/shared/configuration.md
  2. 111
      integration/loki_micro_services_test.go
  3. 865
      pkg/logproto/sketch.pb.go
  4. 17
      pkg/logproto/sketch.proto
  5. 77
      pkg/logql/accumulator.go
  6. 332
      pkg/logql/count_min_sketch.go
  7. 88
      pkg/logql/count_min_sketch_test.go
  8. 66
      pkg/logql/downstream.go
  9. 161
      pkg/logql/downstream_test.go
  10. 13
      pkg/logql/engine.go
  11. 25
      pkg/logql/evaluator.go
  12. 4
      pkg/logql/explain.go
  13. 6
      pkg/logql/explain_test.go
  14. 7
      pkg/logql/limits.go
  15. 48
      pkg/logql/log/vector.go
  16. 19
      pkg/logql/metrics.go
  17. 8
      pkg/logql/quantile_over_time_sketch.go
  18. 7
      pkg/logql/quantile_over_time_sketch_test.go
  19. 99
      pkg/logql/shardmapper.go
  20. 65
      pkg/logql/shardmapper_test.go
  21. 85
      pkg/logql/sketch/cms.go
  22. 4
      pkg/logql/sketch/heap.go
  23. 22
      pkg/logql/sketch/heap_test.go
  24. 4
      pkg/logql/sketch/series_test.go
  25. 36
      pkg/logql/sketch/topk.go
  26. 12
      pkg/logql/sketch/topk_test.go
  27. 5
      pkg/logql/step_evaluator.go
  28. 20
      pkg/logql/syntax/ast.go
  29. 4
      pkg/logql/syntax/expr.y
  30. 837
      pkg/logql/syntax/expr.y.go
  31. 2
      pkg/logql/syntax/lex.go
  32. 4
      pkg/logql/syntax/parser_test.go
  33. 49
      pkg/logql/test_utils.go
  34. 17
      pkg/querier/queryrange/extensions.go
  35. 2
      pkg/querier/queryrange/limits/definitions.go
  36. 22
      pkg/querier/queryrange/marshal.go
  37. 795
      pkg/querier/queryrange/queryrange.pb.go
  38. 11
      pkg/querier/queryrange/queryrange.proto
  39. 9
      pkg/querier/queryrange/querysharding.go
  40. 4
      pkg/querier/queryrange/roundtrip_test.go
  41. 3
      pkg/querier/queryrange/shard_resolver.go
  42. 29
      pkg/util/validation/limits.go
  43. 97
      pkg/util/validation/limits_test.go
  44. 6
      pkg/validation/limits.go
  45. 2
      tools/dev/loki-tsdb-storage-s3/config/loki.yaml
  46. 2
      tools/dev/loki-tsdb-storage-s3/dev.dockerfile

@ -130,6 +130,11 @@ querier_rf1:
# CLI flag: -querier-rf1.engine.max-lookback-period
[max_look_back_period: <duration> | default = 30s]
# The maximum number of labels the heap of a topk query using a count min
# sketch can track.
# CLI flag: -querier-rf1.engine.max-count-min-sketch-heap-size
[max_count_min_sketch_heap_size: <int> | default = 10000]
# The maximum number of queries that can be simultaneously processed by the
# querier.
# CLI flag: -querier-rf1.max-concurrent
@ -3841,6 +3846,9 @@ otlp_config:
# CLI flag: -limits.ingestion-partition-tenant-shard-size
[ingestion_partitions_tenant_shard_size: <int> | default = 0]
# List of LogQL vector and range aggregations that should be sharded.
[shard_aggregations: <list of strings>]
# Enable metric aggregation. When enabled, pushed streams will be sampled for
# bytes and count, and these metric will be written back into Loki as a special
# __aggregated_metric__ stream, which can be queried for faster histogram
@ -4245,6 +4253,11 @@ engine:
# CLI flag: -querier.engine.max-lookback-period
[max_look_back_period: <duration> | default = 30s]
# The maximum number of labels the heap of a topk query using a count min
# sketch can track.
# CLI flag: -querier.engine.max-count-min-sketch-heap-size
[max_count_min_sketch_heap_size: <int> | default = 10000]
# The maximum number of queries that can be simultaneously processed by the
# querier.
# CLI flag: -querier.max-concurrent

@ -94,7 +94,7 @@ func TestMicroServicesIngestQuery(t *testing.T) {
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.encoding=protobuf",
"-querier.shard-aggregations=quantile_over_time",
"-querier.shard-aggregations=quantile_over_time,approx_topk",
"-frontend.tail-proxy-url="+tQuerier.HTTPURL(),
)
)
@ -784,6 +784,115 @@ func TestOTLPLogsIngestQuery(t *testing.T) {
})
}
func TestProbabilisticQuery(t *testing.T) {
clu := cluster.New(nil, cluster.SchemaWithTSDBAndTSDB, func(c *cluster.Cluster) {
c.SetSchemaVer("v13")
})
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// run initially the compactor, indexgateway, and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
"-compactor.compaction-interval=1s",
"-compactor.retention-delete-delay=1s",
// By default, a minute is added to the delete request start time. This compensates for that.
"-compactor.delete-request-cancel-period=-60s",
"-compactor.deletion-mode=filter-and-delete",
)
tIndexGateway = clu.AddComponent(
"index-gateway",
"-target=index-gateway",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only the ingester and query scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
)
require.NoError(t, clu.Run())
// the run querier.
var (
tQuerier = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.encoding=protobuf",
"-querier.shard-aggregations=quantile_over_time,approx_topk",
"-frontend.tail-proxy-url="+tQuerier.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
t.Run("ingest-logs", func(t *testing.T) {
// ingest some log lines
require.NoError(t, cliDistributor.PushLogLine("lineA", now.Add(-45*time.Minute), nil, map[string]string{"job": "one"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", now.Add(-45*time.Minute), nil, map[string]string{"job": "one"}))
require.NoError(t, cliDistributor.PushLogLine("lineC", now, nil, map[string]string{"job": "one"}))
require.NoError(t, cliDistributor.PushLogLine("lineD", now, nil, map[string]string{"job": "two"}))
})
t.Run("query", func(t *testing.T) {
resp, err := cliQueryFrontend.RunQuery(context.Background(), `approx_topk(1, count_over_time({job=~".+"}[1h]))`)
require.NoError(t, err)
assert.Equal(t, "vector", resp.Data.ResultType)
var values []string
var labels []string
for _, value := range resp.Data.Vector {
values = append(values, value.Value)
labels = append(labels, value.Metric["job"])
}
assert.ElementsMatch(t, []string{"3"}, values)
assert.ElementsMatch(t, []string{"one"}, labels)
})
}
func TestCategorizedLabels(t *testing.T) {
clu := cluster.New(nil, cluster.SchemaWithTSDB, func(c *cluster.Cluster) {
c.SetSchemaVer("v13")

File diff suppressed because it is too large Load Diff

@ -45,7 +45,20 @@ message CountMinSketch {
uint32 width = 2;
// counters is a matrix of depth * width.
repeated uint32 counters = 3;
repeated double counters = 3;
bytes hyperloglog = 4;
}
message CountMinSketchVector {
int64 timestamp_ms = 1;
CountMinSketch sketch = 2;
repeated Labels metrics = 3;
}
message Labels {
repeated LabelPair metric = 1;
}
message TopK {
@ -53,7 +66,7 @@ message TopK {
message Pair {
string event = 1;
uint32 count = 2;
double count = 2;
}
repeated Pair list = 2;

@ -115,6 +115,82 @@ func (a *QuantileSketchAccumulator) Result() []logqlmodel.Result {
}
}
type CountMinSketchAccumulator struct {
vec *CountMinSketchVector
stats stats.Result // for accumulating statistics from downstream requests
headers map[string][]string // for accumulating headers from downstream requests
warnings map[string]struct{} // for accumulating warnings from downstream requests}
}
// newCountMinSketchAccumulator returns an accumulator for sharded
// count min sketch queries that merges results as they come in.
func newCountMinSketchAccumulator() *CountMinSketchAccumulator {
return &CountMinSketchAccumulator{
headers: make(map[string][]string),
warnings: make(map[string]struct{}),
}
}
func (a *CountMinSketchAccumulator) Accumulate(_ context.Context, res logqlmodel.Result, _ int) error {
if res.Data.Type() != CountMinSketchVectorType {
return fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), CountMinSketchVectorType)
}
data, ok := res.Data.(CountMinSketchVector)
if !ok {
return fmt.Errorf("unexpected matrix type: got (%T), want (CountMinSketchVector)", res.Data)
}
// TODO(owen-d/ewelch): Shard counts should be set by the querier
// so we don't have to do it in tricky ways in multiple places.
// See pkg/logql/downstream.go:DownstreamEvaluator.Downstream
// for another example.
if res.Statistics.Summary.Shards == 0 {
res.Statistics.Summary.Shards = 1
}
a.stats.Merge(res.Statistics)
metadata.ExtendHeaders(a.headers, res.Headers)
for _, w := range res.Warnings {
a.warnings[w] = struct{}{}
}
if a.vec == nil {
a.vec = &data // TODO: maybe the matrix should already be a pointeer
return nil
}
var err error
a.vec, err = a.vec.Merge(&data)
a.stats.Merge(res.Statistics)
return err
}
func (a *CountMinSketchAccumulator) Result() []logqlmodel.Result {
headers := make([]*definitions.PrometheusResponseHeader, 0, len(a.headers))
for name, vals := range a.headers {
headers = append(
headers,
&definitions.PrometheusResponseHeader{
Name: name,
Values: vals,
},
)
}
warnings := maps.Keys(a.warnings)
sort.Strings(warnings)
return []logqlmodel.Result{
{
Data: a.vec,
Headers: headers,
Warnings: warnings,
Statistics: a.stats,
},
}
}
// heap impl for keeping only the top n results across m streams
// importantly, AccumulatedStreams is _bounded_, so it will only
// store the top `limit` results across all streams.
@ -334,7 +410,6 @@ func (acc *AccumulatedStreams) appendTo(dst, src *logproto.Stream) {
acc.count += len(src.Entries)
heap.Fix(acc, acc.labelmap[dst.Labels])
}
// Pop returns a stream with one entry. It pops the first entry of the first stream

@ -0,0 +1,332 @@
package logql
import (
"container/heap"
"fmt"
"github.com/axiomhq/hyperloglog"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql/sketch"
"github.com/grafana/loki/v3/pkg/logql/syntax"
)
const (
CountMinSketchVectorType = "CountMinSketchVector"
epsilon = 0.0001
// delta of 0.01 results in a sketch size of 27183 * 7 * 4 bytes = 761,124 bytes, 0.05 would yield 543,660 bytes
delta = 0.01
)
// CountMinSketchVector tracks the count or sum of values of a metric, ie list of label value pairs. It's storage for
// the values is upper bound bu delta and epsilon. To limit the storage for labels see HeapCountMinSketchVector.
// The main use case is for a topk approximation.
type CountMinSketchVector struct {
T int64
F *sketch.CountMinSketch
Metrics []labels.Labels
}
func (CountMinSketchVector) SampleVector() promql.Vector {
return promql.Vector{}
}
func (CountMinSketchVector) QuantileSketchVec() ProbabilisticQuantileVector {
return ProbabilisticQuantileVector{}
}
func (v CountMinSketchVector) CountMinSketchVec() CountMinSketchVector {
return v
}
func (v *CountMinSketchVector) Merge(right *CountMinSketchVector) (*CountMinSketchVector, error) {
// The underlying CMS implementation already merges the HLL sketches that are part of that structure.
err := v.F.Merge(right.F)
if err != nil {
return v, err
}
// Merge labels without duplication. Note: the CMS does not limit the number of labels as the
// HeapCountMinSketchVector does.
processed := map[string]struct{}{}
for _, l := range v.Metrics {
processed[l.String()] = struct{}{}
}
for _, r := range right.Metrics {
if _, duplicate := processed[r.String()]; !duplicate {
processed[r.String()] = struct{}{}
v.Metrics = append(v.Metrics, r)
}
}
return v, nil
}
func (CountMinSketchVector) String() string {
return "CountMinSketchVector()"
}
func (CountMinSketchVector) Type() promql_parser.ValueType { return CountMinSketchVectorType }
func (v CountMinSketchVector) ToProto() (*logproto.CountMinSketchVector, error) {
p := &logproto.CountMinSketchVector{
TimestampMs: v.T,
Metrics: make([]*logproto.Labels, len(v.Metrics)),
Sketch: &logproto.CountMinSketch{
Depth: v.F.Depth,
Width: v.F.Width,
},
}
// insert the hll sketch
hllBytes, err := v.F.HyperLogLog.MarshalBinary()
if err != nil {
return nil, err
}
p.Sketch.Hyperloglog = hllBytes
// Serialize CMS
p.Sketch.Counters = make([]float64, 0, v.F.Depth*v.F.Width)
for row := uint32(0); row < v.F.Depth; row++ {
p.Sketch.Counters = append(p.Sketch.Counters, v.F.Counters[row]...)
}
// Serialize metric labels
for i, metric := range v.Metrics {
p.Metrics[i] = &logproto.Labels{
Metric: make([]*logproto.LabelPair, len(metric)),
}
for j, pair := range metric {
p.Metrics[i].Metric[j] = &logproto.LabelPair{
Name: pair.Name,
Value: pair.Value,
}
}
}
return p, nil
}
func CountMinSketchVectorFromProto(p *logproto.CountMinSketchVector) (CountMinSketchVector, error) {
vec := CountMinSketchVector{
T: p.TimestampMs,
Metrics: make([]labels.Labels, len(p.Metrics)),
}
// Deserialize CMS
var err error
vec.F, err = sketch.NewCountMinSketch(p.Sketch.Width, p.Sketch.Depth)
if err != nil {
return vec, err
}
hll := hyperloglog.New()
if err := hll.UnmarshalBinary(p.Sketch.Hyperloglog); err != nil {
return vec, err
}
vec.F.HyperLogLog = hll
for row := 0; row < int(vec.F.Depth); row++ {
s := row * int(vec.F.Width)
e := s + int(vec.F.Width)
copy(vec.F.Counters[row], p.Sketch.Counters[s:e])
}
// Deserialize metric labels
for i, in := range p.Metrics {
lbls := make(labels.Labels, len(in.Metric))
for j, labelPair := range in.Metric {
lbls[j].Name = labelPair.Name
lbls[j].Value = labelPair.Value
}
vec.Metrics[i] = lbls
}
return vec, nil
}
// HeapCountMinSketchVector is a CountMinSketchVector that keeps the number of metrics to a defined maximum.
type HeapCountMinSketchVector struct {
CountMinSketchVector
// internal set of observed events
observed map[string]struct{}
maxLabels int
}
func NewHeapCountMinSketchVector(ts int64, metricsLength, maxLabels int) HeapCountMinSketchVector {
f, _ := sketch.NewCountMinSketchFromErrorAndProbability(epsilon, delta)
if metricsLength >= maxLabels {
metricsLength = maxLabels
}
return HeapCountMinSketchVector{
CountMinSketchVector: CountMinSketchVector{
T: ts,
F: f,
Metrics: make([]labels.Labels, 0, metricsLength),
},
observed: make(map[string]struct{}),
maxLabels: maxLabels,
}
}
func (v *HeapCountMinSketchVector) Add(metric labels.Labels, value float64) {
// TODO: we save a lot of allocations by reusing the buffer inside metric.String
metricString := metric.String()
v.F.Add(metricString, value)
// Add our metric if we haven't seen it
if _, ok := v.observed[metricString]; !ok {
heap.Push(v, metric)
v.observed[metricString] = struct{}{}
} else if v.Metrics[0].String() == metricString {
// The smalles element has been updated to fix the heap.
heap.Fix(v, 0)
}
// The maximum number of labels has been reached, so drop the smallest element.
if len(v.Metrics) > v.maxLabels {
metric := heap.Pop(v).(labels.Labels)
delete(v.observed, metric.String())
}
}
func (v HeapCountMinSketchVector) Len() int {
return len(v.Metrics)
}
func (v HeapCountMinSketchVector) Less(i, j int) bool {
left := v.F.Count(v.Metrics[i].String())
right := v.F.Count(v.Metrics[j].String())
return left < right
}
func (v HeapCountMinSketchVector) Swap(i, j int) {
v.Metrics[i], v.Metrics[j] = v.Metrics[j], v.Metrics[i]
}
func (v *HeapCountMinSketchVector) Push(x any) {
v.Metrics = append(v.Metrics, x.(labels.Labels))
}
func (v *HeapCountMinSketchVector) Pop() any {
old := v.Metrics
n := len(old)
x := old[n-1]
v.Metrics = old[0 : n-1]
return x
}
// JoinCountMinSketchVector joins the results from stepEvaluator into a CountMinSketchVector.
func JoinCountMinSketchVector(_ bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) {
vec := r.CountMinSketchVec()
if stepEvaluator.Error() != nil {
return nil, stepEvaluator.Error()
}
if GetRangeType(params) != InstantType {
return nil, fmt.Errorf("count min sketches are only supported on instant queries")
}
return vec, nil
}
func newCountMinSketchVectorAggEvaluator(nextEvaluator StepEvaluator, expr *syntax.VectorAggregationExpr, maxLabels int) (*countMinSketchVectorAggEvaluator, error) {
if expr.Grouping.Groups != nil {
return nil, fmt.Errorf("count min sketch vector aggregation does not support any grouping")
}
return &countMinSketchVectorAggEvaluator{
nextEvaluator: nextEvaluator,
expr: expr,
buf: make([]byte, 0, 1024),
lb: labels.NewBuilder(nil),
maxLabels: maxLabels,
}, nil
}
// countMinSketchVectorAggEvaluator processes sample vectors and aggregates them in a count min sketch with a heap.
type countMinSketchVectorAggEvaluator struct {
nextEvaluator StepEvaluator
expr *syntax.VectorAggregationExpr
buf []byte
lb *labels.Builder
maxLabels int
}
func (e *countMinSketchVectorAggEvaluator) Next() (bool, int64, StepResult) {
next, ts, r := e.nextEvaluator.Next()
if !next {
return false, 0, CountMinSketchVector{}
}
vec := r.SampleVector()
result := NewHeapCountMinSketchVector(ts, len(vec), e.maxLabels)
for _, s := range vec {
result.Add(s.Metric, s.F)
}
return next, ts, result
}
func (e *countMinSketchVectorAggEvaluator) Explain(parent Node) {
b := parent.Child("CountMinSketchVectorAgg")
e.nextEvaluator.Explain(b)
}
func (e *countMinSketchVectorAggEvaluator) Close() error {
return e.nextEvaluator.Close()
}
func (e *countMinSketchVectorAggEvaluator) Error() error {
return e.nextEvaluator.Error()
}
// CountMinSketchVectorStepEvaluator evaluates a count min sketch into a promql.Vector.
type CountMinSketchVectorStepEvaluator struct {
exhausted bool
vec *CountMinSketchVector
}
var _ StepEvaluator = NewQuantileSketchVectorStepEvaluator(nil, 0)
func NewCountMinSketchVectorStepEvaluator(vec *CountMinSketchVector) *CountMinSketchVectorStepEvaluator {
return &CountMinSketchVectorStepEvaluator{
exhausted: false,
vec: vec,
}
}
func (e *CountMinSketchVectorStepEvaluator) Next() (bool, int64, StepResult) {
if e.exhausted {
return false, 0, SampleVector{}
}
vec := make(promql.Vector, len(e.vec.Metrics))
for i, labels := range e.vec.Metrics {
f := e.vec.F.Count(labels.String())
vec[i] = promql.Sample{
T: e.vec.T,
F: float64(f),
Metric: labels,
}
}
return true, e.vec.T, SampleVector(vec)
}
func (*CountMinSketchVectorStepEvaluator) Close() error { return nil }
func (*CountMinSketchVectorStepEvaluator) Error() error { return nil }

@ -0,0 +1,88 @@
package logql
import (
"testing"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/logql/sketch"
)
func TestHeapCountMinSketchVectorHeap(t *testing.T) {
v := NewHeapCountMinSketchVector(0, 0, 3)
a := labels.Labels{{Name: "event", Value: "a"}}
b := labels.Labels{{Name: "event", Value: "b"}}
c := labels.Labels{{Name: "event", Value: "c"}}
d := labels.Labels{{Name: "event", Value: "d"}}
v.Add(a, 2.0)
v.Add(b, 4.0)
v.Add(d, 5.5)
require.Equal(t, "a", v.Metrics[0][0].Value)
// Adding c drops a
v.Add(c, 3.0)
require.Equal(t, "c", v.Metrics[0][0].Value)
require.Len(t, v.Metrics, v.maxLabels)
require.NotContains(t, v.observed, a.String())
// Increasing c to 6.0 should make b with 4,0 the smallest
v.Add(c, 3.0)
require.Equal(t, "b", v.Metrics[0][0].Value)
// Increasing a to 5.0 drops b because it's the smallest
v.Add(a, 3.0)
require.Equal(t, "a", v.Metrics[0][0].Value)
require.Len(t, v.Metrics, v.maxLabels)
require.NotContains(t, v.observed, b.String())
// Verify final list
final := make([]string, v.maxLabels)
for i, metric := range v.Metrics {
final[i] = metric[0].Value
}
require.ElementsMatch(t, []string{"a", "d", "c"}, final)
}
func TestCountMinSketchSerialization(t *testing.T) {
metric := []labels.Label{{Name: "foo", Value: "bar"}}
cms, err := sketch.NewCountMinSketch(4, 2)
require.NoError(t, err)
vec := HeapCountMinSketchVector{
CountMinSketchVector: CountMinSketchVector{
T: 42,
F: cms,
},
observed: make(map[string]struct{}, 0),
maxLabels: 10_000,
}
vec.Add(metric, 42.0)
hllBytes, _ := vec.F.HyperLogLog.MarshalBinary()
proto := &logproto.CountMinSketchVector{
TimestampMs: 42,
Sketch: &logproto.CountMinSketch{
Depth: 2,
Width: 4,
Counters: []float64{0, 0, 0, 42, 0, 42, 0, 0},
Hyperloglog: hllBytes,
},
Metrics: []*logproto.Labels{
{Metric: []*logproto.LabelPair{{Name: "foo", Value: "bar"}}},
},
}
actual, err := vec.ToProto()
require.NoError(t, err)
require.Equal(t, proto, actual)
round, err := CountMinSketchVectorFromProto(actual)
require.NoError(t, err)
// The HeapCountMinSketchVector is serialized to a CountMinSketchVector.
require.Equal(t, round, vec.CountMinSketchVector)
}

@ -357,6 +357,34 @@ func (e *MergeLastOverTimeExpr) Walk(f syntax.WalkFn) {
}
}
type CountMinSketchEvalExpr struct {
syntax.SampleExpr
downstreams []DownstreamSampleExpr
}
func (e CountMinSketchEvalExpr) String() string {
var sb strings.Builder
for i, d := range e.downstreams {
if i >= defaultMaxDepth {
break
}
if i > 0 {
sb.WriteString(" ++ ")
}
sb.WriteString(d.String())
}
return fmt.Sprintf("CountMinSketchEval<%s>", sb.String())
}
func (e *CountMinSketchEvalExpr) Walk(f syntax.WalkFn) {
f(e)
for _, d := range e.downstreams {
d.Walk(f)
}
}
type Downstreamable interface {
Downstreamer(context.Context) Downstreamer
}
@ -435,7 +463,7 @@ func (errorQuerier) SelectSamples(_ context.Context, _ SelectSampleParams) (iter
func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator {
return &DownstreamEvaluator{
Downstreamer: downstreamer,
defaultEvaluator: NewDefaultEvaluator(&errorQuerier{}, 0),
defaultEvaluator: NewDefaultEvaluator(&errorQuerier{}, 0, 0),
}
}
@ -554,7 +582,6 @@ func (ev *DownstreamEvaluator) NewStepEvaluator(
xs := make([]promql.Matrix, 0, len(queries))
for _, res := range results {
switch data := res.Data.(type) {
case promql.Matrix:
xs = append(xs, data)
@ -591,7 +618,6 @@ func (ev *DownstreamEvaluator) NewStepEvaluator(
xs := make([]promql.Matrix, 0, len(queries))
for _, res := range results {
switch data := res.Data.(type) {
case promql.Matrix:
xs = append(xs, data)
@ -600,6 +626,40 @@ func (ev *DownstreamEvaluator) NewStepEvaluator(
}
}
return NewMergeLastOverTimeStepEvaluator(params, xs), nil
case *CountMinSketchEvalExpr:
queries := make([]DownstreamQuery, len(e.downstreams))
for i, d := range e.downstreams {
qry := DownstreamQuery{
Params: ParamsWithExpressionOverride{
Params: params,
ExpressionOverride: d.SampleExpr,
},
}
if shard := d.shard; shard != nil {
qry.Params = ParamsWithShardsOverride{
Params: qry.Params,
ShardsOverride: Shards{shard.Shard}.Encode(),
}
}
queries[i] = qry
}
acc := newCountMinSketchAccumulator()
results, err := ev.Downstream(ctx, queries, acc)
if err != nil {
return nil, err
}
if len(results) != 1 {
return nil, fmt.Errorf("unexpected results length for sharded count min sketch: got (%d), want (1)", len(results))
}
vector, ok := results[0].Data.(*CountMinSketchVector)
if !ok {
return nil, fmt.Errorf("unexpected matrix type: got (%T), want (CountMinSketchVector)", results[0].Data)
}
return NewCountMinSketchVectorStepEvaluator(vector), nil
default:
return ev.defaultEvaluator.NewStepEvaluator(ctx, nextEvFactory, e, params)
}

@ -2,6 +2,7 @@ package logql
import (
"context"
"fmt"
"math"
"testing"
"time"
@ -18,8 +19,10 @@ import (
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
)
var nilShardMetrics = NewShardMapperMetrics(nil)
var nilRangeMetrics = NewRangeMapperMetrics(nil)
var (
nilShardMetrics = NewShardMapperMetrics(nil)
nilRangeMetrics = NewRangeMapperMetrics(nil)
)
func TestMappingEquivalence(t *testing.T) {
var (
@ -193,7 +196,9 @@ func TestMappingEquivalenceSketches(t *testing.T) {
streams,
)
opts := EngineOpts{}
opts := EngineOpts{
MaxCountMinSketchHeapSize: 10_000,
}
regular := NewEngine(opts, q, NoLimits, log.NewNopLogger())
sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, NoLimits, log.NewNopLogger())
@ -236,8 +241,8 @@ func TestMappingEquivalenceSketches(t *testing.T) {
// plus set step and interval to 0
params, err := NewLiteralParams(
tc.query,
time.Unix(0, int64(rounds+1)),
time.Unix(0, int64(rounds+1)),
time.Unix(1, 0),
time.Unix(1, 0),
0,
0,
logproto.FORWARD,
@ -246,11 +251,12 @@ func TestMappingEquivalenceSketches(t *testing.T) {
nil,
)
require.NoError(t, err)
qry := regular.Query(params)
qry := regular.Query(params.Copy())
ctx := user.InjectOrgID(context.Background(), "fake")
strategy := NewPowerOfTwoStrategy(ConstantShards(shards))
mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime})
mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk})
_, _, mapped, err := mapper.Parse(params.GetExpression())
require.NoError(t, err)
@ -261,6 +267,7 @@ func TestMappingEquivalenceSketches(t *testing.T) {
res, err := qry.Exec(ctx)
require.NoError(t, err)
require.NotEmpty(t, res.Data.(promql.Vector))
shardedRes, err := shardedQry.Exec(ctx)
require.NoError(t, err)
@ -270,6 +277,132 @@ func TestMappingEquivalenceSketches(t *testing.T) {
}
}
func TestApproxTopkSketches(t *testing.T) {
var (
rounds = 20
limit = 100
)
limits := &fakeLimits{
maxSeries: math.MaxInt64,
timeout: time.Hour,
}
for _, tc := range []struct {
labelShards int
totalStreams int
shardedQuery string
regularQuery string
realtiveError float64
//cardinalityEstimate int
}{
// Note:our data generation results in less spread between topk things for 10k streams than for 100k streams
// if we have 1k streams, we can get much more accurate results for topk 10 than topk 100
{
labelShards: 3,
totalStreams: 100,
shardedQuery: `approx_topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.0012,
//cardinalityEstimate: 3,
},
{
labelShards: 10,
totalStreams: 100,
shardedQuery: `approx_topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.005,
},
{
labelShards: 10,
totalStreams: 1_000,
shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.0015,
},
{
labelShards: 100,
totalStreams: 1_000,
shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.022,
},
{
labelShards: 100,
totalStreams: 10_000,
shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.008,
},
{
labelShards: 100,
totalStreams: 100_000,
shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`,
realtiveError: 0.0015,
},
} {
t.Run(fmt.Sprintf("%s/%d/%d", tc.shardedQuery, tc.labelShards, tc.totalStreams), func(t *testing.T) {
streams := randomStreams(tc.totalStreams, rounds+1, tc.labelShards, []string{"a", "b", "c", "d"}, true)
q := NewMockQuerier(
tc.labelShards,
streams,
)
opts := EngineOpts{
MaxCountMinSketchHeapSize: 10_000,
}
regular := NewEngine(opts, q, limits, log.NewNopLogger())
sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, limits, log.NewNopLogger())
// for an instant query we set the start and end to the same timestamp
// plus set step and interval to 0
params, err := NewLiteralParams(
tc.regularQuery,
time.Unix(1, 0),
time.Unix(1, 0),
0,
0,
logproto.FORWARD,
uint32(limit),
nil,
nil,
)
require.NoError(t, err)
qry := regular.Query(params.Copy())
ctx := user.InjectOrgID(context.Background(), "fake")
strategy := NewPowerOfTwoStrategy(ConstantShards(tc.labelShards))
mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk})
params.queryString = tc.shardedQuery
params.queryExpr, err = syntax.ParseExpr(params.queryString)
require.NoError(t, err)
_, _, mapped, err := mapper.Parse(params.GetExpression())
require.NoError(t, err)
shardedQry := sharded.Query(ctx, ParamsWithExpressionOverride{
Params: params,
ExpressionOverride: mapped,
})
res, err := qry.Exec(ctx)
require.NoError(t, err)
require.NotEmpty(t, res.Data.(promql.Vector))
shardedRes, err := shardedQry.Exec(ctx)
require.NoError(t, err)
relativeErrorVector(t, res.Data.(promql.Vector), shardedRes.Data.(promql.Vector), tc.realtiveError)
// we can't check this here currently because the CMS vector step evaluators Next function translates
// each steps probabilistic result into just a promql.Vector
// require.Equal(t, tc.cardinalityEstimate, res.Data.(CountMinSketchVector).F.HyperLogLog.Estimate())
})
}
}
func TestShardCounter(t *testing.T) {
var (
shards = 3
@ -591,7 +724,7 @@ func TestRangeMappingEquivalence(t *testing.T) {
rangeQry := downstreamEngine.Query(ctx, ParamsWithExpressionOverride{Params: params, ExpressionOverride: rangeExpr})
rangeRes, err := rangeQry.Exec(ctx)
require.Nil(t, err)
require.NoError(t, err)
require.Equal(t, res.Data, rangeRes.Data)
})
@ -661,14 +794,16 @@ func relativeErrorVector(t *testing.T, expected, actual promql.Vector, alpha flo
e := make([]float64, len(expected))
a := make([]float64, len(expected))
inTopk := 0
for i := 0; i < len(expected); i++ {
require.Equal(t, expected[i].Metric, actual[i].Metric)
e[i] = expected[i].F
a[i] = expected[i].F
if labels.Equal(expected[i].Metric, actual[i].Metric) {
e[i] = expected[i].F
a[i] = actual[i].F
inTopk++
}
}
require.True(t, float64(inTopk/len(expected)) > 0.9, "not enough of the real topk elements were in the output %f", float64(inTopk/len(expected)))
require.InEpsilonSlice(t, e, a, alpha)
}
func TestFormat_ShardedExpr(t *testing.T) {

@ -146,10 +146,15 @@ type EngineOpts struct {
// LogExecutingQuery will control if we log the query when Exec is called.
LogExecutingQuery bool `yaml:"-"`
// MaxCountMinSketchHeapSize is the maximum number of labels the heap for a topk query using a count min sketch
// can track. This impacts the memory usage and accuracy of a sharded probabilistic topk query.
MaxCountMinSketchHeapSize int `yaml:"max_count_min_sketch_heap_size"`
}
func (opts *EngineOpts) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
f.DurationVar(&opts.MaxLookBackPeriod, prefix+".engine.max-lookback-period", 30*time.Second, "The maximum amount of time to look back for log lines. Used only for instant log queries.")
f.IntVar(&opts.MaxCountMinSketchHeapSize, prefix+".engine.max-count-min-sketch-heap-size", 10_000, "The maximum number of labels the heap of a topk query using a count min sketch can track.")
// Log executing query by default
opts.LogExecutingQuery = true
}
@ -176,7 +181,7 @@ func NewEngine(opts EngineOpts, q Querier, l Limits, logger log.Logger) *Engine
}
return &Engine{
logger: logger,
evaluatorFactory: NewDefaultEvaluator(q, opts.MaxLookBackPeriod),
evaluatorFactory: NewDefaultEvaluator(q, opts.MaxLookBackPeriod, opts.MaxCountMinSketchHeapSize),
limits: l,
opts: opts,
}
@ -386,7 +391,11 @@ func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_
}
return q.JoinSampleVector(next, vec, stepEvaluator, maxSeries, mfl)
case ProbabilisticQuantileVector:
return MergeQuantileSketchVector(next, vec, stepEvaluator, q.params)
return JoinQuantileSketchVector(next, vec, stepEvaluator, q.params)
case CountMinSketchVector:
return JoinCountMinSketchVector(next, vec, stepEvaluator, q.params)
case HeapCountMinSketchVector:
return JoinCountMinSketchVector(next, vec.CountMinSketchVector, stepEvaluator, q.params)
default:
return nil, fmt.Errorf("unsupported result type: %T", r)
}

@ -282,15 +282,17 @@ func EvaluatorUnsupportedType(expr syntax.Expr, ev EvaluatorFactory) error {
}
type DefaultEvaluator struct {
maxLookBackPeriod time.Duration
querier Querier
maxLookBackPeriod time.Duration
maxCountMinSketchHeapSize int
querier Querier
}
// NewDefaultEvaluator constructs a DefaultEvaluator
func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) *DefaultEvaluator {
func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration, maxCountMinSketchHeapSize int) *DefaultEvaluator {
return &DefaultEvaluator{
querier: querier,
maxLookBackPeriod: maxLookBackPeriod,
querier: querier,
maxLookBackPeriod: maxLookBackPeriod,
maxCountMinSketchHeapSize: maxCountMinSketchHeapSize,
}
}
@ -350,7 +352,7 @@ func (ev *DefaultEvaluator) NewStepEvaluator(
return newRangeAggEvaluator(iter.NewPeekingSampleIterator(it), rangExpr, q, rangExpr.Left.Offset)
})
}
return newVectorAggEvaluator(ctx, nextEvFactory, e, q)
return newVectorAggEvaluator(ctx, nextEvFactory, e, q, ev.maxCountMinSketchHeapSize)
case *syntax.RangeAggregationExpr:
it, err := ev.querier.SelectSamples(ctx, SelectSampleParams{
&logproto.SampleQueryRequest{
@ -391,7 +393,8 @@ func newVectorAggEvaluator(
evFactory SampleEvaluatorFactory,
expr *syntax.VectorAggregationExpr,
q Params,
) (*VectorAggEvaluator, error) {
maxCountMinSketchHeapSize int,
) (StepEvaluator, error) {
if expr.Grouping == nil {
return nil, errors.Errorf("aggregation operator '%q' without grouping", expr.Operation)
}
@ -401,6 +404,10 @@ func newVectorAggEvaluator(
}
sort.Strings(expr.Grouping.Groups)
if expr.Operation == syntax.OpTypeCountMinSketch {
return newCountMinSketchVectorAggEvaluator(nextEvaluator, expr, maxCountMinSketchHeapSize)
}
return &VectorAggEvaluator{
nextEvaluator: nextEvaluator,
expr: expr,
@ -1198,7 +1205,8 @@ type VectorIterator struct {
}
func newVectorIterator(val float64,
stepMs, startMs, endMs int64) *VectorIterator {
stepMs, startMs, endMs int64,
) *VectorIterator {
if stepMs == 0 {
stepMs = 1
}
@ -1294,6 +1302,7 @@ func (e *LabelReplaceEvaluator) Next() (bool, int64, StepResult) {
func (e *LabelReplaceEvaluator) Close() error {
return e.nextEvaluator.Close()
}
func (e *LabelReplaceEvaluator) Error() error {
return e.nextEvaluator.Error()
}

@ -67,6 +67,10 @@ func (e *mergeOverTimeStepEvaluator) Explain(parent Node) {
parent.Child("MergeFirstOverTime")
}
func (e *CountMinSketchVectorStepEvaluator) Explain(parent Node) {
parent.Child("CountMinSketchVector")
}
func (EmptyEvaluator[SampleVector]) Explain(parent Node) {
parent.Child("Empty")
}

@ -14,7 +14,6 @@ import (
)
func TestExplain(t *testing.T) {
query := `topk(5, avg_over_time({app="loki"} |= "caller=metrics.go" | logfmt | unwrap bytes [5s]))`
// TODO(karsten): Ideally the querier and downstreamer are not required
@ -25,7 +24,7 @@ func TestExplain(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "fake")
defaultEv := NewDefaultEvaluator(querier, 30*time.Second)
defaultEv := NewDefaultEvaluator(querier, 30*time.Second, 10_000)
downEv := &DownstreamEvaluator{Downstreamer: MockDownstreamer{regular}, defaultEvaluator: defaultEv}
strategy := NewPowerOfTwoStrategy(ConstantShards(4))
@ -46,8 +45,7 @@ func TestExplain(t *testing.T) {
tree := NewTree()
ev.Explain(tree)
expected :=
`[topk, by ()] VectorAgg
expected := `[topk, by ()] VectorAgg
Concat
VectorStep
...

@ -8,9 +8,10 @@ import (
"github.com/grafana/loki/v3/pkg/util/validation"
)
var (
NoLimits = &fakeLimits{maxSeries: math.MaxInt32}
)
var NoLimits = &fakeLimits{
maxSeries: math.MaxInt32,
timeout: math.MaxInt32,
}
// Limits allow the engine to fetch limits for a given users.
type Limits interface {

@ -0,0 +1,48 @@
package log
type VectorInt []int64
type VectorString struct {
Offsets VectorInt
Lines []byte
}
// TODO: we might want an interface to support different types of batches. https://github.com/jeschkies/loki/blob/065a34a1afb765e45d15430c143ac522d0308646/pkg/logql/vectorized.go#L54
type Batch struct {
Timestamps VectorInt
Entries VectorString
Selection []int
// TODO: Add selection
}
// Returns the timestamp and line for index i or false
func (b *Batch) Get(i int) (int64, []byte, bool) {
if i < 0 || i >= len(b.Timestamps) {
return 0, nil, false
}
prevOffset := 0
if i > 0 {
prevOffset = int(b.Entries.Offsets[i-1])
}
return b.Timestamps[i], b.Entries.Lines[prevOffset:b.Entries.Offsets[i]], true
}
func (b *Batch) Iter(yield func(int64, []byte) bool) {
prevOffset := 0
for i, ts := range b.Timestamps {
if i > 0 {
prevOffset = int(b.Entries.Offsets[i-1])
}
line := b.Entries.Lines[prevOffset:b.Entries.Offsets[i]]
if !yield(ts, line) {
return
}
}
}
func (b *Batch) Append(ts int64, line []byte) {
b.Timestamps = append(b.Timestamps, ts)
b.Entries.Offsets = append(b.Entries.Offsets, int64(len(b.Entries.Lines)))
b.Entries.Lines = append(b.Entries.Lines, line...)
}

@ -95,12 +95,13 @@ func RecordRangeAndInstantQueryMetrics(
result promql_parser.Value,
) {
var (
logger = fixLogger(ctx, log)
rangeType = GetRangeType(p)
rt = string(rangeType)
latencyType = latencyTypeFast
returnedLines = 0
queryTags, _ = ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty.
logger = fixLogger(ctx, log)
rangeType = GetRangeType(p)
rt = string(rangeType)
latencyType = latencyTypeFast
returnedLines = 0
cardinalityEstimate = uint64(0)
queryTags, _ = ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty.
)
queryType, err := QueryType(p.GetExpression())
@ -141,6 +142,10 @@ func RecordRangeAndInstantQueryMetrics(
bloomRatio = float64(stats.Index.TotalChunks-stats.Index.PostFilterChunks) / float64(stats.Index.TotalChunks)
}
if r, ok := result.(CountMinSketchVector); ok {
cardinalityEstimate = r.F.HyperLogLog.Estimate()
}
logValues = append(logValues, []interface{}{
"latency", latencyType, // this can be used to filter log lines.
"query", query,
@ -187,6 +192,8 @@ func RecordRangeAndInstantQueryMetrics(
"cache_result_hit", resultCache.EntriesFound,
"cache_result_download_time", resultCache.CacheDownloadTime(),
"cache_result_query_length_served", resultCache.CacheQueryLengthServed(),
// Cardinality estimate for some approximate query types
"cardinality_estimate", cardinalityEstimate,
// The total of chunk reference fetched from index.
"ingester_chunk_refs", stats.Ingester.Store.GetTotalChunksRef(),
// Total number of chunks fetched.

@ -64,6 +64,10 @@ func (q ProbabilisticQuantileVector) QuantileSketchVec() ProbabilisticQuantileVe
return q
}
func (ProbabilisticQuantileVector) CountMinSketchVec() CountMinSketchVector {
return CountMinSketchVector{}
}
func (q ProbabilisticQuantileVector) ToProto() *logproto.QuantileSketchVector {
samples := make([]*logproto.QuantileSketchSample, len(q))
for i, sample := range q {
@ -265,8 +269,8 @@ func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sk
return s
}
// MergeQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix.
func MergeQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) {
// JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix.
func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) {
vec := r.QuantileSketchVec()
if stepEvaluator.Error() != nil {
return nil, stepEvaluator.Error()

@ -16,7 +16,7 @@ import (
"github.com/grafana/loki/v3/pkg/logqlmodel"
)
func TestProbabilisticMQuantileMatrixSerialization(t *testing.T) {
func TestProbabilisticQuantileMatrixSerialization(t *testing.T) {
emptySketch := sketch.NewDDSketch()
ddsketchBytes := make([]byte, 0)
emptySketch.Encode(&ddsketchBytes, false)
@ -69,7 +69,7 @@ func TestJoinQuantileSketchVectorError(t *testing.T) {
ev := errorStepEvaluator{
err: errors.New("could not evaluate"),
}
_, err := MergeQuantileSketchVector(true, result, ev, LiteralParams{})
_, err := JoinQuantileSketchVector(true, result, ev, LiteralParams{})
require.ErrorContains(t, err, "could not evaluate")
}
@ -113,7 +113,6 @@ func (e errorStepEvaluator) Error() error {
func (e errorStepEvaluator) Explain(Node) {}
func BenchmarkJoinQuantileSketchVector(b *testing.B) {
selRange := (5 * time.Second).Nanoseconds()
step := (30 * time.Second)
offset := int64(0)
@ -136,7 +135,7 @@ func BenchmarkJoinQuantileSketchVector(b *testing.B) {
iter: iter,
}
_, _, r := ev.Next()
m, err := MergeQuantileSketchVector(true, r.QuantileSketchVec(), ev, params)
m, err := JoinQuantileSketchVector(true, r.QuantileSketchVec(), ev, params)
require.NoError(b, err)
m.(ProbabilisticQuantileMatrix).Release()
}

@ -16,6 +16,7 @@ const (
ShardLastOverTime = "last_over_time"
ShardFirstOverTime = "first_over_time"
ShardQuantileOverTime = "quantile_over_time"
SupportApproxTopk = "approx_topk"
)
type ShardMapper struct {
@ -24,29 +25,32 @@ type ShardMapper struct {
quantileOverTimeSharding bool
lastOverTimeSharding bool
firstOverTimeSharding bool
approxTopkSupport bool
}
func NewShardMapper(strategy ShardingStrategy, metrics *MapperMetrics, shardAggregation []string) ShardMapper {
quantileOverTimeSharding := false
lastOverTimeSharding := false
firstOverTimeSharding := false
mapper := ShardMapper{
shards: strategy,
metrics: metrics,
quantileOverTimeSharding: false,
lastOverTimeSharding: false,
firstOverTimeSharding: false,
approxTopkSupport: false,
}
for _, a := range shardAggregation {
switch a {
case ShardQuantileOverTime:
quantileOverTimeSharding = true
mapper.quantileOverTimeSharding = true
case ShardLastOverTime:
lastOverTimeSharding = true
mapper.lastOverTimeSharding = true
case ShardFirstOverTime:
firstOverTimeSharding = true
mapper.firstOverTimeSharding = true
case SupportApproxTopk:
mapper.approxTopkSupport = true
}
}
return ShardMapper{
shards: strategy,
metrics: metrics,
quantileOverTimeSharding: quantileOverTimeSharding,
firstOverTimeSharding: firstOverTimeSharding,
lastOverTimeSharding: lastOverTimeSharding,
}
return mapper
}
func NewShardMapperMetrics(registerer prometheus.Registerer) *MapperMetrics {
@ -185,7 +189,6 @@ func (m ShardMapper) mapLogSelectorExpr(expr syntax.LogSelectorExpr, r *downstre
func (m ShardMapper) mapSampleExpr(expr syntax.SampleExpr, r *downstreamRecorder) (syntax.SampleExpr, uint64, error) {
var head *ConcatSampleExpr
shards, maxBytesPerShard, err := m.shards.Shards(expr)
if err != nil {
return nil, 0, err
}
@ -233,7 +236,6 @@ func (m ShardMapper) wrappedShardedVectorAggr(expr *syntax.VectorAggregationExpr
// in descendent nodes in the AST. This optimization is currently avoided for simplicity.
func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr, r *downstreamRecorder, topLevel bool) (syntax.SampleExpr, uint64, error) {
if expr.Shardable(topLevel) {
switch expr.Operation {
case syntax.OpTypeSum:
@ -286,6 +288,71 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr
Grouping: expr.Grouping,
Operation: syntax.OpTypeSum,
}, bytesPerShard, nil
case syntax.OpTypeApproxTopK:
if !m.approxTopkSupport {
return nil, 0, fmt.Errorf("approx_topk is not enabled. See -limits.shard_aggregations")
}
// TODO(owen-d): integrate bounded sharding with approx_topk
// I'm not doing this now because it uses a separate code path and may not handle
// bounded shards in the same way
shards, bytesPerShard, err := m.shards.Resolver().Shards(expr)
if err != nil {
return nil, 0, err
}
// approx_topk(k, inner) ->
// topk(
// k,
// eval_cms(
// __count_min_sketch__(inner, shard=1) ++ __count_min_sketch__(inner, shard=2)...
// )
// )
countMinSketchExpr := syntax.MustClone(expr)
countMinSketchExpr.Operation = syntax.OpTypeCountMinSketch
countMinSketchExpr.Params = 0
// Even if this query is not sharded the user wants an approximation. This is helpful if some
// inferred label has a very high cardinality. Note that the querier does not support CountMinSketchEvalExpr
// which is why it's evaluated on the front end.
if shards == 0 {
return &syntax.VectorAggregationExpr{
Left: &CountMinSketchEvalExpr{
downstreams: []DownstreamSampleExpr{{
SampleExpr: countMinSketchExpr,
}},
},
Grouping: expr.Grouping,
Operation: syntax.OpTypeTopK,
Params: expr.Params,
}, bytesPerShard, nil
}
downstreams := make([]DownstreamSampleExpr, 0, shards)
for shard := 0; shard < shards; shard++ {
s := NewPowerOfTwoShard(index.ShardAnnotation{
Shard: uint32(shard),
Of: uint32(shards),
})
downstreams = append(downstreams, DownstreamSampleExpr{
shard: &ShardWithChunkRefs{
Shard: s,
},
SampleExpr: countMinSketchExpr,
})
}
sharded := &CountMinSketchEvalExpr{
downstreams: downstreams,
}
return &syntax.VectorAggregationExpr{
Left: sharded,
Grouping: expr.Grouping,
Operation: syntax.OpTypeTopK,
Params: expr.Params,
}, bytesPerShard, nil
default:
// this should not be reachable. If an operation is shardable it should
// have an optimization listed. Nonetheless, we log this as a warning
@ -300,7 +367,6 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr
}
return expr, exprStats.Bytes, nil
}
}
// if this AST contains unshardable operations, don't shard this at this level,
@ -320,7 +386,6 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr
Params: expr.Params,
Operation: expr.Operation,
}, bytesPerShard, nil
}
func (m ShardMapper) mapLabelReplaceExpr(expr *syntax.LabelReplaceExpr, r *downstreamRecorder, topLevel bool) (syntax.SampleExpr, uint64, error) {

@ -52,7 +52,6 @@ func TestShardedStringer(t *testing.T) {
}
func TestMapSampleExpr(t *testing.T) {
strategy := NewPowerOfTwoStrategy(ConstantShards(2))
m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime})
@ -117,7 +116,7 @@ func TestMapSampleExpr(t *testing.T) {
func TestMappingStrings(t *testing.T) {
strategy := NewPowerOfTwoStrategy(ConstantShards(2))
m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime})
m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk})
for _, tc := range []struct {
in string
out string
@ -168,6 +167,15 @@ func TestMappingStrings(t *testing.T) {
++ downstream<rate({foo="bar"}[5m]), shard=1_of_2>
)`,
},
{
in: `approx_topk(3, sum by(ip)(rate({foo="bar"}[5m])))`,
out: `topk(3,
CountMinSketchEval<
downstream<__count_min_sketch__(sum by(ip)(rate({foo="bar"}[5m]))), shard=0_of_2>
++ downstream<__count_min_sketch__(sum by(ip)(rate({foo="bar"}[5m]))), shard=1_of_2>
>
)`,
},
{
in: `sum(max(rate({foo="bar"}[5m])))`,
out: `sum(max(
@ -500,7 +508,6 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) {
},
} {
t.Run(tc.in, func(t *testing.T) {
shardedMapper := NewShardMapper(NewPowerOfTwoStrategy(ConstantShards(2)), nilShardMetrics, []string{ShardQuantileOverTime})
ast, err := syntax.ParseExpr(tc.in)
@ -526,7 +533,7 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) {
func TestMapping(t *testing.T) {
strategy := NewPowerOfTwoStrategy(ConstantShards(2))
m := NewShardMapper(strategy, nilShardMetrics, []string{})
m := NewShardMapper(strategy, nilShardMetrics, []string{SupportApproxTopk})
for _, tc := range []struct {
in string
@ -776,6 +783,56 @@ func TestMapping(t *testing.T) {
},
},
},
{
in: `approx_topk(3, rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Params: 3,
Operation: syntax.OpTypeTopK,
Left: &CountMinSketchEvalExpr{
downstreams: []DownstreamSampleExpr{
{
shard: NewPowerOfTwoShard(index.ShardAnnotation{
Shard: 0,
Of: 2,
}).Bind(nil),
SampleExpr: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeCountMinSketch,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
Grouping: &syntax.Grouping{},
},
},
{
shard: NewPowerOfTwoShard(index.ShardAnnotation{
Shard: 1,
Of: 2,
}).Bind(nil),
SampleExpr: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeCountMinSketch,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
Grouping: &syntax.Grouping{},
},
},
},
},
},
},
{
in: `count(rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{

@ -3,44 +3,58 @@ package sketch
import (
"fmt"
"math"
"github.com/axiomhq/hyperloglog"
)
type CountMinSketch struct {
depth, width uint32
counters [][]uint32
Depth, Width uint32
Counters [][]float64
HyperLogLog *hyperloglog.Sketch //hyperloglog.New16(),
}
// NewCountMinSketch creates a new CMS for a given width and depth.
func NewCountMinSketch(w, d uint32) (*CountMinSketch, error) {
return &CountMinSketch{
depth: d,
width: w,
counters: make2dslice(w, d),
Depth: d,
Width: w,
Counters: make2dslice(w, d),
HyperLogLog: hyperloglog.New16(),
}, nil
}
func make2dslice(col, row uint32) [][]uint32 {
ret := make([][]uint32, row)
// NewCountMinSketchFromErrorAndProbability creates a new CMS for a given epsilon and delta. The sketch width and depth
// are calculated according to the RedisBloom implementation.
// See https://github.com/RedisBloom/RedisBloom/blob/7bc047d1ea4113419b60eb6446ac3d4e61877a7b/src/cms.c#L38-L39
func NewCountMinSketchFromErrorAndProbability(epsilon float64, delta float64) (*CountMinSketch, error) {
width := math.Ceil(math.E / epsilon)
depth := math.Ceil(math.Log(delta) / math.Log(0.5))
return NewCountMinSketch(uint32(width), uint32(depth))
}
func make2dslice(col, row uint32) [][]float64 {
ret := make([][]float64, row)
for i := range ret {
ret[i] = make([]uint32, col)
ret[i] = make([]float64, col)
}
return ret
}
func (s *CountMinSketch) getPos(h1, h2, row uint32) uint32 {
pos := (h1 + row*h2) % s.width
pos := (h1 + row*h2) % s.Width
return pos
}
// Add 'count' occurrences of the given input.
func (s *CountMinSketch) Add(event string, count int) {
func (s *CountMinSketch) Add(event string, count float64) {
s.HyperLogLog.Insert(unsafeGetBytes(event))
// see the comments in the hashn function for how using only 2
// hash functions rather than a function per row still fullfils
// the pairwise indendent hash functions requirement for CMS
h1, h2 := hashn(event)
for i := uint32(0); i < s.depth; i++ {
for i := uint32(0); i < s.Depth; i++ {
pos := s.getPos(h1, h2, i)
s.counters[i][pos] += uint32(count)
s.Counters[i][pos] += count
}
}
@ -55,43 +69,45 @@ func (s *CountMinSketch) Increment(event string) {
// value that's less than Count(h) + count rather than all counters that h hashed to.
// Returns the new estimate for the event as well as the both hashes which can be used
// to identify the event for other things that need a hash.
func (s *CountMinSketch) ConservativeAdd(event string, count uint32) (uint32, uint32, uint32) {
min := uint32(math.MaxUint32)
func (s *CountMinSketch) ConservativeAdd(event string, count float64) (float64, uint32, uint32) {
s.HyperLogLog.Insert(unsafeGetBytes(event))
min := float64(math.MaxUint64)
h1, h2 := hashn(event)
// inline Count to save time/memory
var pos uint32
for i := uint32(0); i < s.depth; i++ {
for i := uint32(0); i < s.Depth; i++ {
pos = s.getPos(h1, h2, i)
if s.counters[i][pos] < min {
min = s.counters[i][pos]
if s.Counters[i][pos] < min {
min = s.Counters[i][pos]
}
}
min += count
for i := uint32(0); i < s.depth; i++ {
for i := uint32(0); i < s.Depth; i++ {
pos = s.getPos(h1, h2, i)
v := s.counters[i][pos]
v := s.Counters[i][pos]
if v < min {
s.counters[i][pos] = min
s.Counters[i][pos] = min
}
}
return min, h1, h2
}
func (s *CountMinSketch) ConservativeIncrement(event string) (uint32, uint32, uint32) {
return s.ConservativeAdd(event, 1)
func (s *CountMinSketch) ConservativeIncrement(event string) (float64, uint32, uint32) {
return s.ConservativeAdd(event, float64(1))
}
// Count returns the approximate min count for the given input.
func (s *CountMinSketch) Count(event string) uint32 {
min := uint32(math.MaxUint32)
func (s *CountMinSketch) Count(event string) float64 {
min := float64(math.MaxUint64)
h1, h2 := hashn(event)
var pos uint32
for i := uint32(0); i < s.depth; i++ {
for i := uint32(0); i < s.Depth; i++ {
pos = s.getPos(h1, h2, i)
if s.counters[i][pos] < min {
min = s.counters[i][pos]
if s.Counters[i][pos] < min {
min = s.Counters[i][pos]
}
}
return min
@ -100,14 +116,23 @@ func (s *CountMinSketch) Count(event string) uint32 {
// Merge the given sketch into this one.
// The sketches must have the same dimensions.
func (s *CountMinSketch) Merge(from *CountMinSketch) error {
if s.depth != from.depth || s.width != from.width {
if s.Depth != from.Depth || s.Width != from.Width {
return fmt.Errorf("Can't merge different sketches with different dimensions")
}
for i, l := range from.counters {
for i, l := range from.Counters {
for j, v := range l {
s.counters[i][j] += v
s.Counters[i][j] += v
}
}
// merge the cardinality sketches
s.HyperLogLog.Merge(from.HyperLogLog)
return nil
}
// Cardinality returns the estimated cardinality of the input to the CMS.
func (s *CountMinSketch) Cardinality() uint64 {
return s.HyperLogLog.Estimate()
}

@ -6,7 +6,7 @@ import (
type node struct {
event string
count uint32
count float64
// used for the container heap Fix function
index uint16
sketchPositions []uint32
@ -50,7 +50,7 @@ func (h *MinHeap) Peek() interface{} {
}
// update modifies the count and value of an Item in the queue.
func (h *MinHeap) update(event string, count uint32) {
func (h *MinHeap) update(event string, count float64) {
updateNode := -1
for i, k := range *h {
if k.event == event {

@ -12,21 +12,21 @@ func TestHeap(t *testing.T) {
heap.Init(&h)
heap.Push(&h, &node{event: "1", count: 70})
assert.Equal(t, uint32(70), h.Peek().(*node).count, "expected: %d and got %d", uint32(70), h.Peek().(*node).count)
heap.Push(&h, &node{event: "1", count: 70.0})
assert.Equal(t, 70.0, h.Peek().(*node).count, "expected: %f and got %f", 70.0, h.Peek().(*node).count)
heap.Push(&h, &node{event: "2", count: 20})
assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count)
heap.Push(&h, &node{event: "2", count: 20.0})
assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20, h.Peek().(*node).count)
heap.Push(&h, &node{event: "3", count: 50})
assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count)
assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20.0, h.Peek().(*node).count)
heap.Push(&h, &node{event: "4", count: 60})
assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count)
heap.Push(&h, &node{event: "4", count: 60.0})
assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20.0, h.Peek().(*node).count)
heap.Push(&h, &node{event: "5", count: 10})
assert.Equal(t, uint32(10), h.Peek().(*node).count, "expected: %d and got %d", uint32(10), h.Peek().(*node).count)
heap.Push(&h, &node{event: "5", count: 10.0})
assert.Equal(t, 10.0, h.Peek().(*node).count, "expected: %f and got %f", 10.0, h.Peek().(*node).count)
assert.Equal(t, heap.Pop(&h).(*node).count, uint32(10))
assert.Equal(t, h.Peek().(*node).count, uint32(20))
assert.Equal(t, heap.Pop(&h).(*node).count, 10.0)
assert.Equal(t, h.Peek().(*node).count, 20.0)
}

@ -36,7 +36,7 @@ func TestTopKMatrixProto(t *testing.T) {
require.NoError(t, err)
require.Len(t, deserialized, 1)
require.Equal(t, original.sketch.counters, deserialized[0].topk.sketch.counters)
require.Equal(t, original.sketch.Counters, deserialized[0].topk.sketch.Counters)
require.Equal(t, *original.hll, *deserialized[0].topk.hll)
oCardinality, _ := original.Cardinality()
fmt.Println("ocardinality: ", oCardinality)
@ -74,7 +74,7 @@ func TestTopKMatrixProtoMerge(t *testing.T) {
require.NoError(t, err)
require.Len(t, deserialized, 1)
require.Equal(t, original.sketch.counters, deserialized[0].topk.sketch.counters)
require.Equal(t, original.sketch.Counters, deserialized[0].topk.sketch.Counters)
require.Equal(t, *original.hll, *deserialized[0].topk.hll)
oCardinality, _ := original.Cardinality()
dCardinality, _ := deserialized[0].topk.Cardinality()

@ -14,7 +14,7 @@ import (
type element struct {
Event string
Count int64
Count float64
}
type TopKResult []element
@ -108,13 +108,13 @@ func newCMSTopK(k int, w, d uint32) (*Topk, error) {
func TopkFromProto(t *logproto.TopK) (*Topk, error) {
cms := &CountMinSketch{
depth: t.Cms.Depth,
width: t.Cms.Width,
Depth: t.Cms.Depth,
Width: t.Cms.Width,
}
for row := uint32(0); row < cms.depth; row++ {
s := row * cms.width
e := s + cms.width
cms.counters = append(cms.counters, t.Cms.Counters[s:e])
for row := uint32(0); row < cms.Depth; row++ {
s := row * cms.Width
e := s + cms.Width
cms.Counters = append(cms.Counters, t.Cms.Counters[s:e])
}
hll := hyperloglog.New()
@ -143,12 +143,12 @@ func TopkFromProto(t *logproto.TopK) (*Topk, error) {
func (t *Topk) ToProto() (*logproto.TopK, error) {
cms := &logproto.CountMinSketch{
Depth: t.sketch.depth,
Width: t.sketch.width,
Depth: t.sketch.Depth,
Width: t.sketch.Width,
}
cms.Counters = make([]uint32, 0, cms.Depth*cms.Width)
cms.Counters = make([]float64, 0, cms.Depth*cms.Width)
for row := uint32(0); row < cms.Depth; row++ {
cms.Counters = append(cms.Counters, t.sketch.counters[row]...)
cms.Counters = append(cms.Counters, t.sketch.Counters[row]...)
}
hllBytes, err := t.hll.MarshalBinary()
@ -175,7 +175,7 @@ func (t *Topk) ToProto() (*logproto.TopK, error) {
// wrapper to bundle together updating of the bf portion of the sketch and pushing of a new element
// to the heap
func (t *Topk) heapPush(h *MinHeap, event string, estimate, h1, h2 uint32) {
func (t *Topk) heapPush(h *MinHeap, event string, estimate float64, h1, h2 uint32) {
var pos uint32
for i := range t.bf {
pos = t.sketch.getPos(h1, h2, uint32(i))
@ -186,7 +186,7 @@ func (t *Topk) heapPush(h *MinHeap, event string, estimate, h1, h2 uint32) {
// wrapper to bundle together updating of the bf portion of the sketch for the removed and added event
// as well as replacing the min heap element with the new event and it's count
func (t *Topk) heapMinReplace(event string, estimate uint32, removed string) {
func (t *Topk) heapMinReplace(event string, estimate float64, removed string) {
t.updateBF(removed, event)
(*t.heap)[0].event = event
(*t.heap)[0].count = estimate
@ -269,7 +269,7 @@ func (t *Topk) Observe(event string) {
if estimate > t.heap.Peek().(*node).count {
if len(*t.heap) == t.max {
e := t.heap.Peek().(*node).event
//r1, r2 := hashn(e)
// r1, r2 := hashn(e)
t.heapMinReplace(event, estimate, e)
return
}
@ -304,11 +304,11 @@ func (t *Topk) Merge(from *Topk) error {
var all TopKResult
for _, e := range *t.heap {
all = append(all, element{Event: e.event, Count: int64(t.sketch.Count(e.event))})
all = append(all, element{Event: e.event, Count: t.sketch.Count(e.event)})
}
for _, e := range *from.heap {
all = append(all, element{Event: e.event, Count: int64(t.sketch.Count(e.event))})
all = append(all, element{Event: e.event, Count: t.sketch.Count(e.event)})
}
all = removeDuplicates(all)
@ -318,7 +318,7 @@ func (t *Topk) Merge(from *Topk) error {
// TODO: merging should also potentially replace it's bloomfilter? or 0 everything in the bloomfilter
for _, e := range all[:t.max] {
h1, h2 = hashn(e.Event)
t.heapPush(temp, e.Event, uint32(e.Count), h1, h2)
t.heapPush(temp, e.Event, float64(e.Count), h1, h2)
}
t.heap = temp
@ -347,7 +347,7 @@ func (t *Topk) Topk() TopKResult {
for _, e := range *t.heap {
res = append(res, element{
Event: e.event,
Count: int64(t.sketch.Count(e.event)),
Count: t.sketch.Count(e.event),
})
}
sort.Sort(res)

@ -131,7 +131,7 @@ outer2:
require.LessOrEqualf(t, singleMissing, 2, "more than acceptable misses: %d > %d", singleMissing, 2)
// this condition is never actually true
//require.LessOrEqualf(t, mergedMissing, singleMissing, "merged sketch should be at least as accurate as a single sketch")
// require.LessOrEqualf(t, mergedMissing, singleMissing, "merged sketch should be at least as accurate as a single sketch")
}
// compare the accuracy of cms topk and hk to the real topk
@ -142,7 +142,7 @@ func TestRealTopK(t *testing.T) {
defer f.Close()
scanner := bufio.NewScanner(f)
m := make(map[string]uint32)
m := make(map[string]float64)
h := MinHeap{}
hll := hyperloglog.New16()
@ -170,7 +170,7 @@ func TestRealTopK(t *testing.T) {
res := make(TopKResult, 0, len(h))
for i := 0; i < len(h); i++ {
res = append(res, element{h[i].event, int64(h[i].count)})
res = append(res, element{h[i].event, h[i].count})
}
sort.Sort(res)
@ -219,7 +219,7 @@ func TestRealTop_Merge(t *testing.T) {
scanner := bufio.NewScanner(combined)
m := make(map[string]uint32)
m := make(map[string]float64)
h := MinHeap{}
hll := hyperloglog.New16()
// HK gets more inaccurate with merging the more shards we have
@ -251,7 +251,7 @@ func TestRealTop_Merge(t *testing.T) {
res := make(TopKResult, 0, len(h))
for i := 0; i < len(h); i++ {
res = append(res, element{h[i].event, int64(h[i].count)})
res = append(res, element{h[i].event, h[i].count})
}
sort.Sort(res)
@ -265,7 +265,7 @@ func TestRealTop_Merge(t *testing.T) {
scanner = bufio.NewScanner(combined)
scanner.Split(bufio.ScanWords)
var cms = make([]*Topk, shards)
cms := make([]*Topk, shards)
for i := range cms {
cms[i], _ = newCMSTopK(k, 2048, 5)
}

@ -7,6 +7,7 @@ import (
type StepResult interface {
SampleVector() promql.Vector
QuantileSketchVec() ProbabilisticQuantileVector
CountMinSketchVec() CountMinSketchVector
}
type SampleVector promql.Vector
@ -21,6 +22,10 @@ func (p SampleVector) QuantileSketchVec() ProbabilisticQuantileVector {
return ProbabilisticQuantileVector{}
}
func (SampleVector) CountMinSketchVec() CountMinSketchVector {
return CountMinSketchVector{}
}
// StepEvaluator evaluate a single step of a query.
type StepEvaluator interface {
// while Next returns a promql.Value, the only acceptable types are Scalar and Vector.

@ -47,7 +47,7 @@ func Clone[T Expr](e T) (T, error) {
}
func MustClone[T Expr](e T) T {
copied, err := Clone[T](e)
copied, err := Clone(e)
if err != nil {
panic(err)
}
@ -800,6 +800,7 @@ func (e *DecolorizeExpr) Shardable(_ bool) bool { return true }
func (e *DecolorizeExpr) Stage() (log.Stage, error) {
return log.NewDecolorizer()
}
func (e *DecolorizeExpr) String() string {
return fmt.Sprintf("%s %s", OpPipe, OpDecolorize)
}
@ -823,6 +824,7 @@ func (e *DropLabelsExpr) Shardable(_ bool) bool { return true }
func (e *DropLabelsExpr) Stage() (log.Stage, error) {
return log.NewDropLabels(e.dropLabels), nil
}
func (e *DropLabelsExpr) String() string {
var sb strings.Builder
@ -1234,7 +1236,7 @@ const (
OpRangeTypeLast = "last_over_time"
OpRangeTypeAbsent = "absent_over_time"
//vector
// vector
OpTypeVector = "vector"
// binops - logical/set
@ -1305,6 +1307,11 @@ const (
OpRangeTypeQuantileSketch = "__quantile_sketch_over_time__"
OpRangeTypeFirstWithTimestamp = "__first_over_time_ts__"
OpRangeTypeLastWithTimestamp = "__last_over_time_ts__"
OpTypeCountMinSketch = "__count_min_sketch__"
// probabilistic aggregations
OpTypeApproxTopK = "approx_topk"
)
func IsComparisonOperator(op string) bool {
@ -1533,7 +1540,7 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin
var p int
var err error
switch operation {
case OpTypeBottomK, OpTypeTopK:
case OpTypeBottomK, OpTypeTopK, OpTypeApproxTopK:
if params == nil {
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0)}
}
@ -1544,6 +1551,9 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin
if p <= 0 {
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("invalid parameter (must be greater than 0) %s(%s", operation, *params), 0, 0)}
}
if operation == OpTypeApproxTopK && gr != nil {
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("grouping not allowed for %s aggregation", operation), 0, 0)}
}
default:
if params != nil {
@ -1609,7 +1619,7 @@ func (e *VectorAggregationExpr) String() string {
var params []string
switch e.Operation {
// bottomK and topk can have first parameter as 0
case OpTypeBottomK, OpTypeTopK:
case OpTypeBottomK, OpTypeTopK, OpTypeApproxTopK:
params = []string{fmt.Sprintf("%d", e.Params), e.Left.String()}
default:
if e.Params != 0 {
@ -2278,6 +2288,8 @@ var shardableOps = map[string]bool{
OpTypeMax: true,
OpTypeMin: true,
OpTypeApproxTopK: true,
// range vector ops
OpRangeTypeAvg: true,
OpRangeTypeCount: true,

@ -135,7 +135,8 @@ import (
%token <str> IDENTIFIER STRING NUMBER PARSER_FLAG
%token <duration> DURATION RANGE
%token <val> MATCHERS LABELS EQ RE NRE NPA OPEN_BRACE CLOSE_BRACE OPEN_BRACKET CLOSE_BRACKET COMMA DOT PIPE_MATCH PIPE_EXACT PIPE_PATTERN
OPEN_PARENTHESIS CLOSE_PARENTHESIS BY WITHOUT COUNT_OVER_TIME RATE RATE_COUNTER SUM SORT SORT_DESC AVG MAX MIN COUNT STDDEV STDVAR BOTTOMK TOPK
OPEN_PARENTHESIS CLOSE_PARENTHESIS BY WITHOUT COUNT_OVER_TIME RATE RATE_COUNTER SUM SORT SORT_DESC AVG
MAX MIN COUNT STDDEV STDVAR BOTTOMK TOPK APPROX_TOPK
BYTES_OVER_TIME BYTES_RATE BOOL JSON REGEXP LOGFMT PIPE LINE_FMT LABEL_FMT UNWRAP AVG_OVER_TIME SUM_OVER_TIME MIN_OVER_TIME
MAX_OVER_TIME STDVAR_OVER_TIME STDDEV_OVER_TIME QUANTILE_OVER_TIME BYTES_CONV DURATION_CONV DURATION_SECONDS_CONV
FIRST_OVER_TIME LAST_OVER_TIME ABSENT_OVER_TIME VECTOR LABEL_REPLACE UNPACK OFFSET PATTERN IP ON IGNORING GROUP_LEFT GROUP_RIGHT
@ -545,6 +546,7 @@ vectorOp:
| TOPK { $$ = OpTypeTopK }
| SORT { $$ = OpTypeSort }
| SORT_DESC { $$ = OpTypeSortDesc }
| APPROX_TOPK { $$ = OpTypeApproxTopK }
;
rangeOp:

File diff suppressed because it is too large Load Diff

@ -121,6 +121,8 @@ var functionTokens = map[string]int{
OpTypeSortDesc: SORT_DESC,
OpLabelReplace: LABEL_REPLACE,
OpTypeApproxTopK: APPROX_TOPK,
// conversion Op
OpConvBytes: BYTES_CONV,
OpConvDuration: DURATION_CONV,

@ -364,6 +364,10 @@ var ParseTestCases = []struct {
in: `absent_over_time({ foo = "bar" }[5h]) by (foo)`,
err: logqlmodel.NewParseError("grouping not allowed for absent_over_time aggregation", 0, 0),
},
{
in: `approx_topk(2, count_over_time({ foo = "bar" }[5h])) by (foo)`,
err: logqlmodel.NewParseError("grouping not allowed for approx_topk aggregation", 0, 0),
},
{
in: `rate({ foo = "bar" }[5minutes])`,
err: logqlmodel.NewParseError(`unknown unit "minutes" in duration "5minutes"`, 0, 21),

@ -7,10 +7,12 @@ import (
"math/rand"
"sort"
"strings"
"sync"
"time"
"unicode"
"github.com/cespare/xxhash/v2"
"github.com/grafana/dskit/concurrency"
"github.com/prometheus/prometheus/model/labels"
promql_parser "github.com/prometheus/prometheus/promql/parser"
@ -21,6 +23,8 @@ import (
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
)
const ConCurrency = 100
func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier {
return MockQuerier{
shards: shards,
@ -45,7 +49,6 @@ func (q MockQuerier) extractOldShard(xs []string) (*index.ShardAnnotation, error
}
return parsed[0].PowerOfTwo, nil
}
func (q MockQuerier) SelectLogs(_ context.Context, req SelectLogParams) (iter.EntryIterator, error) {
@ -113,8 +116,8 @@ func processStream(in []logproto.Stream, pipeline log.Pipeline) []logproto.Strea
resByStream := map[string]*logproto.Stream{}
for _, stream := range in {
sp := pipeline.ForStream(mustParseLabels(stream.Labels))
for _, e := range stream.Entries {
sp := pipeline.ForStream(mustParseLabels(stream.Labels))
if l, out, matches := sp.Process(e.Timestamp.UnixNano(), []byte(e.Line)); matches {
var s *logproto.Stream
var found bool
@ -137,12 +140,12 @@ func processStream(in []logproto.Stream, pipeline log.Pipeline) []logproto.Strea
return streams
}
func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Series {
func processSeries(in []logproto.Stream, ex log.SampleExtractor) ([]logproto.Series, error) {
resBySeries := map[string]*logproto.Series{}
for _, stream := range in {
exs := ex.ForStream(mustParseLabels(stream.Labels))
for _, e := range stream.Entries {
exs := ex.ForStream(mustParseLabels(stream.Labels))
if f, lbs, ok := exs.Process(e.Timestamp.UnixNano(), []byte(e.Line)); ok {
var s *logproto.Series
var found bool
@ -151,6 +154,7 @@ func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Seri
s = &logproto.Series{Labels: lbs.String(), StreamHash: exs.BaseLabels().Hash()}
resBySeries[lbs.String()] = s
}
s.Samples = append(s.Samples, logproto.Sample{
Timestamp: e.Timestamp.UnixNano(),
Value: f,
@ -159,12 +163,13 @@ func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Seri
}
}
}
series := []logproto.Series{}
for _, s := range resBySeries {
sort.Sort(s)
series = append(series, *s)
}
return series
return series, nil
}
func (q MockQuerier) SelectSamples(_ context.Context, req SelectSampleParams) (iter.SampleIterator, error) {
@ -212,7 +217,10 @@ outer:
matched = append(matched, stream)
}
filtered := processSeries(matched, extractor)
filtered, err := processSeries(matched, extractor)
if err != nil {
return nil, err
}
return iter.NewTimeRangedSampleIterator(
iter.NewMultiSeriesIterator(filtered),
@ -227,26 +235,23 @@ type MockDownstreamer struct {
func (m MockDownstreamer) Downstreamer(_ context.Context) Downstreamer { return m }
func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery, _ Accumulator) ([]logqlmodel.Result, error) {
results := make([]logqlmodel.Result, 0, len(queries))
for _, query := range queries {
res, err := m.Query(query.Params).Exec(ctx)
func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery, acc Accumulator) ([]logqlmodel.Result, error) {
mu := sync.Mutex{}
err := concurrency.ForEachJob(ctx, len(queries), ConCurrency, func(ctx context.Context, idx int) error {
res, err := m.Query(queries[idx].Params).Exec(ctx)
if err != nil {
return nil, err
return err
}
results = append(results, res)
mu.Lock()
defer mu.Unlock()
err = acc.Accumulate(ctx, res, idx)
return err
})
if err != nil {
return nil, err
}
if matrix, ok := results[0].Data.(ProbabilisticQuantileMatrix); ok {
if len(results) == 1 {
return results, nil
}
for _, m := range results[1:] {
matrix, _ = matrix.Merge(m.Data.(ProbabilisticQuantileMatrix))
}
return []logqlmodel.Result{{Data: matrix}}, nil
}
return results, nil
return acc.Result(), nil
}
// create nStreams of nEntries with labelNames each where each label value

@ -221,6 +221,23 @@ func (m *QuantileSketchResponse) WithHeaders(h []queryrangebase.PrometheusRespon
return m
}
// GetHeaders returns the HTTP headers in the response.
func (m *CountMinSketchResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader {
if m != nil {
return convertPrometheusResponseHeadersToPointers(m.Headers)
}
return nil
}
func (m *CountMinSketchResponse) SetHeader(name, value string) {
m.Headers = setHeader(m.Headers, name, value)
}
func (m *CountMinSketchResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response {
m.Headers = h
return m
}
func (m *ShardsResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader {
if m != nil {
return convertPrometheusResponseHeadersToPointers(m.Headers)

@ -36,4 +36,6 @@ type Limits interface {
MaxStatsCacheFreshness(context.Context, string) time.Duration
MaxMetadataCacheFreshness(context.Context, string) time.Duration
VolumeEnabled(string) bool
ShardAggregations(string) []string
}

@ -136,6 +136,13 @@ func ResultToResponse(result logqlmodel.Result, params logql.Params) (queryrange
Warnings: result.Warnings,
Statistics: result.Statistics,
}, nil
case logql.CountMinSketchVector:
r, err := data.ToProto()
return &CountMinSketchResponse{
Response: r,
Warnings: result.Warnings,
Statistics: result.Statistics,
}, err
}
return nil, fmt.Errorf("unsupported data type: %T", result.Data)
@ -202,6 +209,17 @@ func ResponseToResult(resp queryrangebase.Response) (logqlmodel.Result, error) {
Warnings: r.Warnings,
Statistics: r.Statistics,
}, nil
case *CountMinSketchResponse:
cms, err := logql.CountMinSketchVectorFromProto(r.Response)
if err != nil {
return logqlmodel.Result{}, fmt.Errorf("cannot decode count min sketch vector: %w", err)
}
return logqlmodel.Result{
Data: cms,
Headers: resp.GetHeaders(),
Warnings: r.Warnings,
Statistics: r.Statistics,
}, nil
default:
return logqlmodel.Result{}, fmt.Errorf("cannot decode (%T)", resp)
}
@ -237,6 +255,8 @@ func QueryResponseUnwrap(res *QueryResponse) (queryrangebase.Response, error) {
return concrete.DetectedLabels, nil
case *QueryResponse_DetectedFields:
return concrete.DetectedFields, nil
case *QueryResponse_CountMinSketches:
return concrete.CountMinSketches, nil
default:
return nil, fmt.Errorf("unsupported QueryResponse response type, got (%T)", res.Response)
}
@ -278,6 +298,8 @@ func QueryResponseWrap(res queryrangebase.Response) (*QueryResponse, error) {
p.Response = &QueryResponse_DetectedLabels{response}
case *DetectedFieldsResponse:
p.Response = &QueryResponse_DetectedFields{response}
case *CountMinSketchResponse:
p.Response = &QueryResponse_CountMinSketches{response}
default:
return nil, fmt.Errorf("invalid response format, got (%T)", res)
}

@ -908,6 +908,59 @@ func (m *QuantileSketchResponse) GetStatistics() stats.Result {
return stats.Result{}
}
type CountMinSketchResponse struct {
Response *github_com_grafana_loki_v3_pkg_logproto.CountMinSketchVector `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/v3/pkg/logproto.CountMinSketchVector" json:"response,omitempty"`
Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"`
Warnings []string `protobuf:"bytes,3,rep,name=warnings,proto3" json:"warnings,omitempty"`
Statistics stats.Result `protobuf:"bytes,4,opt,name=statistics,proto3" json:"statistics"`
}
func (m *CountMinSketchResponse) Reset() { *m = CountMinSketchResponse{} }
func (*CountMinSketchResponse) ProtoMessage() {}
func (*CountMinSketchResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{13}
}
func (m *CountMinSketchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *CountMinSketchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_CountMinSketchResponse.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *CountMinSketchResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_CountMinSketchResponse.Merge(m, src)
}
func (m *CountMinSketchResponse) XXX_Size() int {
return m.Size()
}
func (m *CountMinSketchResponse) XXX_DiscardUnknown() {
xxx_messageInfo_CountMinSketchResponse.DiscardUnknown(m)
}
var xxx_messageInfo_CountMinSketchResponse proto.InternalMessageInfo
func (m *CountMinSketchResponse) GetWarnings() []string {
if m != nil {
return m.Warnings
}
return nil
}
func (m *CountMinSketchResponse) GetStatistics() stats.Result {
if m != nil {
return m.Statistics
}
return stats.Result{}
}
type ShardsResponse struct {
Response *github_com_grafana_loki_v3_pkg_logproto.ShardsResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/v3/pkg/logproto.ShardsResponse" json:"response,omitempty"`
Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"`
@ -916,7 +969,7 @@ type ShardsResponse struct {
func (m *ShardsResponse) Reset() { *m = ShardsResponse{} }
func (*ShardsResponse) ProtoMessage() {}
func (*ShardsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{13}
return fileDescriptor_51b9d53b40d11902, []int{14}
}
func (m *ShardsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -953,7 +1006,7 @@ type DetectedFieldsResponse struct {
func (m *DetectedFieldsResponse) Reset() { *m = DetectedFieldsResponse{} }
func (*DetectedFieldsResponse) ProtoMessage() {}
func (*DetectedFieldsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{14}
return fileDescriptor_51b9d53b40d11902, []int{15}
}
func (m *DetectedFieldsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -990,7 +1043,7 @@ type QueryPatternsResponse struct {
func (m *QueryPatternsResponse) Reset() { *m = QueryPatternsResponse{} }
func (*QueryPatternsResponse) ProtoMessage() {}
func (*QueryPatternsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{15}
return fileDescriptor_51b9d53b40d11902, []int{16}
}
func (m *QueryPatternsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1027,7 +1080,7 @@ type DetectedLabelsResponse struct {
func (m *DetectedLabelsResponse) Reset() { *m = DetectedLabelsResponse{} }
func (*DetectedLabelsResponse) ProtoMessage() {}
func (*DetectedLabelsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{16}
return fileDescriptor_51b9d53b40d11902, []int{17}
}
func (m *DetectedLabelsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1071,13 +1124,14 @@ type QueryResponse struct {
// *QueryResponse_DetectedFields
// *QueryResponse_PatternsResponse
// *QueryResponse_DetectedLabels
// *QueryResponse_CountMinSketches
Response isQueryResponse_Response `protobuf_oneof:"response"`
}
func (m *QueryResponse) Reset() { *m = QueryResponse{} }
func (*QueryResponse) ProtoMessage() {}
func (*QueryResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{17}
return fileDescriptor_51b9d53b40d11902, []int{18}
}
func (m *QueryResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1149,6 +1203,9 @@ type QueryResponse_PatternsResponse struct {
type QueryResponse_DetectedLabels struct {
DetectedLabels *DetectedLabelsResponse `protobuf:"bytes,13,opt,name=detectedLabels,proto3,oneof"`
}
type QueryResponse_CountMinSketches struct {
CountMinSketches *CountMinSketchResponse `protobuf:"bytes,14,opt,name=countMinSketches,proto3,oneof"`
}
func (*QueryResponse_Series) isQueryResponse_Response() {}
func (*QueryResponse_Labels) isQueryResponse_Response() {}
@ -1162,6 +1219,7 @@ func (*QueryResponse_ShardsResponse) isQueryResponse_Response() {}
func (*QueryResponse_DetectedFields) isQueryResponse_Response() {}
func (*QueryResponse_PatternsResponse) isQueryResponse_Response() {}
func (*QueryResponse_DetectedLabels) isQueryResponse_Response() {}
func (*QueryResponse_CountMinSketches) isQueryResponse_Response() {}
func (m *QueryResponse) GetResponse() isQueryResponse_Response {
if m != nil {
@ -1261,6 +1319,13 @@ func (m *QueryResponse) GetDetectedLabels() *DetectedLabelsResponse {
return nil
}
func (m *QueryResponse) GetCountMinSketches() *CountMinSketchResponse {
if x, ok := m.GetResponse().(*QueryResponse_CountMinSketches); ok {
return x.CountMinSketches
}
return nil
}
// XXX_OneofWrappers is for the internal use of the proto package.
func (*QueryResponse) XXX_OneofWrappers() []interface{} {
return []interface{}{
@ -1276,6 +1341,7 @@ func (*QueryResponse) XXX_OneofWrappers() []interface{} {
(*QueryResponse_DetectedFields)(nil),
(*QueryResponse_PatternsResponse)(nil),
(*QueryResponse_DetectedLabels)(nil),
(*QueryResponse_CountMinSketches)(nil),
}
}
@ -1298,7 +1364,7 @@ type QueryRequest struct {
func (m *QueryRequest) Reset() { *m = QueryRequest{} }
func (*QueryRequest) ProtoMessage() {}
func (*QueryRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{18}
return fileDescriptor_51b9d53b40d11902, []int{19}
}
func (m *QueryRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1490,6 +1556,7 @@ func init() {
proto.RegisterType((*VolumeResponse)(nil), "queryrange.VolumeResponse")
proto.RegisterType((*TopKSketchesResponse)(nil), "queryrange.TopKSketchesResponse")
proto.RegisterType((*QuantileSketchResponse)(nil), "queryrange.QuantileSketchResponse")
proto.RegisterType((*CountMinSketchResponse)(nil), "queryrange.CountMinSketchResponse")
proto.RegisterType((*ShardsResponse)(nil), "queryrange.ShardsResponse")
proto.RegisterType((*DetectedFieldsResponse)(nil), "queryrange.DetectedFieldsResponse")
proto.RegisterType((*QueryPatternsResponse)(nil), "queryrange.QueryPatternsResponse")
@ -1504,129 +1571,132 @@ func init() {
}
var fileDescriptor_51b9d53b40d11902 = []byte{
// 1944 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcb, 0x6f, 0x1b, 0xc7,
0x19, 0xe7, 0xf2, 0x29, 0x7e, 0x7a, 0x58, 0x1d, 0xab, 0xca, 0x56, 0x71, 0xb8, 0x2c, 0x81, 0x26,
0x6a, 0xd1, 0x92, 0x31, 0x95, 0xb8, 0x89, 0xea, 0x1a, 0xf1, 0x5a, 0x76, 0x65, 0xd7, 0x69, 0x9c,
0x95, 0xd0, 0x43, 0x2f, 0xc5, 0x88, 0x1c, 0x91, 0x5b, 0x91, 0xbb, 0xeb, 0xdd, 0xa1, 0x6c, 0x01,
0x45, 0x91, 0x7f, 0x20, 0x68, 0xee, 0xbd, 0x17, 0xbd, 0x15, 0x05, 0x7a, 0xea, 0xa9, 0xc7, 0xe4,
0x50, 0xc0, 0xc7, 0x80, 0x40, 0xb7, 0x35, 0xdd, 0x43, 0xa1, 0x53, 0x80, 0xfe, 0x03, 0xc5, 0x3c,
0x76, 0x39, 0x43, 0x52, 0x35, 0xa5, 0x16, 0x05, 0x14, 0xe4, 0x22, 0xcd, 0xe3, 0xfb, 0xcd, 0xce,
0xfc, 0xbe, 0xdf, 0xf7, 0xcd, 0x83, 0xf0, 0x46, 0x70, 0xd4, 0x69, 0x3c, 0x1e, 0x90, 0xd0, 0x25,
0x21, 0xff, 0x7f, 0x12, 0x62, 0xaf, 0x43, 0x94, 0x62, 0x3d, 0x08, 0x7d, 0xea, 0x23, 0x18, 0xb7,
0x6c, 0x34, 0x3b, 0x2e, 0xed, 0x0e, 0x0e, 0xea, 0x2d, 0xbf, 0xdf, 0xe8, 0xf8, 0x1d, 0xbf, 0xd1,
0xf1, 0xfd, 0x4e, 0x8f, 0xe0, 0xc0, 0x8d, 0x64, 0xb1, 0x11, 0x06, 0xad, 0x46, 0x44, 0x31, 0x1d,
0x44, 0x02, 0xbf, 0xb1, 0xc6, 0x0c, 0x79, 0x91, 0x43, 0x64, 0xab, 0x25, 0xcd, 0x79, 0xed, 0x60,
0x70, 0xd8, 0xa0, 0x6e, 0x9f, 0x44, 0x14, 0xf7, 0x83, 0xc4, 0x80, 0xcd, 0xaf, 0xe7, 0x77, 0x04,
0xd2, 0xf5, 0xda, 0xe4, 0x69, 0x07, 0x53, 0xf2, 0x04, 0x9f, 0x48, 0x83, 0x57, 0x35, 0x83, 0xa4,
0x20, 0x3b, 0x37, 0xb4, 0xce, 0x00, 0x53, 0x4a, 0x42, 0x4f, 0xf6, 0x7d, 0x43, 0xeb, 0x8b, 0x8e,
0x08, 0x6d, 0x75, 0x65, 0x57, 0x55, 0x76, 0x3d, 0xee, 0xf5, 0xfd, 0x36, 0xe9, 0xf1, 0x85, 0x44,
0xe2, 0xaf, 0xb4, 0xb8, 0xca, 0x2c, 0x82, 0x41, 0xd4, 0xe5, 0x7f, 0x64, 0xe3, 0x9d, 0x97, 0x72,
0x79, 0x80, 0x23, 0xd2, 0x68, 0x93, 0x43, 0xd7, 0x73, 0xa9, 0xeb, 0x7b, 0x91, 0x5a, 0x96, 0x83,
0xdc, 0x98, 0x6f, 0x90, 0x49, 0xff, 0x6c, 0xbc, 0xc9, 0x70, 0x11, 0xf5, 0x43, 0xdc, 0x21, 0x8d,
0x56, 0x77, 0xe0, 0x1d, 0x35, 0x5a, 0xb8, 0xd5, 0x25, 0x8d, 0x90, 0x44, 0x83, 0x1e, 0x8d, 0x44,
0x85, 0x9e, 0x04, 0x44, 0x7e, 0xa9, 0xf6, 0x59, 0x1e, 0x16, 0x1f, 0xfa, 0x47, 0xae, 0x43, 0x1e,
0x0f, 0x48, 0x44, 0xd1, 0x1a, 0x14, 0xf8, 0xa8, 0xa6, 0x51, 0x35, 0x36, 0xcb, 0x8e, 0xa8, 0xb0,
0xd6, 0x9e, 0xdb, 0x77, 0xa9, 0x99, 0xad, 0x1a, 0x9b, 0xcb, 0x8e, 0xa8, 0x20, 0x04, 0xf9, 0x88,
0x92, 0xc0, 0xcc, 0x55, 0x8d, 0xcd, 0x9c, 0xc3, 0xcb, 0x68, 0x03, 0x16, 0x5c, 0x8f, 0x92, 0xf0,
0x18, 0xf7, 0xcc, 0x32, 0x6f, 0x4f, 0xeb, 0xe8, 0x16, 0x94, 0x22, 0x8a, 0x43, 0xba, 0x1f, 0x99,
0xf9, 0xaa, 0xb1, 0xb9, 0xd8, 0xdc, 0xa8, 0x0b, 0xcf, 0xd7, 0x13, 0xcf, 0xd7, 0xf7, 0x13, 0xcf,
0xdb, 0x0b, 0x9f, 0xc6, 0x56, 0xe6, 0x93, 0xbf, 0x59, 0x86, 0x93, 0x80, 0xd0, 0x36, 0x14, 0x88,
0xd7, 0xde, 0x8f, 0xcc, 0xc2, 0x39, 0xd0, 0x02, 0x82, 0xae, 0x43, 0xb9, 0xed, 0x86, 0xa4, 0xc5,
0x58, 0x36, 0x8b, 0x55, 0x63, 0x73, 0xa5, 0x79, 0xb5, 0x9e, 0x0a, 0x65, 0x27, 0xe9, 0x72, 0xc6,
0x56, 0x6c, 0x79, 0x01, 0xa6, 0x5d, 0xb3, 0xc4, 0x99, 0xe0, 0x65, 0x54, 0x83, 0x62, 0xd4, 0xc5,
0x61, 0x3b, 0x32, 0x17, 0xaa, 0xb9, 0xcd, 0xb2, 0x0d, 0xa7, 0xb1, 0x25, 0x5b, 0x1c, 0xf9, 0x1f,
0xfd, 0x1c, 0xf2, 0x41, 0x0f, 0x7b, 0x26, 0xf0, 0x59, 0xae, 0xd6, 0x15, 0x2f, 0x3d, 0xea, 0x61,
0xcf, 0x7e, 0x77, 0x18, 0x5b, 0x6f, 0xab, 0xc1, 0x13, 0xe2, 0x43, 0xec, 0xe1, 0x46, 0xcf, 0x3f,
0x72, 0x1b, 0xc7, 0x5b, 0x0d, 0xd5, 0xf7, 0x6c, 0xa0, 0xfa, 0x87, 0x6c, 0x00, 0x06, 0x75, 0xf8,
0xc0, 0xe8, 0x01, 0x2c, 0x32, 0x1f, 0x93, 0x3b, 0xcc, 0xc1, 0x91, 0xb9, 0xc8, 0xbf, 0xf3, 0xca,
0x78, 0x35, 0xbc, 0xdd, 0x21, 0x87, 0x3f, 0x0a, 0xfd, 0x41, 0x60, 0x5f, 0x39, 0x8d, 0x2d, 0xd5,
0xde, 0x51, 0x2b, 0xe8, 0x01, 0xac, 0x30, 0x51, 0xb8, 0x5e, 0xe7, 0x83, 0x80, 0x2b, 0xd0, 0x5c,
0xe2, 0xc3, 0x5d, 0xab, 0xab, 0x92, 0xa9, 0xdf, 0xd1, 0x6c, 0xec, 0x3c, 0xa3, 0xd7, 0x99, 0x40,
0xd6, 0x46, 0x39, 0x40, 0x4c, 0x4b, 0xf7, 0xbd, 0x88, 0x62, 0x8f, 0x5e, 0x44, 0x52, 0x37, 0xa1,
0xc8, 0x82, 0x7f, 0x3f, 0xe2, 0xa2, 0x9a, 0xd7, 0xc7, 0x12, 0xa3, 0x3b, 0x39, 0x7f, 0x2e, 0x27,
0x17, 0x66, 0x3a, 0xb9, 0xf8, 0x52, 0x27, 0x97, 0xfe, 0x4f, 0x4e, 0x5e, 0xf8, 0xdf, 0x3a, 0xb9,
0x7c, 0x61, 0x27, 0x9b, 0x90, 0x67, 0xb3, 0x44, 0xab, 0x90, 0x0b, 0xf1, 0x13, 0xee, 0xd3, 0x25,
0x87, 0x15, 0x6b, 0xa3, 0x3c, 0x2c, 0x89, 0x54, 0x12, 0x05, 0xbe, 0x17, 0x11, 0xc6, 0xe3, 0x1e,
0xcf, 0xfe, 0xc2, 0xf3, 0x92, 0x47, 0xde, 0xe2, 0xc8, 0x1e, 0xf4, 0x1e, 0xe4, 0x77, 0x30, 0xc5,
0x5c, 0x05, 0x8b, 0xcd, 0x35, 0x95, 0x47, 0x36, 0x16, 0xeb, 0xb3, 0xd7, 0xd9, 0x44, 0x4e, 0x63,
0x6b, 0xa5, 0x8d, 0x29, 0xfe, 0xae, 0xdf, 0x77, 0x29, 0xe9, 0x07, 0xf4, 0xc4, 0xe1, 0x48, 0xf4,
0x36, 0x94, 0xef, 0x86, 0xa1, 0x1f, 0xee, 0x9f, 0x04, 0x84, 0xab, 0xa6, 0x6c, 0xbf, 0x72, 0x1a,
0x5b, 0x57, 0x49, 0xd2, 0xa8, 0x20, 0xc6, 0x96, 0xe8, 0xdb, 0x50, 0xe0, 0x15, 0xae, 0x93, 0xb2,
0x7d, 0xf5, 0x34, 0xb6, 0xae, 0x70, 0x88, 0x62, 0x2e, 0x2c, 0x74, 0x59, 0x15, 0xe6, 0x92, 0x55,
0xaa, 0xee, 0xa2, 0xaa, 0x6e, 0x13, 0x4a, 0xc7, 0x24, 0x8c, 0xd8, 0x30, 0x25, 0xde, 0x9e, 0x54,
0xd1, 0x6d, 0x00, 0x46, 0x8c, 0x1b, 0x51, 0xb7, 0x95, 0x38, 0x7b, 0xb9, 0x2e, 0x36, 0x1b, 0x87,
0xfb, 0xc8, 0x46, 0x92, 0x05, 0xc5, 0xd0, 0x51, 0xca, 0xe8, 0xf7, 0x06, 0x94, 0x76, 0x09, 0x6e,
0x93, 0x90, 0xb9, 0x37, 0xb7, 0xb9, 0xd8, 0xfc, 0x56, 0x5d, 0xdd, 0x59, 0x1e, 0x85, 0x7e, 0x9f,
0xd0, 0x2e, 0x19, 0x44, 0x89, 0x83, 0x84, 0xb5, 0xed, 0x0d, 0x63, 0x8b, 0xcc, 0x29, 0xd5, 0xb9,
0x36, 0xb4, 0x33, 0x3f, 0x75, 0x1a, 0x5b, 0xc6, 0xf7, 0x9c, 0x64, 0x96, 0xa8, 0x09, 0x0b, 0x4f,
0x70, 0xe8, 0xb9, 0x5e, 0x27, 0x32, 0x81, 0x47, 0xda, 0xfa, 0x69, 0x6c, 0xa1, 0xa4, 0x4d, 0x71,
0x44, 0x6a, 0x57, 0xfb, 0xab, 0x01, 0x5f, 0x63, 0xc2, 0xd8, 0x63, 0xf3, 0x89, 0x94, 0x14, 0xd3,
0xc7, 0xb4, 0xd5, 0x35, 0x0d, 0x36, 0x8c, 0x23, 0x2a, 0xea, 0x7e, 0x93, 0xfd, 0xaf, 0xf6, 0x9b,
0xdc, 0xf9, 0xf7, 0x9b, 0x24, 0xaf, 0xe4, 0x67, 0xe6, 0x95, 0xc2, 0x59, 0x79, 0xa5, 0xf6, 0x6b,
0x99, 0x43, 0x93, 0xf5, 0x9d, 0x23, 0x94, 0xee, 0xa5, 0xa1, 0x94, 0xe3, 0xb3, 0x4d, 0x15, 0x2a,
0xc6, 0xba, 0xdf, 0x26, 0x1e, 0x75, 0x0f, 0x5d, 0x12, 0xbe, 0x24, 0xa0, 0x14, 0x95, 0xe6, 0x74,
0x95, 0xaa, 0x12, 0xcb, 0x5f, 0x0a, 0x89, 0xe9, 0x71, 0x55, 0xb8, 0x40, 0x5c, 0xd5, 0xfe, 0x95,
0x85, 0x75, 0xe6, 0x91, 0x87, 0xf8, 0x80, 0xf4, 0x7e, 0x82, 0xfb, 0xe7, 0xf4, 0xca, 0xeb, 0x8a,
0x57, 0xca, 0x36, 0xfa, 0x8a, 0xf5, 0xf9, 0x58, 0xff, 0xad, 0x01, 0x0b, 0xc9, 0x06, 0x80, 0xea,
0x00, 0x02, 0xc6, 0x73, 0xbc, 0xe0, 0x7a, 0x85, 0x81, 0xc3, 0xb4, 0xd5, 0x51, 0x2c, 0xd0, 0x2f,
0xa0, 0x28, 0x6a, 0x32, 0x16, 0x94, 0x6d, 0x73, 0x8f, 0x86, 0x04, 0xf7, 0x6f, 0xb7, 0x71, 0x40,
0x49, 0x68, 0xbf, 0xcb, 0x66, 0x31, 0x8c, 0xad, 0x37, 0xce, 0x62, 0x29, 0x39, 0xe1, 0x4b, 0x1c,
0xf3, 0xaf, 0xf8, 0xa6, 0x23, 0xbf, 0x50, 0xfb, 0xd8, 0x80, 0x55, 0x36, 0x51, 0x46, 0x4d, 0x2a,
0x8c, 0x1d, 0x58, 0x08, 0x65, 0x99, 0x4f, 0x77, 0xb1, 0x59, 0xab, 0xeb, 0xb4, 0xce, 0xa0, 0x92,
0x6f, 0xb8, 0x86, 0x93, 0x22, 0xd1, 0x96, 0x46, 0x63, 0x76, 0x16, 0x8d, 0x62, 0x8f, 0x56, 0x89,
0xfb, 0x73, 0x16, 0xd0, 0x7d, 0x76, 0x43, 0x62, 0xfa, 0x1b, 0x4b, 0xf5, 0xe9, 0xd4, 0x8c, 0xae,
0x8d, 0x49, 0x99, 0xb6, 0xb7, 0x6f, 0x0d, 0x63, 0x6b, 0xfb, 0x25, 0xda, 0xf9, 0x0f, 0x78, 0x65,
0x15, 0xaa, 0x7c, 0xb3, 0x97, 0x41, 0xbe, 0xb5, 0x3f, 0x66, 0x61, 0xe5, 0xa7, 0x7e, 0x6f, 0xd0,
0x27, 0x29, 0x7d, 0xc1, 0x14, 0x7d, 0xe6, 0x98, 0x3e, 0xdd, 0xd6, 0xde, 0x1e, 0xc6, 0xd6, 0x8d,
0x79, 0xa9, 0xd3, 0xb1, 0x97, 0x9a, 0xb6, 0xdf, 0xe4, 0x60, 0x6d, 0xdf, 0x0f, 0x7e, 0xbc, 0xc7,
0x6f, 0xd1, 0x4a, 0x9a, 0xec, 0x4e, 0x91, 0xb7, 0x36, 0x26, 0x8f, 0x21, 0xde, 0xc7, 0x34, 0x74,
0x9f, 0xda, 0x37, 0x86, 0xb1, 0xd5, 0x9c, 0x97, 0xb8, 0x31, 0xee, 0x32, 0x93, 0xa6, 0x9d, 0x81,
0x72, 0xf3, 0x9d, 0x81, 0x26, 0xf2, 0x42, 0x7e, 0xbe, 0xbc, 0xf0, 0x87, 0x1c, 0xac, 0x7f, 0x38,
0xc0, 0x1e, 0x75, 0x7b, 0x44, 0x78, 0x28, 0xf5, 0xcf, 0x2f, 0xa7, 0xfc, 0x53, 0x19, 0xfb, 0x47,
0xc7, 0x48, 0x4f, 0xbd, 0x37, 0x8c, 0xad, 0x9b, 0xf3, 0x7a, 0x6a, 0xd6, 0x08, 0x5f, 0xf9, 0x6c,
0x1e, 0x9f, 0xfd, 0x29, 0x0b, 0x2b, 0x7b, 0xe2, 0x7c, 0x98, 0xb0, 0x75, 0x3c, 0xc3, 0x57, 0xea,
0x83, 0x58, 0x70, 0x50, 0xd7, 0x11, 0xe7, 0x4b, 0x47, 0x3a, 0xf6, 0x52, 0xa7, 0xa3, 0xbf, 0x64,
0x61, 0x7d, 0x87, 0x50, 0xd2, 0xa2, 0xa4, 0x7d, 0xcf, 0x25, 0x3d, 0x85, 0xc4, 0x8f, 0x8c, 0x29,
0x16, 0xab, 0xca, 0x85, 0x6e, 0x26, 0xc8, 0xb6, 0x87, 0xb1, 0x75, 0x6b, 0x5e, 0x1e, 0x67, 0x8f,
0x71, 0xa9, 0xf9, 0xfc, 0x2c, 0x0b, 0x5f, 0x17, 0x8f, 0x14, 0xe2, 0x05, 0x75, 0x4c, 0xe7, 0xaf,
0xa6, 0xd8, 0xb4, 0xd4, 0xfc, 0x31, 0x03, 0x62, 0xdf, 0x1e, 0xc6, 0xd6, 0x0f, 0xe7, 0x4f, 0x20,
0x33, 0x86, 0xf8, 0xd2, 0x68, 0x93, 0xdf, 0x2b, 0xce, 0xab, 0x4d, 0x1d, 0x74, 0x31, 0x6d, 0xea,
0x63, 0x5c, 0x6a, 0x3e, 0xff, 0x51, 0x84, 0x65, 0xae, 0x92, 0x94, 0xc6, 0xef, 0x80, 0xbc, 0x88,
0x49, 0x0e, 0x51, 0x72, 0x79, 0x0f, 0x83, 0x56, 0x7d, 0x4f, 0x5e, 0xd1, 0x84, 0x05, 0x7a, 0x07,
0x8a, 0x11, 0xbf, 0x22, 0xcb, 0x33, 0x76, 0x65, 0xf2, 0x15, 0x4a, 0xbf, 0x8c, 0xef, 0x66, 0x1c,
0x69, 0x8f, 0x6e, 0x42, 0xb1, 0xc7, 0x59, 0x94, 0x4f, 0x04, 0xb5, 0x49, 0xe4, 0xf4, 0xa5, 0x91,
0xa1, 0x05, 0x06, 0xdd, 0x80, 0x02, 0xdf, 0x00, 0xe4, 0x76, 0xa0, 0x7d, 0x76, 0xfa, 0x48, 0xbd,
0x9b, 0x71, 0x84, 0x39, 0x6a, 0x42, 0x3e, 0x08, 0xfd, 0xbe, 0xbc, 0x58, 0x5d, 0x9b, 0xfc, 0xa6,
0x7a, 0x13, 0xd9, 0xcd, 0x38, 0xdc, 0x16, 0xbd, 0x05, 0xa5, 0x88, 0x5f, 0x61, 0x22, 0xfe, 0x24,
0xc5, 0xce, 0xaf, 0x13, 0x30, 0x05, 0x92, 0x98, 0xa2, 0xb7, 0xa0, 0x78, 0xcc, 0x0f, 0xa8, 0xf2,
0x9d, 0x73, 0x43, 0x05, 0xe9, 0x47, 0x57, 0xb6, 0x2e, 0x61, 0x8b, 0xee, 0xc1, 0x12, 0xf5, 0x83,
0xa3, 0xe4, 0x1c, 0x28, 0x9f, 0xb3, 0xaa, 0x2a, 0x76, 0xd6, 0x39, 0x71, 0x37, 0xe3, 0x68, 0x38,
0xf4, 0x08, 0x56, 0x1f, 0x6b, 0x67, 0x07, 0x92, 0x3c, 0x5c, 0x6a, 0x3c, 0xcf, 0x3e, 0xd5, 0xec,
0x66, 0x9c, 0x29, 0x34, 0xda, 0x81, 0x95, 0x48, 0xdb, 0xe1, 0xe4, 0x23, 0xbd, 0xb6, 0x2e, 0x7d,
0x0f, 0xdc, 0xcd, 0x38, 0x13, 0x18, 0xf4, 0x10, 0x56, 0xda, 0x5a, 0x7e, 0x97, 0x4f, 0xf0, 0xda,
0xac, 0x66, 0xef, 0x00, 0x6c, 0x34, 0x1d, 0x8b, 0x3e, 0x80, 0xd5, 0x60, 0x22, 0xb7, 0xc9, 0x37,
0xf8, 0x6f, 0xea, 0xab, 0x9c, 0x91, 0x04, 0xd9, 0x22, 0x27, 0xc1, 0xea, 0xf4, 0x44, 0x88, 0x9b,
0xcb, 0x67, 0x4f, 0x4f, 0x4f, 0x02, 0xea, 0xf4, 0x44, 0x8f, 0x0d, 0xe3, 0x74, 0x54, 0xfb, 0xb8,
0x08, 0x4b, 0x32, 0xcc, 0xc4, 0xbb, 0xdb, 0xf7, 0xd3, 0xc8, 0x11, 0x51, 0xf6, 0xda, 0x59, 0x91,
0xc3, 0xcd, 0x95, 0xc0, 0x79, 0x33, 0x0d, 0x1c, 0x11, 0x72, 0xeb, 0xe3, 0x14, 0xc7, 0xbf, 0xab,
0x20, 0x64, 0xb0, 0x6c, 0x25, 0xc1, 0x22, 0x22, 0xed, 0xd5, 0xd9, 0xb7, 0xd7, 0x04, 0x25, 0x23,
0x65, 0x1b, 0x4a, 0xae, 0xf8, 0x31, 0x62, 0x56, 0x8c, 0x4d, 0xff, 0x56, 0xc1, 0xb4, 0x2f, 0x01,
0x68, 0x6b, 0x1c, 0x31, 0x05, 0xf9, 0xf8, 0x3e, 0x15, 0x31, 0x29, 0x28, 0x09, 0x98, 0xeb, 0x69,
0xc0, 0x14, 0x27, 0x1f, 0xec, 0x93, 0x70, 0x49, 0x17, 0x26, 0xa3, 0xe5, 0x2e, 0x2c, 0x27, 0xfa,
0xe2, 0x5d, 0x32, 0x5c, 0x5e, 0x3b, 0xeb, 0x58, 0x97, 0xe0, 0x75, 0x14, 0xba, 0x3f, 0x25, 0xca,
0xf2, 0xe4, 0x56, 0x3c, 0x29, 0xc9, 0x64, 0xa4, 0x49, 0x45, 0x3e, 0x80, 0x2b, 0x63, 0x51, 0x89,
0x39, 0xc1, 0xf4, 0xb5, 0x40, 0x93, 0x63, 0x32, 0xd4, 0x24, 0x50, 0x9d, 0x96, 0x14, 0xe3, 0xe2,
0x59, 0xd3, 0x4a, 0xa4, 0x38, 0x35, 0x2d, 0xd1, 0x81, 0x76, 0x61, 0xa1, 0x4f, 0x28, 0x6e, 0x63,
0x8a, 0xcd, 0x12, 0xdf, 0x96, 0x5e, 0x9f, 0x0a, 0x10, 0x89, 0xae, 0xbf, 0x2f, 0x0d, 0xef, 0x7a,
0x34, 0x3c, 0x91, 0x27, 0xeb, 0x14, 0xbd, 0xf1, 0x03, 0x58, 0xd6, 0x0c, 0xd0, 0x2a, 0xe4, 0x8e,
0x48, 0xf2, 0x03, 0x15, 0x2b, 0xa2, 0x35, 0x28, 0x1c, 0xe3, 0xde, 0x80, 0x70, 0x7d, 0x96, 0x1d,
0x51, 0xd9, 0xce, 0xbe, 0x63, 0xd8, 0x65, 0x28, 0x85, 0xe2, 0x2b, 0x76, 0xe7, 0xd9, 0xf3, 0x4a,
0xe6, 0xf3, 0xe7, 0x95, 0xcc, 0x17, 0xcf, 0x2b, 0xc6, 0x47, 0xa3, 0x8a, 0xf1, 0xbb, 0x51, 0xc5,
0xf8, 0x74, 0x54, 0x31, 0x9e, 0x8d, 0x2a, 0xc6, 0xdf, 0x47, 0x15, 0xe3, 0x9f, 0xa3, 0x4a, 0xe6,
0x8b, 0x51, 0xc5, 0xf8, 0xe4, 0x45, 0x25, 0xf3, 0xec, 0x45, 0x25, 0xf3, 0xf9, 0x8b, 0x4a, 0xe6,
0x67, 0xd7, 0xcf, 0xbd, 0x43, 0x1e, 0x14, 0x39, 0x53, 0x5b, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff,
0x6b, 0xd4, 0xf8, 0x77, 0xa9, 0x1f, 0x00, 0x00,
// 1991 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x6f, 0x1b, 0xc7,
0x15, 0xe6, 0xf2, 0xa7, 0x38, 0x92, 0x68, 0x75, 0xac, 0x2a, 0x5b, 0xc5, 0xe1, 0xb2, 0x04, 0x9a,
0xa8, 0x45, 0x4b, 0xc6, 0x54, 0xe2, 0x26, 0xaa, 0x6b, 0xc4, 0x6b, 0xd9, 0x95, 0x5d, 0xbb, 0x71,
0x56, 0x42, 0x0e, 0xbd, 0x14, 0x23, 0x72, 0x44, 0x6e, 0x45, 0xee, 0xae, 0x77, 0x86, 0xb2, 0x05,
0x14, 0x45, 0xfe, 0x81, 0xa0, 0xb9, 0xf7, 0x5e, 0xf4, 0x56, 0x14, 0x28, 0x7a, 0xe8, 0xa9, 0xc7,
0xe4, 0x50, 0xc0, 0xc7, 0x80, 0x40, 0xd9, 0x9a, 0xbe, 0x14, 0x3a, 0x05, 0xe8, 0x3f, 0x50, 0xcc,
0x8f, 0x5d, 0xce, 0x70, 0x57, 0x35, 0xe9, 0x16, 0x05, 0x54, 0xf8, 0x42, 0xee, 0xcc, 0xbc, 0x6f,
0x76, 0xe6, 0x7b, 0xdf, 0x9b, 0x37, 0x33, 0x0b, 0xde, 0x0a, 0x8e, 0xbb, 0xcd, 0x47, 0x43, 0x1c,
0xba, 0x38, 0xe4, 0xff, 0xa7, 0x21, 0xf2, 0xba, 0x58, 0x79, 0x6c, 0x04, 0xa1, 0x4f, 0x7d, 0x08,
0xa6, 0x35, 0x9b, 0xad, 0xae, 0x4b, 0x7b, 0xc3, 0xc3, 0x46, 0xdb, 0x1f, 0x34, 0xbb, 0x7e, 0xd7,
0x6f, 0x76, 0x7d, 0xbf, 0xdb, 0xc7, 0x28, 0x70, 0x89, 0x7c, 0x6c, 0x86, 0x41, 0xbb, 0x49, 0x28,
0xa2, 0x43, 0x22, 0xf0, 0x9b, 0xeb, 0xcc, 0x90, 0x3f, 0x72, 0x88, 0xac, 0xb5, 0xa4, 0x39, 0x2f,
0x1d, 0x0e, 0x8f, 0x9a, 0xd4, 0x1d, 0x60, 0x42, 0xd1, 0x20, 0x88, 0x0c, 0xd8, 0xf8, 0xfa, 0x7e,
0x57, 0x20, 0x5d, 0xaf, 0x83, 0x9f, 0x74, 0x11, 0xc5, 0x8f, 0xd1, 0xa9, 0x34, 0x78, 0x5d, 0x33,
0x88, 0x1e, 0x64, 0xe3, 0xa6, 0xd6, 0x18, 0x20, 0x4a, 0x71, 0xe8, 0xc9, 0xb6, 0x6f, 0x68, 0x6d,
0xe4, 0x18, 0xd3, 0x76, 0x4f, 0x36, 0xd5, 0x64, 0xd3, 0xa3, 0xfe, 0xc0, 0xef, 0xe0, 0x3e, 0x9f,
0x08, 0x11, 0xbf, 0xd2, 0xe2, 0x32, 0xb3, 0x08, 0x86, 0xa4, 0xc7, 0x7f, 0x64, 0xe5, 0xad, 0x17,
0x72, 0x79, 0x88, 0x08, 0x6e, 0x76, 0xf0, 0x91, 0xeb, 0xb9, 0xd4, 0xf5, 0x3d, 0xa2, 0x3e, 0xcb,
0x4e, 0xae, 0xcd, 0xd7, 0xc9, 0xac, 0x7f, 0x36, 0xdf, 0x66, 0x38, 0x42, 0xfd, 0x10, 0x75, 0x71,
0xb3, 0xdd, 0x1b, 0x7a, 0xc7, 0xcd, 0x36, 0x6a, 0xf7, 0x70, 0x33, 0xc4, 0x64, 0xd8, 0xa7, 0x44,
0x14, 0xe8, 0x69, 0x80, 0xe5, 0x9b, 0xea, 0x5f, 0xe4, 0xc1, 0xf2, 0x7d, 0xff, 0xd8, 0x75, 0xf0,
0xa3, 0x21, 0x26, 0x14, 0xae, 0x83, 0x02, 0xef, 0xd5, 0x34, 0x6a, 0xc6, 0x56, 0xd9, 0x11, 0x05,
0x56, 0xdb, 0x77, 0x07, 0x2e, 0x35, 0xb3, 0x35, 0x63, 0x6b, 0xd5, 0x11, 0x05, 0x08, 0x41, 0x9e,
0x50, 0x1c, 0x98, 0xb9, 0x9a, 0xb1, 0x95, 0x73, 0xf8, 0x33, 0xdc, 0x04, 0x4b, 0xae, 0x47, 0x71,
0x78, 0x82, 0xfa, 0x66, 0x99, 0xd7, 0xc7, 0x65, 0x78, 0x03, 0x94, 0x08, 0x45, 0x21, 0x3d, 0x20,
0x66, 0xbe, 0x66, 0x6c, 0x2d, 0xb7, 0x36, 0x1b, 0xc2, 0xf3, 0x8d, 0xc8, 0xf3, 0x8d, 0x83, 0xc8,
0xf3, 0xf6, 0xd2, 0xe7, 0x63, 0x2b, 0xf3, 0xd9, 0xdf, 0x2c, 0xc3, 0x89, 0x40, 0x70, 0x07, 0x14,
0xb0, 0xd7, 0x39, 0x20, 0x66, 0x61, 0x01, 0xb4, 0x80, 0xc0, 0xab, 0xa0, 0xdc, 0x71, 0x43, 0xdc,
0x66, 0x2c, 0x9b, 0xc5, 0x9a, 0xb1, 0x55, 0x69, 0x5d, 0x6e, 0xc4, 0x42, 0xd9, 0x8d, 0x9a, 0x9c,
0xa9, 0x15, 0x9b, 0x5e, 0x80, 0x68, 0xcf, 0x2c, 0x71, 0x26, 0xf8, 0x33, 0xac, 0x83, 0x22, 0xe9,
0xa1, 0xb0, 0x43, 0xcc, 0xa5, 0x5a, 0x6e, 0xab, 0x6c, 0x83, 0xb3, 0xb1, 0x25, 0x6b, 0x1c, 0xf9,
0x0f, 0x7f, 0x06, 0xf2, 0x41, 0x1f, 0x79, 0x26, 0xe0, 0xa3, 0x5c, 0x6b, 0x28, 0x5e, 0x7a, 0xd8,
0x47, 0x9e, 0xfd, 0xfe, 0x68, 0x6c, 0xbd, 0xab, 0x06, 0x4f, 0x88, 0x8e, 0x90, 0x87, 0x9a, 0x7d,
0xff, 0xd8, 0x6d, 0x9e, 0x6c, 0x37, 0x55, 0xdf, 0xb3, 0x8e, 0x1a, 0x1f, 0xb1, 0x0e, 0x18, 0xd4,
0xe1, 0x1d, 0xc3, 0x7b, 0x60, 0x99, 0xf9, 0x18, 0xdf, 0x62, 0x0e, 0x26, 0xe6, 0x32, 0x7f, 0xcf,
0x6b, 0xd3, 0xd9, 0xf0, 0x7a, 0x07, 0x1f, 0xfd, 0x28, 0xf4, 0x87, 0x81, 0x7d, 0xe9, 0x6c, 0x6c,
0xa9, 0xf6, 0x8e, 0x5a, 0x80, 0xf7, 0x40, 0x85, 0x89, 0xc2, 0xf5, 0xba, 0x1f, 0x06, 0x5c, 0x81,
0xe6, 0x0a, 0xef, 0xee, 0x4a, 0x43, 0x95, 0x4c, 0xe3, 0x96, 0x66, 0x63, 0xe7, 0x19, 0xbd, 0xce,
0x0c, 0xb2, 0x3e, 0xc9, 0x01, 0xc8, 0xb4, 0x74, 0xd7, 0x23, 0x14, 0x79, 0xf4, 0x65, 0x24, 0x75,
0x1d, 0x14, 0x59, 0xf0, 0x1f, 0x10, 0x2e, 0xaa, 0x79, 0x7d, 0x2c, 0x31, 0xba, 0x93, 0xf3, 0x0b,
0x39, 0xb9, 0x90, 0xea, 0xe4, 0xe2, 0x0b, 0x9d, 0x5c, 0xfa, 0x1f, 0x39, 0x79, 0xe9, 0xbf, 0xeb,
0xe4, 0xf2, 0x4b, 0x3b, 0xd9, 0x04, 0x79, 0x36, 0x4a, 0xb8, 0x06, 0x72, 0x21, 0x7a, 0xcc, 0x7d,
0xba, 0xe2, 0xb0, 0xc7, 0xfa, 0x24, 0x0f, 0x56, 0xc4, 0x52, 0x42, 0x02, 0xdf, 0x23, 0x98, 0xf1,
0xb8, 0xcf, 0x57, 0x7f, 0xe1, 0x79, 0xc9, 0x23, 0xaf, 0x71, 0x64, 0x0b, 0xfc, 0x00, 0xe4, 0x77,
0x11, 0x45, 0x5c, 0x05, 0xcb, 0xad, 0x75, 0x95, 0x47, 0xd6, 0x17, 0x6b, 0xb3, 0x37, 0xd8, 0x40,
0xce, 0xc6, 0x56, 0xa5, 0x83, 0x28, 0xfa, 0xae, 0x3f, 0x70, 0x29, 0x1e, 0x04, 0xf4, 0xd4, 0xe1,
0x48, 0xf8, 0x2e, 0x28, 0xdf, 0x0e, 0x43, 0x3f, 0x3c, 0x38, 0x0d, 0x30, 0x57, 0x4d, 0xd9, 0x7e,
0xed, 0x6c, 0x6c, 0x5d, 0xc6, 0x51, 0xa5, 0x82, 0x98, 0x5a, 0xc2, 0x6f, 0x83, 0x02, 0x2f, 0x70,
0x9d, 0x94, 0xed, 0xcb, 0x67, 0x63, 0xeb, 0x12, 0x87, 0x28, 0xe6, 0xc2, 0x42, 0x97, 0x55, 0x61,
0x2e, 0x59, 0xc5, 0xea, 0x2e, 0xaa, 0xea, 0x36, 0x41, 0xe9, 0x04, 0x87, 0x84, 0x75, 0x53, 0xe2,
0xf5, 0x51, 0x11, 0xde, 0x04, 0x80, 0x11, 0xe3, 0x12, 0xea, 0xb6, 0x23, 0x67, 0xaf, 0x36, 0x44,
0xb2, 0x71, 0xb8, 0x8f, 0x6c, 0x28, 0x59, 0x50, 0x0c, 0x1d, 0xe5, 0x19, 0xfe, 0xce, 0x00, 0xa5,
0x3d, 0x8c, 0x3a, 0x38, 0x64, 0xee, 0xcd, 0x6d, 0x2d, 0xb7, 0xbe, 0xd5, 0x50, 0x33, 0xcb, 0xc3,
0xd0, 0x1f, 0x60, 0xda, 0xc3, 0x43, 0x12, 0x39, 0x48, 0x58, 0xdb, 0xde, 0x68, 0x6c, 0xe1, 0x39,
0xa5, 0x3a, 0x57, 0x42, 0x3b, 0xf7, 0x55, 0x67, 0x63, 0xcb, 0xf8, 0x9e, 0x13, 0x8d, 0x12, 0xb6,
0xc0, 0xd2, 0x63, 0x14, 0x7a, 0xae, 0xd7, 0x25, 0x26, 0xe0, 0x91, 0xb6, 0x71, 0x36, 0xb6, 0x60,
0x54, 0xa7, 0x38, 0x22, 0xb6, 0xab, 0xff, 0xd5, 0x00, 0x5f, 0x63, 0xc2, 0xd8, 0x67, 0xe3, 0x21,
0xca, 0x12, 0x33, 0x40, 0xb4, 0xdd, 0x33, 0x0d, 0xd6, 0x8d, 0x23, 0x0a, 0x6a, 0xbe, 0xc9, 0xfe,
0x47, 0xf9, 0x26, 0xb7, 0x78, 0xbe, 0x89, 0xd6, 0x95, 0x7c, 0xea, 0xba, 0x52, 0x38, 0x6f, 0x5d,
0xa9, 0xff, 0x4a, 0xae, 0xa1, 0xd1, 0xfc, 0x16, 0x08, 0xa5, 0x3b, 0x71, 0x28, 0xe5, 0xf8, 0x68,
0x63, 0x85, 0x8a, 0xbe, 0xee, 0x76, 0xb0, 0x47, 0xdd, 0x23, 0x17, 0x87, 0x2f, 0x08, 0x28, 0x45,
0xa5, 0x39, 0x5d, 0xa5, 0xaa, 0xc4, 0xf2, 0x17, 0x42, 0x62, 0x7a, 0x5c, 0x15, 0x5e, 0x22, 0xae,
0xea, 0xff, 0xcc, 0x82, 0x0d, 0xe6, 0x91, 0xfb, 0xe8, 0x10, 0xf7, 0x7f, 0x82, 0x06, 0x0b, 0x7a,
0xe5, 0x4d, 0xc5, 0x2b, 0x65, 0x1b, 0xbe, 0x62, 0x7d, 0x3e, 0xd6, 0x7f, 0x63, 0x80, 0xa5, 0x28,
0x01, 0xc0, 0x06, 0x00, 0x02, 0xc6, 0xd7, 0x78, 0xc1, 0x75, 0x85, 0x81, 0xc3, 0xb8, 0xd6, 0x51,
0x2c, 0xe0, 0xcf, 0x41, 0x51, 0x94, 0x64, 0x2c, 0x28, 0x69, 0x73, 0x9f, 0x86, 0x18, 0x0d, 0x6e,
0x76, 0x50, 0x40, 0x71, 0x68, 0xbf, 0xcf, 0x46, 0x31, 0x1a, 0x5b, 0x6f, 0x9d, 0xc7, 0x52, 0xb4,
0xc3, 0x97, 0x38, 0xe6, 0x5f, 0xf1, 0x4e, 0x47, 0xbe, 0xa1, 0xfe, 0xa9, 0x01, 0xd6, 0xd8, 0x40,
0x19, 0x35, 0xb1, 0x30, 0x76, 0xc1, 0x52, 0x28, 0x9f, 0xf9, 0x70, 0x97, 0x5b, 0xf5, 0x86, 0x4e,
0x6b, 0x0a, 0x95, 0x3c, 0xe1, 0x1a, 0x4e, 0x8c, 0x84, 0xdb, 0x1a, 0x8d, 0xd9, 0x34, 0x1a, 0x45,
0x8e, 0x56, 0x89, 0xfb, 0x73, 0x16, 0xc0, 0xbb, 0xec, 0x84, 0xc4, 0xf4, 0x37, 0x95, 0xea, 0x93,
0xc4, 0x88, 0xae, 0x4c, 0x49, 0x49, 0xda, 0xdb, 0x37, 0x46, 0x63, 0x6b, 0xe7, 0x05, 0xda, 0xf9,
0x37, 0x78, 0x65, 0x16, 0xaa, 0x7c, 0xb3, 0x17, 0x41, 0xbe, 0xf5, 0x3f, 0x64, 0x41, 0xe5, 0x63,
0xbf, 0x3f, 0x1c, 0xe0, 0x98, 0xbe, 0x20, 0x41, 0x9f, 0x39, 0xa5, 0x4f, 0xb7, 0xb5, 0x77, 0x46,
0x63, 0xeb, 0xda, 0xbc, 0xd4, 0xe9, 0xd8, 0x0b, 0x4d, 0xdb, 0xaf, 0x73, 0x60, 0xfd, 0xc0, 0x0f,
0x7e, 0xbc, 0xcf, 0x4f, 0xd1, 0xca, 0x32, 0xd9, 0x4b, 0x90, 0xb7, 0x3e, 0x25, 0x8f, 0x21, 0x1e,
0x20, 0x1a, 0xba, 0x4f, 0xec, 0x6b, 0xa3, 0xb1, 0xd5, 0x9a, 0x97, 0xb8, 0x29, 0xee, 0x22, 0x93,
0xa6, 0xed, 0x81, 0x72, 0xf3, 0xed, 0x81, 0x66, 0xd6, 0x85, 0xfc, 0x7c, 0xeb, 0xc2, 0xef, 0x73,
0x60, 0xe3, 0xa3, 0x21, 0xf2, 0xa8, 0xdb, 0xc7, 0xc2, 0x43, 0xb1, 0x7f, 0x7e, 0x91, 0xf0, 0x4f,
0x75, 0xea, 0x1f, 0x1d, 0x23, 0x3d, 0xf5, 0xc1, 0x68, 0x6c, 0x5d, 0x9f, 0xd7, 0x53, 0x69, 0x3d,
0xbc, 0xf2, 0xd9, 0xbc, 0x3e, 0xbb, 0xe5, 0x0f, 0x3d, 0xfa, 0xc0, 0xf5, 0x16, 0xf1, 0x99, 0x8e,
0xf9, 0x18, 0xb7, 0xa9, 0x1f, 0x2e, 0xe6, 0xb3, 0xb4, 0x1e, 0x5e, 0xf9, 0x6c, 0x1e, 0x9f, 0xfd,
0x29, 0x0b, 0x2a, 0xfb, 0x62, 0x4f, 0x1f, 0xb1, 0x75, 0x92, 0xe2, 0x2b, 0xf5, 0x12, 0x33, 0x38,
0x6c, 0xe8, 0x88, 0xc5, 0x52, 0x88, 0x8e, 0xbd, 0xd0, 0x29, 0xe4, 0x2f, 0x59, 0xb0, 0xb1, 0x8b,
0x29, 0x6e, 0x53, 0xdc, 0xb9, 0xe3, 0xe2, 0xbe, 0x42, 0xe2, 0x27, 0x46, 0x82, 0xc5, 0x9a, 0x72,
0x08, 0x4f, 0x05, 0xd9, 0xf6, 0x68, 0x6c, 0xdd, 0x98, 0x97, 0xc7, 0xf4, 0x3e, 0x2e, 0x34, 0x9f,
0x5f, 0x64, 0xc1, 0xd7, 0xc5, 0xc5, 0x92, 0xb8, 0xf5, 0x9e, 0xd2, 0xf9, 0xcb, 0x04, 0x9b, 0x96,
0xba, 0xe6, 0xa7, 0x40, 0xec, 0x9b, 0xa3, 0xb1, 0xf5, 0xc3, 0xf9, 0x17, 0xfd, 0x94, 0x2e, 0xfe,
0x6f, 0xb4, 0xc9, 0xcf, 0x82, 0x8b, 0x6a, 0x53, 0x07, 0xbd, 0x9c, 0x36, 0xf5, 0x3e, 0x2e, 0x34,
0x9f, 0x7f, 0x2c, 0x81, 0x55, 0xae, 0x92, 0x98, 0xc6, 0xef, 0x00, 0x79, 0x78, 0x96, 0x1c, 0xc2,
0xe8, 0xc2, 0x25, 0x0c, 0xda, 0x8d, 0x7d, 0x79, 0xac, 0x16, 0x16, 0xf0, 0x3d, 0x50, 0x24, 0xfc,
0x5a, 0x43, 0x9e, 0x8b, 0xaa, 0xb3, 0x37, 0x87, 0xfa, 0x05, 0xca, 0x5e, 0xc6, 0x91, 0xf6, 0xf0,
0x3a, 0x28, 0xf6, 0x39, 0x8b, 0xf2, 0x5a, 0xa7, 0x3e, 0x8b, 0x4c, 0x1e, 0xf4, 0x19, 0x5a, 0x60,
0xe0, 0x35, 0x50, 0xe0, 0x09, 0x40, 0xa6, 0x03, 0xed, 0xb5, 0xc9, 0x63, 0xd0, 0x5e, 0xc6, 0x11,
0xe6, 0xb0, 0x05, 0xf2, 0x41, 0xe8, 0x0f, 0xe4, 0x61, 0xf8, 0xca, 0xec, 0x3b, 0xd5, 0xd3, 0xe3,
0x5e, 0xc6, 0xe1, 0xb6, 0xf0, 0x1d, 0x50, 0x22, 0xfc, 0xd8, 0x49, 0xf8, 0x35, 0x22, 0x3b, 0x73,
0xcc, 0xc0, 0x14, 0x48, 0x64, 0x0a, 0xdf, 0x01, 0xc5, 0x13, 0x7e, 0xa8, 0x90, 0x77, 0xd3, 0x9b,
0x2a, 0x48, 0x3f, 0x6e, 0xb0, 0x79, 0x09, 0x5b, 0x78, 0x07, 0xac, 0x50, 0x3f, 0x38, 0x8e, 0xf6,
0xee, 0xf2, 0x0a, 0xb2, 0xa6, 0x62, 0xd3, 0xf6, 0xf6, 0x7b, 0x19, 0x47, 0xc3, 0xc1, 0x87, 0x60,
0xed, 0x91, 0xb6, 0xdf, 0xc3, 0xd1, 0x65, 0xb3, 0xc6, 0x73, 0xfa, 0x4e, 0x74, 0x2f, 0xe3, 0x24,
0xd0, 0x70, 0x17, 0x54, 0x88, 0x96, 0xe1, 0xe4, 0x87, 0x15, 0x6d, 0x5e, 0x7a, 0x0e, 0xdc, 0xcb,
0x38, 0x33, 0x18, 0x78, 0x1f, 0x54, 0x3a, 0xda, 0xfa, 0x2e, 0x3f, 0x9b, 0x68, 0xa3, 0x4a, 0xcf,
0x00, 0xac, 0x37, 0x1d, 0x0b, 0x3f, 0x04, 0x6b, 0xc1, 0xcc, 0xda, 0x26, 0xbf, 0x9b, 0x7c, 0x53,
0x9f, 0x65, 0xca, 0x22, 0xc8, 0x26, 0x39, 0x0b, 0x56, 0x87, 0x27, 0x42, 0xdc, 0x5c, 0x3d, 0x7f,
0x78, 0xfa, 0x22, 0xa0, 0x0e, 0x4f, 0xb4, 0x30, 0x27, 0xb4, 0xb5, 0x0d, 0x1c, 0x26, 0x66, 0x25,
0xd9, 0x5f, 0xfa, 0xd6, 0x92, 0x8d, 0x6f, 0x16, 0x6d, 0x83, 0xe9, 0x02, 0x57, 0xff, 0xb4, 0x08,
0x56, 0x64, 0xe0, 0x8a, 0xdb, 0xd7, 0xef, 0xc7, 0xb1, 0x28, 0xe2, 0xf6, 0x8d, 0xf3, 0x62, 0x91,
0x9b, 0x2b, 0xa1, 0xf8, 0x76, 0x1c, 0x8a, 0x22, 0x88, 0x37, 0xa6, 0x8b, 0x26, 0x9f, 0x89, 0x82,
0x90, 0xe1, 0xb7, 0x1d, 0x85, 0x9f, 0x88, 0xdd, 0xd7, 0xd3, 0xef, 0x30, 0x22, 0x94, 0x8c, 0xbd,
0x1d, 0x50, 0x72, 0xc5, 0x27, 0xa9, 0xb4, 0xa8, 0x4d, 0x7e, 0xb1, 0x62, 0xd1, 0x24, 0x01, 0x70,
0x7b, 0x1a, 0x83, 0x05, 0xf9, 0x09, 0x26, 0x11, 0x83, 0x31, 0x28, 0x0a, 0xc1, 0xab, 0x71, 0x08,
0x16, 0x67, 0x3f, 0xdb, 0x44, 0x01, 0x18, 0x4f, 0x4c, 0xc6, 0xdf, 0x6d, 0xb0, 0x1a, 0x29, 0x96,
0x37, 0xc9, 0x00, 0x7c, 0xe3, 0xbc, 0x8d, 0x62, 0x84, 0xd7, 0x51, 0xf0, 0x6e, 0x42, 0xe6, 0xe5,
0xd9, 0xe4, 0x3e, 0x2b, 0xf2, 0xa8, 0xa7, 0x59, 0x8d, 0xdf, 0x03, 0x97, 0xa6, 0x32, 0x15, 0x63,
0x02, 0xc9, 0xc3, 0xa1, 0x26, 0xf0, 0xa8, 0xab, 0x59, 0xa0, 0x3a, 0x2c, 0x29, 0xef, 0xe5, 0xf3,
0x86, 0x15, 0x89, 0x3b, 0x31, 0x2c, 0xa9, 0xed, 0x3d, 0xb0, 0x34, 0xc0, 0x14, 0x75, 0x10, 0x45,
0x66, 0x89, 0x27, 0xba, 0x37, 0x13, 0x21, 0x27, 0xd1, 0x8d, 0x07, 0xd2, 0xf0, 0xb6, 0x47, 0xc3,
0x53, 0xb9, 0x57, 0x8f, 0xd1, 0x9b, 0x3f, 0x00, 0xab, 0x9a, 0x01, 0x5c, 0x03, 0xb9, 0x63, 0x1c,
0x7d, 0xa6, 0x64, 0x8f, 0x70, 0x1d, 0x14, 0x4e, 0x50, 0x7f, 0x88, 0xb9, 0x3e, 0xcb, 0x8e, 0x28,
0xec, 0x64, 0xdf, 0x33, 0xec, 0x32, 0x28, 0x85, 0xe2, 0x2d, 0x76, 0xf7, 0xe9, 0xb3, 0x6a, 0xe6,
0xcb, 0x67, 0xd5, 0xcc, 0x57, 0xcf, 0xaa, 0xc6, 0x27, 0x93, 0xaa, 0xf1, 0xdb, 0x49, 0xd5, 0xf8,
0x7c, 0x52, 0x35, 0x9e, 0x4e, 0xaa, 0xc6, 0xdf, 0x27, 0x55, 0xe3, 0x1f, 0x93, 0x6a, 0xe6, 0xab,
0x49, 0xd5, 0xf8, 0xec, 0x79, 0x35, 0xf3, 0xf4, 0x79, 0x35, 0xf3, 0xe5, 0xf3, 0x6a, 0xe6, 0xa7,
0x57, 0x17, 0xce, 0xb9, 0x87, 0x45, 0xce, 0xd4, 0xf6, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xaa,
0x37, 0x05, 0xb8, 0xaf, 0x21, 0x00, 0x00,
}
func (this *LokiRequest) Equal(that interface{}) bool {
@ -2200,6 +2270,53 @@ func (this *QuantileSketchResponse) Equal(that interface{}) bool {
}
return true
}
func (this *CountMinSketchResponse) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*CountMinSketchResponse)
if !ok {
that2, ok := that.(CountMinSketchResponse)
if ok {
that1 = &that2
} else {
return false
}
}
if that1 == nil {
return this == nil
} else if this == nil {
return false
}
if that1.Response == nil {
if this.Response != nil {
return false
}
} else if !this.Response.Equal(*that1.Response) {
return false
}
if len(this.Headers) != len(that1.Headers) {
return false
}
for i := range this.Headers {
if !this.Headers[i].Equal(that1.Headers[i]) {
return false
}
}
if len(this.Warnings) != len(that1.Warnings) {
return false
}
for i := range this.Warnings {
if this.Warnings[i] != that1.Warnings[i] {
return false
}
}
if !this.Statistics.Equal(&that1.Statistics) {
return false
}
return true
}
func (this *ShardsResponse) Equal(that interface{}) bool {
if that == nil {
return this == nil
@ -2665,6 +2782,30 @@ func (this *QueryResponse_DetectedLabels) Equal(that interface{}) bool {
}
return true
}
func (this *QueryResponse_CountMinSketches) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*QueryResponse_CountMinSketches)
if !ok {
that2, ok := that.(QueryResponse_CountMinSketches)
if ok {
that1 = &that2
} else {
return false
}
}
if that1 == nil {
return this == nil
} else if this == nil {
return false
}
if !this.CountMinSketches.Equal(that1.CountMinSketches) {
return false
}
return true
}
func (this *QueryRequest) Equal(that interface{}) bool {
if that == nil {
return this == nil
@ -3135,6 +3276,19 @@ func (this *QuantileSketchResponse) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *CountMinSketchResponse) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 8)
s = append(s, "&queryrange.CountMinSketchResponse{")
s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n")
s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n")
s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n")
s = append(s, "Statistics: "+strings.Replace(this.Statistics.GoString(), `&`, ``, 1)+",\n")
s = append(s, "}")
return strings.Join(s, "")
}
func (this *ShardsResponse) GoString() string {
if this == nil {
return "nil"
@ -3183,7 +3337,7 @@ func (this *QueryResponse) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 17)
s := make([]string, 0, 18)
s = append(s, "&queryrange.QueryResponse{")
if this.Status != nil {
s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n")
@ -3290,6 +3444,14 @@ func (this *QueryResponse_DetectedLabels) GoString() string {
`DetectedLabels:` + fmt.Sprintf("%#v", this.DetectedLabels) + `}`}, ", ")
return s
}
func (this *QueryResponse_CountMinSketches) GoString() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&queryrange.QueryResponse_CountMinSketches{` +
`CountMinSketches:` + fmt.Sprintf("%#v", this.CountMinSketches) + `}`}, ", ")
return s
}
func (this *QueryRequest) GoString() string {
if this == nil {
return "nil"
@ -4277,6 +4439,74 @@ func (m *QuantileSketchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error)
return len(dAtA) - i, nil
}
func (m *CountMinSketchResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *CountMinSketchResponse) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *CountMinSketchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
{
size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintQueryrange(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x22
if len(m.Warnings) > 0 {
for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- {
i -= len(m.Warnings[iNdEx])
copy(dAtA[i:], m.Warnings[iNdEx])
i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Warnings[iNdEx])))
i--
dAtA[i] = 0x1a
}
}
if len(m.Headers) > 0 {
for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- {
{
size := m.Headers[iNdEx].Size()
i -= size
if _, err := m.Headers[iNdEx].MarshalTo(dAtA[i:]); err != nil {
return 0, err
}
i = encodeVarintQueryrange(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
}
}
if m.Response != nil {
{
size := m.Response.Size()
i -= size
if _, err := m.Response.MarshalTo(dAtA[i:]); err != nil {
return 0, err
}
i = encodeVarintQueryrange(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
return len(dAtA) - i, nil
}
func (m *ShardsResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -4757,6 +4987,26 @@ func (m *QueryResponse_DetectedLabels) MarshalToSizedBuffer(dAtA []byte) (int, e
}
return len(dAtA) - i, nil
}
func (m *QueryResponse_CountMinSketches) MarshalTo(dAtA []byte) (int, error) {
return m.MarshalToSizedBuffer(dAtA[:m.Size()])
}
func (m *QueryResponse_CountMinSketches) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
if m.CountMinSketches != nil {
{
size, err := m.CountMinSketches.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintQueryrange(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x72
}
return len(dAtA) - i, nil
}
func (m *QueryRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -5383,6 +5633,33 @@ func (m *QuantileSketchResponse) Size() (n int) {
return n
}
func (m *CountMinSketchResponse) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Response != nil {
l = m.Response.Size()
n += 1 + l + sovQueryrange(uint64(l))
}
if len(m.Headers) > 0 {
for _, e := range m.Headers {
l = e.Size()
n += 1 + l + sovQueryrange(uint64(l))
}
}
if len(m.Warnings) > 0 {
for _, s := range m.Warnings {
l = len(s)
n += 1 + l + sovQueryrange(uint64(l))
}
}
l = m.Statistics.Size()
n += 1 + l + sovQueryrange(uint64(l))
return n
}
func (m *ShardsResponse) Size() (n int) {
if m == nil {
return 0
@ -5619,6 +5896,18 @@ func (m *QueryResponse_DetectedLabels) Size() (n int) {
}
return n
}
func (m *QueryResponse_CountMinSketches) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.CountMinSketches != nil {
l = m.CountMinSketches.Size()
n += 1 + l + sovQueryrange(uint64(l))
}
return n
}
func (m *QueryRequest) Size() (n int) {
if m == nil {
return 0
@ -5951,6 +6240,19 @@ func (this *QuantileSketchResponse) String() string {
}, "")
return s
}
func (this *CountMinSketchResponse) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&CountMinSketchResponse{`,
`Response:` + fmt.Sprintf("%v", this.Response) + `,`,
`Headers:` + fmt.Sprintf("%v", this.Headers) + `,`,
`Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`,
`Statistics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statistics), "Result", "stats.Result", 1), `&`, ``, 1) + `,`,
`}`,
}, "")
return s
}
func (this *ShardsResponse) String() string {
if this == nil {
return "nil"
@ -6126,6 +6428,16 @@ func (this *QueryResponse_DetectedLabels) String() string {
}, "")
return s
}
func (this *QueryResponse_CountMinSketches) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&QueryResponse_CountMinSketches{`,
`CountMinSketches:` + strings.Replace(fmt.Sprintf("%v", this.CountMinSketches), "CountMinSketchResponse", "CountMinSketchResponse", 1) + `,`,
`}`,
}, "")
return s
}
func (this *QueryRequest) String() string {
if this == nil {
return "nil"
@ -8887,6 +9199,194 @@ func (m *QuantileSketchResponse) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *CountMinSketchResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: CountMinSketchResponse: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: CountMinSketchResponse: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthQueryrange
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthQueryrange
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if m.Response == nil {
m.Response = &github_com_grafana_loki_v3_pkg_logproto.CountMinSketchVector{}
}
if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthQueryrange
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthQueryrange
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Headers = append(m.Headers, github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader{})
if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 3:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Warnings", 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.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex
case 4:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthQueryrange
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthQueryrange
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := m.Statistics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipQueryrange(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthQueryrange
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthQueryrange
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *ShardsResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@ -9864,6 +10364,41 @@ func (m *QueryResponse) Unmarshal(dAtA []byte) error {
}
m.Response = &QueryResponse_DetectedLabels{v}
iNdEx = postIndex
case 14:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field CountMinSketches", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowQueryrange
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthQueryrange
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthQueryrange
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
v := &CountMinSketchResponse{}
if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
m.Response = &QueryResponse_CountMinSketches{v}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipQueryrange(dAtA[iNdEx:])

@ -182,6 +182,16 @@ message QuantileSketchResponse {
stats.Result statistics = 4 [(gogoproto.nullable) = false];
}
message CountMinSketchResponse {
logproto.CountMinSketchVector response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/logproto.CountMinSketchVector"];
repeated definitions.PrometheusResponseHeader Headers = 2 [
(gogoproto.jsontag) = "-",
(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader"
];
repeated string warnings = 3 [(gogoproto.jsontag) = "warnings,omitempty"];
stats.Result statistics = 4 [(gogoproto.nullable) = false];
}
message ShardsResponse {
indexgatewaypb.ShardsResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/logproto.ShardsResponse"];
repeated definitions.PrometheusResponseHeader Headers = 2 [
@ -228,6 +238,7 @@ message QueryResponse {
DetectedFieldsResponse detectedFields = 11;
QueryPatternsResponse patternsResponse = 12;
DetectedLabelsResponse detectedLabels = 13;
CountMinSketchResponse countMinSketches = 14;
}
}

@ -4,6 +4,7 @@ import (
"context"
"fmt"
"net/http"
"slices"
"time"
"github.com/dustin/go-humanize"
@ -216,7 +217,13 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que
}
strategy := version.Strategy(resolver, uint64(ast.limits.TSDBMaxBytesPerShard(tenants[0])))
mapper := logql.NewShardMapper(strategy, ast.metrics, ast.shardAggregation)
// Merge global shard aggregations and tenant overrides.
limitShardAggregation := validation.IntersectionPerTenant(tenants, func(tenant string) []string {
return ast.limits.ShardAggregations(tenant)
})
mergedShardAggregation := slices.Compact(append(limitShardAggregation, ast.shardAggregation...))
mapper := logql.NewShardMapper(strategy, ast.metrics, mergedShardAggregation)
noop, bytesPerShard, parsed, err := mapper.Parse(params.GetExpression())
if err != nil {

@ -1536,6 +1536,10 @@ func (f fakeLimits) TSDBShardingStrategy(string) string {
return logql.PowerOfTwoVersion.String()
}
func (f fakeLimits) ShardAggregations(string) []string {
return nil
}
type ingesterQueryOpts struct {
queryStoreOnly bool
queryIngestersWithin time.Duration

@ -201,7 +201,7 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) {
maxBytesPerShard := validation.SmallestPositiveIntPerTenant(tenantIDs, r.limits.TSDBMaxBytesPerShard)
factor := sharding.GuessShardFactor(combined.Bytes, uint64(maxBytesPerShard), r.maxShards)
var bytesPerShard = combined.Bytes
bytesPerShard := combined.Bytes
if factor > 0 {
bytesPerShard = combined.Bytes / uint64(factor)
}
@ -275,7 +275,6 @@ func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerSh
Query: expr.String(),
TargetBytesPerShard: targetBytesPerShard,
})
if err != nil {
return nil, nil, errors.Wrapf(err, "failed to get shards for expression, got %T: %+v", err, err)
}

@ -1,6 +1,7 @@
package validation
import (
"slices"
"time"
)
@ -87,3 +88,31 @@ func MaxDurationOrZeroPerTenant(tenantIDs []string, f func(string) time.Duration
}
return *result
}
// IntersectionPerTenant is returning the intersection of feature flags. This is useful to determine the minimal
// feature set supported.
func IntersectionPerTenant(tenantIDs []string, f func(string) []string) []string {
var result []string
for _, tenantID := range tenantIDs {
v := f(tenantID)
slices.Sort(v)
if result == nil {
result = v
continue
}
var updatedResult []string
for i, j := 0, 0; i < len(result) && j < len(v); {
if result[i] == v[j] {
updatedResult = append(updatedResult, result[i])
i++
j++
} else if result[i] < v[j] {
i++
} else {
j++
}
}
result = updatedResult
}
return result
}

@ -3,6 +3,8 @@ package validation
import (
"testing"
"time"
"github.com/stretchr/testify/require"
)
// nolint:goconst
@ -220,3 +222,98 @@ func TestMaxDurationOrZeroPerTenant(t *testing.T) {
})
}
}
func TestIntersectionPerTenant(t *testing.T) {
tests := []struct {
name string
tenantIDs []string
f func(string) []string
expected []string
}{
{
name: "no tenants",
tenantIDs: []string{},
f: func(_ string) []string {
return nil
},
expected: []string{},
},
{
name: "single tenant with features",
tenantIDs: []string{"tenant1"},
f: func(tenantID string) []string {
if tenantID == "tenant1" {
return []string{"featureA", "featureB", "featureC"}
}
return nil
},
expected: []string{"featureA", "featureB", "featureC"},
},
{
name: "multiple tenants with common features",
tenantIDs: []string{"tenant1", "tenant2"},
f: func(tenantID string) []string {
if tenantID == "tenant1" {
return []string{"featureA", "featureB", "featureC"}
}
if tenantID == "tenant2" {
return []string{"featureB", "featureC", "featureD"}
}
return nil
},
expected: []string{"featureB", "featureC"},
},
{
name: "multiple tenants with no common features",
tenantIDs: []string{"tenant1", "tenant2"},
f: func(tenantID string) []string {
if tenantID == "tenant1" {
return []string{"featureA"}
}
if tenantID == "tenant2" {
return []string{"featureB"}
}
return nil
},
expected: []string{},
},
{
name: "multiple tenants with overlapping features",
tenantIDs: []string{"tenant1", "tenant2", "tenant3"},
f: func(tenantID string) []string {
if tenantID == "tenant1" {
return []string{"featureA", "featureB"}
}
if tenantID == "tenant2" {
return []string{"featureB", "featureC"}
}
if tenantID == "tenant3" {
return []string{"featureB", "featureD"}
}
return nil
},
expected: []string{"featureB"},
},
{
name: "tenant with empty feature set",
tenantIDs: []string{"tenant1", "tenant2"},
f: func(tenantID string) []string {
if tenantID == "tenant1" {
return []string{"featureA", "featureB"}
}
if tenantID == "tenant2" {
return []string{}
}
return nil
},
expected: []string{},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
actual := IntersectionPerTenant(tt.tenantIDs, tt.f)
require.ElementsMatch(t, actual, tt.expected)
})
}
}

@ -228,6 +228,8 @@ type Limits struct {
IngestionPartitionsTenantShardSize int `yaml:"ingestion_partitions_tenant_shard_size" json:"ingestion_partitions_tenant_shard_size" category:"experimental"`
ShardAggregations []string `yaml:"shard_aggregations,omitempty" json:"shard_aggregations,omitempty" doc:"description=List of LogQL vector and range aggregations that should be sharded."`
PatternIngesterTokenizableJSONFieldsDefault dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_default" json:"pattern_ingester_tokenizable_json_fields_default" doc:"hidden"`
PatternIngesterTokenizableJSONFieldsAppend dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_append" json:"pattern_ingester_tokenizable_json_fields_append" doc:"hidden"`
PatternIngesterTokenizableJSONFieldsDelete dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_delete" json:"pattern_ingester_tokenizable_json_fields_delete" doc:"hidden"`
@ -1086,6 +1088,10 @@ func (o *Overrides) BlockIngestionStatusCode(userID string) int {
return o.getOverridesForUser(userID).BlockIngestionStatusCode
}
func (o *Overrides) ShardAggregations(userID string) []string {
return o.getOverridesForUser(userID).ShardAggregations
}
func (o *Overrides) PatternIngesterTokenizableJSONFields(userID string) []string {
defaultFields := o.getOverridesForUser(userID).PatternIngesterTokenizableJSONFieldsDefault
appendFields := o.getOverridesForUser(userID).PatternIngesterTokenizableJSONFieldsAppend

@ -85,6 +85,8 @@ limits_config:
reject_old_samples: true
reject_old_samples_max_age: 168h
split_queries_by_interval: 15m
shard_aggregations:
- approx_topk
querier:
query_ingesters_within: 2h
multi_tenant_queries_enabled: true

@ -1,6 +1,6 @@
FROM golang:1.23
ENV CGO_ENABLED=0
RUN go install github.com/go-delve/delve/cmd/dlv@v1.22.1
RUN go install github.com/go-delve/delve/cmd/dlv@v1.23.1
FROM alpine:3.20.3

Loading…
Cancel
Save