From dc620e7882ccb09cacfd1e231b92d2e4fe19470b Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Wed, 8 May 2024 14:08:44 -0600 Subject: [PATCH 01/38] feat: collect and serve pre-agg bytes and count * pre-aggregate bytes and count per stream in the pattern ingester * serve bytes_over_time and count_over_time queries from the patterns endpoint --- cmd/loki/loki-local-config.yaml | 3 + pkg/logproto/extensions.go | 18 +- pkg/logproto/extensions_test.go | 69 +++++ pkg/logproto/pattern.pb.go | 316 +++++++++++++++++++---- pkg/logproto/pattern.proto | 5 +- pkg/logql/evaluator.go | 24 +- pkg/logql/range_vector.go | 49 ++-- pkg/pattern/chunk/util.go | 14 ++ pkg/pattern/drain/chunk.go | 32 ++- pkg/pattern/drain/chunk_test.go | 32 ++- pkg/pattern/flush_test.go | 4 +- pkg/pattern/ingester.go | 74 +++++- pkg/pattern/ingester_querier.go | 51 +++- pkg/pattern/ingester_querier_test.go | 6 +- pkg/pattern/ingester_test.go | 265 +++++++++++++++++-- pkg/pattern/instance.go | 57 ++++- pkg/pattern/instance_test.go | 115 +++++++++ pkg/pattern/iter/batch.go | 46 +++- pkg/pattern/iter/batch_test.go | 20 +- pkg/pattern/iter/iterator.go | 176 ++++++++++++- pkg/pattern/iter/iterator_test.go | 135 +++++++--- pkg/pattern/iter/merge.go | 14 +- pkg/pattern/iter/merge_test.go | 188 +++++++++----- pkg/pattern/iter/query_client.go | 18 +- pkg/pattern/metric/chunk.go | 201 +++++++++++++++ pkg/pattern/metric/chunk_test.go | 329 ++++++++++++++++++++++++ pkg/pattern/metric/evaluator.go | 354 ++++++++++++++++++++++++++ pkg/pattern/metric/evaluator_test.go | 363 +++++++++++++++++++++++++++ pkg/pattern/stream.go | 155 ++++++++++++ pkg/pattern/stream_test.go | 4 +- pkg/util/marshal/marshal.go | 10 +- pkg/util/marshal/marshal_test.go | 82 ++++-- 32 files changed, 2921 insertions(+), 308 deletions(-) create mode 100644 pkg/pattern/chunk/util.go create mode 100644 pkg/pattern/instance_test.go create mode 100644 pkg/pattern/metric/chunk.go create mode 100644 pkg/pattern/metric/chunk_test.go create mode 100644 pkg/pattern/metric/evaluator.go create mode 100644 pkg/pattern/metric/evaluator_test.go diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 03b579647753..5f717a3d6a81 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -33,6 +33,9 @@ schema_config: prefix: index_ period: 24h +pattern_ingester: + enabled: true + ruler: alertmanager_url: http://localhost:9093 diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index 3de5c0fd7580..bd726bbb0d35 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -161,7 +161,8 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { var v struct { Status string `json:"status"` Data []struct { - Pattern string `json:"pattern"` + Pattern string `json:"pattern,omitempty"` + Labels string `json:"labels,omitempty"` Samples [][]int64 `json:"samples"` } `json:"data"` } @@ -174,7 +175,12 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { for _, s := range d.Samples { samples = append(samples, &PatternSample{Timestamp: model.TimeFromUnix(s[0]), Value: s[1]}) } - r.Series = append(r.Series, &PatternSeries{Pattern: d.Pattern, Samples: samples}) + + if pattern := d.Pattern; pattern != "" { + r.Series = append(r.Series, NewPatternSeriesWithPattern(pattern, samples)) + } else if labels := d.Labels; labels != "" { + r.Series = append(r.Series, NewPatternSeriesWithLabels(labels, samples)) + } } return nil } @@ -188,3 +194,11 @@ func (m *ShardsResponse) Merge(other *ShardsResponse) { m.ChunkGroups = append(m.ChunkGroups, other.ChunkGroups...) m.Statistics.Merge(other.Statistics) } + +func NewPatternSeriesWithPattern(pattern string, samples []*PatternSample) *PatternSeries { + return &PatternSeries{Identifier: &PatternSeries_Pattern{pattern}, Samples: samples} +} + +func NewPatternSeriesWithLabels(labels string, samples []*PatternSample) *PatternSeries { + return &PatternSeries{Identifier: &PatternSeries_Labels{labels}, Samples: samples} +} diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go index d1c96c76bbed..de8e84e34cd3 100644 --- a/pkg/logproto/extensions_test.go +++ b/pkg/logproto/extensions_test.go @@ -3,6 +3,7 @@ package logproto import ( "testing" + "github.com/prometheus/common/model" "github.com/stretchr/testify/require" ) @@ -40,3 +41,71 @@ func TestShard_SpaceFor(t *testing.T) { }) } } + +func TestQueryPatternsResponse_UnmarshalJSON(t *testing.T) { + t.Run("unmarshals patterns", func(t *testing.T) { + mockData := []byte(`{ + "status": "success", + "data": [ + { + "pattern": "foo <*> bar", + "samples": [[1609459200, 10], [1609545600, 15]] + }, + { + "pattern": "foo <*> buzz", + "samples": [[1609459200, 20], [1609545600, 25]] + } + ] + }`) + + expectedSeries := []*PatternSeries{ + NewPatternSeriesWithPattern("foo <*> bar", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, + }), + NewPatternSeriesWithPattern("foo <*> buzz", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, + }), + } + + r := &QueryPatternsResponse{} + err := r.UnmarshalJSON(mockData) + + require.Nil(t, err) + require.Equal(t, expectedSeries, r.Series) + }) + + t.Run("unmarshals labels", func(t *testing.T) { + mockData := []byte(`{ + "status": "success", + "data": [ + { + "labels": "{foo=\"bar\"}", + "samples": [[1609459200, 10], [1609545600, 15]] + }, + { + "labels": "{foo=\"buzz\"}", + "samples": [[1609459200, 20], [1609545600, 25]] + } + ] + }`) + + expectedSeries := []*PatternSeries{ + NewPatternSeriesWithLabels(`{foo="bar"}`, []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, + }), + NewPatternSeriesWithLabels(`{foo="buzz"}`, []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, + }), + } + + r := &QueryPatternsResponse{} + err := r.UnmarshalJSON(mockData) + + require.Nil(t, err) + require.Equal(t, expectedSeries, r.Series) + }) +} diff --git a/pkg/logproto/pattern.pb.go b/pkg/logproto/pattern.pb.go index a666a3285012..facf0b4bfa90 100644 --- a/pkg/logproto/pattern.pb.go +++ b/pkg/logproto/pattern.pb.go @@ -146,8 +146,11 @@ func (m *QueryPatternsResponse) GetSeries() []*PatternSeries { } type PatternSeries struct { - Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3" json:"pattern,omitempty"` - Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` + // Types that are valid to be assigned to Identifier: + // *PatternSeries_Pattern + // *PatternSeries_Labels + Identifier isPatternSeries_Identifier `protobuf_oneof:"identifier"` + Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` } func (m *PatternSeries) Reset() { *m = PatternSeries{} } @@ -182,9 +185,40 @@ func (m *PatternSeries) XXX_DiscardUnknown() { var xxx_messageInfo_PatternSeries proto.InternalMessageInfo -func (m *PatternSeries) GetPattern() string { +type isPatternSeries_Identifier interface { + isPatternSeries_Identifier() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type PatternSeries_Pattern struct { + Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3,oneof"` +} +type PatternSeries_Labels struct { + Labels string `protobuf:"bytes,3,opt,name=labels,proto3,oneof"` +} + +func (*PatternSeries_Pattern) isPatternSeries_Identifier() {} +func (*PatternSeries_Labels) isPatternSeries_Identifier() {} + +func (m *PatternSeries) GetIdentifier() isPatternSeries_Identifier { if m != nil { - return m.Pattern + return m.Identifier + } + return nil +} + +func (m *PatternSeries) GetPattern() string { + if x, ok := m.GetIdentifier().(*PatternSeries_Pattern); ok { + return x.Pattern + } + return "" +} + +func (m *PatternSeries) GetLabels() string { + if x, ok := m.GetIdentifier().(*PatternSeries_Labels); ok { + return x.Labels } return "" } @@ -196,6 +230,14 @@ func (m *PatternSeries) GetSamples() []*PatternSample { return nil } +// XXX_OneofWrappers is for the internal use of the proto package. +func (*PatternSeries) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*PatternSeries_Pattern)(nil), + (*PatternSeries_Labels)(nil), + } +} + type PatternSample struct { Timestamp github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=timestamp,proto3,customtype=github.com/prometheus/common/model.Time" json:"timestamp"` Value int64 `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` @@ -250,38 +292,39 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/pattern.proto", fileDescriptor_aaf4192acc66a4ea) } var fileDescriptor_aaf4192acc66a4ea = []byte{ - // 483 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x52, 0xb1, 0x6e, 0xd3, 0x40, - 0x18, 0xf6, 0xd5, 0x49, 0xd3, 0x5e, 0xc5, 0x72, 0xa4, 0x60, 0x19, 0xe9, 0x1c, 0x79, 0x21, 0x93, - 0x0f, 0x52, 0x09, 0x24, 0xc6, 0x4c, 0x0c, 0x20, 0x15, 0xc3, 0x84, 0x60, 0x70, 0xda, 0xbf, 0xb6, - 0x55, 0xdb, 0xe7, 0xfa, 0xee, 0x2a, 0xb1, 0xf1, 0x08, 0x79, 0x0c, 0x1e, 0x80, 0x87, 0xe8, 0x98, - 0xb1, 0x62, 0x28, 0xc4, 0x59, 0x18, 0xfb, 0x08, 0xc8, 0x77, 0x76, 0x93, 0x56, 0x74, 0xe8, 0x92, - 0xdc, 0xff, 0x7f, 0xdf, 0xff, 0xf9, 0xbb, 0xff, 0x3b, 0xec, 0x96, 0xa7, 0x31, 0xcb, 0x78, 0x5c, - 0x56, 0x5c, 0x72, 0x56, 0x46, 0x52, 0x42, 0x55, 0x04, 0xba, 0x22, 0x3b, 0x5d, 0xdf, 0x1d, 0xc6, - 0x3c, 0xe6, 0x86, 0xd2, 0x9c, 0x0c, 0xee, 0x7a, 0x31, 0xe7, 0x71, 0x06, 0x4c, 0x57, 0x33, 0x75, - 0xc2, 0x64, 0x9a, 0x83, 0x90, 0x51, 0x5e, 0xb6, 0x84, 0x67, 0xb7, 0xc4, 0xbb, 0x43, 0x0b, 0x3e, - 0x6e, 0xc0, 0x52, 0x89, 0x44, 0xff, 0x98, 0xa6, 0xff, 0x13, 0xe1, 0xe1, 0x07, 0x05, 0xd5, 0xb7, - 0x43, 0xe3, 0x44, 0x84, 0x70, 0xa6, 0x40, 0x48, 0x32, 0xc4, 0xfd, 0xb3, 0xa6, 0xef, 0xa0, 0x11, - 0x1a, 0xef, 0x86, 0xa6, 0x20, 0x6f, 0x70, 0x5f, 0xc8, 0xa8, 0x92, 0xce, 0xd6, 0x08, 0x8d, 0xf7, - 0x26, 0x6e, 0x60, 0x1c, 0x05, 0x9d, 0xa3, 0xe0, 0x53, 0xe7, 0x68, 0xba, 0x73, 0x71, 0xe5, 0x59, - 0xf3, 0xdf, 0x1e, 0x0a, 0xcd, 0x08, 0x79, 0x85, 0x6d, 0x28, 0x8e, 0x1d, 0xfb, 0x01, 0x93, 0xcd, - 0x00, 0x21, 0xb8, 0x27, 0x24, 0x94, 0x4e, 0x6f, 0x84, 0xc6, 0x76, 0xa8, 0xcf, 0xfe, 0x5b, 0xbc, - 0x7f, 0xc7, 0xb5, 0x28, 0x79, 0x21, 0x80, 0x30, 0xbc, 0x2d, 0xa0, 0x4a, 0x41, 0x38, 0x68, 0x64, - 0x8f, 0xf7, 0x26, 0x4f, 0x83, 0x9b, 0x2d, 0xb4, 0xdc, 0x8f, 0x1a, 0x0e, 0x5b, 0x9a, 0xff, 0x05, - 0x3f, 0xba, 0x05, 0x10, 0x07, 0x0f, 0xda, 0x54, 0xda, 0xab, 0x77, 0x25, 0x79, 0x89, 0x07, 0x22, - 0xca, 0xcb, 0x0c, 0x84, 0xb3, 0x75, 0x9f, 0xb8, 0xc6, 0xc3, 0x8e, 0xe7, 0xcb, 0xb5, 0xba, 0xee, - 0x90, 0xf7, 0x78, 0xf7, 0x26, 0x34, 0xad, 0x6f, 0x4f, 0x59, 0x73, 0xdd, 0x5f, 0x57, 0xde, 0xf3, - 0x38, 0x95, 0x89, 0x9a, 0x05, 0x47, 0x3c, 0x6f, 0x12, 0xce, 0x41, 0x26, 0xa0, 0x04, 0x3b, 0xe2, - 0x79, 0xce, 0x0b, 0x96, 0xf3, 0x63, 0xc8, 0xf4, 0x92, 0xc2, 0xb5, 0x42, 0x93, 0xd2, 0x79, 0x94, - 0x29, 0xd0, 0x79, 0xd8, 0xa1, 0x29, 0x26, 0x73, 0x84, 0x07, 0xed, 0x67, 0xc9, 0x6b, 0xdc, 0x3b, - 0x54, 0x22, 0x21, 0xfb, 0x1b, 0x5e, 0x95, 0x48, 0xda, 0x98, 0xdd, 0x27, 0x77, 0xdb, 0x66, 0x8f, - 0xbe, 0x45, 0xde, 0xe1, 0xbe, 0x5e, 0x31, 0xa1, 0x6b, 0xca, 0xff, 0x5e, 0x8a, 0xeb, 0xdd, 0x8b, - 0x77, 0x5a, 0x2f, 0xd0, 0xf4, 0xeb, 0x62, 0x49, 0xad, 0xcb, 0x25, 0xb5, 0xae, 0x97, 0x14, 0x7d, - 0xaf, 0x29, 0xfa, 0x51, 0x53, 0x74, 0x51, 0x53, 0xb4, 0xa8, 0x29, 0xfa, 0x53, 0x53, 0xf4, 0xb7, - 0xa6, 0xd6, 0x75, 0x4d, 0xd1, 0x7c, 0x45, 0xad, 0xc5, 0x8a, 0x5a, 0x97, 0x2b, 0x6a, 0x7d, 0xde, - 0x5c, 0x49, 0x5c, 0x45, 0x27, 0x51, 0x11, 0xb1, 0x8c, 0x9f, 0xa6, 0xec, 0xfc, 0x80, 0x6d, 0x3e, - 0xf5, 0xd9, 0xb6, 0xfe, 0x3b, 0xf8, 0x17, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x4f, 0x5c, 0x50, 0x5e, - 0x03, 0x00, 0x00, + // 511 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0xbd, 0x6e, 0xd4, 0x40, + 0x10, 0xf6, 0xc6, 0xf7, 0x93, 0x6c, 0xa0, 0x59, 0x2e, 0x60, 0x19, 0x69, 0x7d, 0x72, 0xc3, 0x55, + 0x5e, 0xb8, 0x48, 0x20, 0x51, 0x5e, 0x95, 0x02, 0xa4, 0x60, 0xa8, 0x90, 0x28, 0x7c, 0xb9, 0x39, + 0xdb, 0x8a, 0xed, 0x75, 0xbc, 0xeb, 0x48, 0x74, 0x54, 0xd4, 0xf7, 0x18, 0x3c, 0x00, 0x0f, 0x91, + 0xf2, 0xca, 0x88, 0x22, 0x70, 0xbe, 0x86, 0x32, 0x8f, 0x80, 0xbc, 0x6b, 0xe7, 0x2e, 0x11, 0x29, + 0xd2, 0xd8, 0x33, 0xf3, 0x7d, 0x33, 0xfb, 0xed, 0xcc, 0x2c, 0xb6, 0xf3, 0xd3, 0x90, 0x25, 0x3c, + 0xcc, 0x0b, 0x2e, 0x39, 0xcb, 0x03, 0x29, 0xa1, 0xc8, 0x3c, 0xe5, 0x91, 0xdd, 0x36, 0x6e, 0x0f, + 0x42, 0x1e, 0x72, 0x4d, 0xa9, 0x2d, 0x8d, 0xdb, 0x4e, 0xc8, 0x79, 0x98, 0x00, 0x53, 0xde, 0xb4, + 0x9c, 0x33, 0x19, 0xa7, 0x20, 0x64, 0x90, 0xe6, 0x0d, 0xe1, 0xf9, 0xad, 0xe2, 0xad, 0xd1, 0x80, + 0x4f, 0x6a, 0x30, 0x2f, 0x45, 0xa4, 0x3e, 0x3a, 0xe8, 0xfe, 0x44, 0x78, 0xf0, 0xa1, 0x84, 0xe2, + 0xeb, 0xb1, 0x56, 0x22, 0x7c, 0x38, 0x2b, 0x41, 0x48, 0x32, 0xc0, 0xdd, 0xb3, 0x3a, 0x6e, 0xa1, + 0x21, 0x1a, 0xed, 0xf9, 0xda, 0x21, 0x6f, 0x71, 0x57, 0xc8, 0xa0, 0x90, 0xd6, 0xce, 0x10, 0x8d, + 0xf6, 0xc7, 0xb6, 0xa7, 0x15, 0x79, 0xad, 0x22, 0xef, 0x53, 0xab, 0x68, 0xb2, 0x7b, 0x71, 0xe5, + 0x18, 0x8b, 0xdf, 0x0e, 0xf2, 0x75, 0x0a, 0x79, 0x8d, 0x4d, 0xc8, 0x66, 0x96, 0xf9, 0x80, 0xcc, + 0x3a, 0x81, 0x10, 0xdc, 0x11, 0x12, 0x72, 0xab, 0x33, 0x44, 0x23, 0xd3, 0x57, 0xb6, 0x7b, 0x84, + 0x0f, 0xee, 0xa8, 0x16, 0x39, 0xcf, 0x04, 0x10, 0x86, 0x7b, 0x02, 0x8a, 0x18, 0x84, 0x85, 0x86, + 0xe6, 0x68, 0x7f, 0xfc, 0xcc, 0xbb, 0xe9, 0x42, 0xc3, 0xfd, 0xa8, 0x60, 0xbf, 0xa1, 0xb9, 0xdf, + 0x11, 0x7e, 0x7c, 0x0b, 0x21, 0x36, 0xee, 0x37, 0x63, 0xd1, 0x77, 0x3f, 0x32, 0xfc, 0x36, 0x40, + 0x2c, 0xdc, 0x4b, 0x82, 0x29, 0x24, 0x42, 0x5d, 0xa3, 0x86, 0x1a, 0x9f, 0xbc, 0xc2, 0x7d, 0x11, + 0xa4, 0x79, 0x02, 0xc2, 0xda, 0xb9, 0xef, 0x64, 0x85, 0xfb, 0x2d, 0x6f, 0xf2, 0x08, 0xe3, 0x78, + 0x06, 0x99, 0x8c, 0xe7, 0x31, 0x14, 0xae, 0xdc, 0xe8, 0x50, 0x38, 0x79, 0x8f, 0xf7, 0x6e, 0xe6, + 0xab, 0x94, 0x98, 0x13, 0x56, 0x77, 0xe6, 0xd7, 0x95, 0xf3, 0x22, 0x8c, 0x65, 0x54, 0x4e, 0xbd, + 0x13, 0x9e, 0xd6, 0xcb, 0x90, 0x82, 0x8c, 0xa0, 0x14, 0xec, 0x84, 0xa7, 0x29, 0xcf, 0x58, 0xca, + 0x67, 0x90, 0xa8, 0x7e, 0xfa, 0x9b, 0x0a, 0xf5, 0x40, 0xcf, 0x83, 0xa4, 0x04, 0x35, 0x3a, 0xd3, + 0xd7, 0xce, 0x78, 0x81, 0x70, 0xbf, 0x39, 0x96, 0xbc, 0xc1, 0x9d, 0xe3, 0x52, 0x44, 0xe4, 0x60, + 0x4b, 0x79, 0x29, 0xa2, 0x66, 0x23, 0xec, 0xa7, 0x77, 0xc3, 0xba, 0xe5, 0xae, 0x41, 0xde, 0xe1, + 0xae, 0x9a, 0x06, 0xa1, 0x1b, 0xca, 0xff, 0x96, 0xca, 0x76, 0xee, 0xc5, 0xdb, 0x5a, 0x2f, 0xd1, + 0xe4, 0xcb, 0x72, 0x45, 0x8d, 0xcb, 0x15, 0x35, 0xae, 0x57, 0x14, 0x7d, 0xab, 0x28, 0xfa, 0x51, + 0x51, 0x74, 0x51, 0x51, 0xb4, 0xac, 0x28, 0xfa, 0x53, 0x51, 0xf4, 0xb7, 0xa2, 0xc6, 0x75, 0x45, + 0xd1, 0x62, 0x4d, 0x8d, 0xe5, 0x9a, 0x1a, 0x97, 0x6b, 0x6a, 0x7c, 0xde, 0x6e, 0x49, 0x58, 0x04, + 0xf3, 0x20, 0x0b, 0x58, 0xc2, 0x4f, 0x63, 0x76, 0x7e, 0xc8, 0xb6, 0x5f, 0xc5, 0xb4, 0xa7, 0x7e, + 0x87, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x1c, 0x3a, 0x0e, 0x65, 0x89, 0x03, 0x00, 0x00, } func (this *QueryPatternsRequest) Equal(that interface{}) bool { @@ -365,7 +408,13 @@ func (this *PatternSeries) Equal(that interface{}) bool { } else if this == nil { return false } - if this.Pattern != that1.Pattern { + if that1.Identifier == nil { + if this.Identifier != nil { + return false + } + } else if this.Identifier == nil { + return false + } else if !this.Identifier.Equal(that1.Identifier) { return false } if len(this.Samples) != len(that1.Samples) { @@ -378,6 +427,54 @@ func (this *PatternSeries) Equal(that interface{}) bool { } return true } +func (this *PatternSeries_Pattern) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PatternSeries_Pattern) + if !ok { + that2, ok := that.(PatternSeries_Pattern) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Pattern != that1.Pattern { + return false + } + return true +} +func (this *PatternSeries_Labels) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PatternSeries_Labels) + if !ok { + that2, ok := that.(PatternSeries_Labels) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Labels != that1.Labels { + return false + } + return true +} func (this *PatternSample) Equal(that interface{}) bool { if that == nil { return this == nil @@ -434,15 +531,33 @@ func (this *PatternSeries) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) + s := make([]string, 0, 7) s = append(s, "&logproto.PatternSeries{") - s = append(s, "Pattern: "+fmt.Sprintf("%#v", this.Pattern)+",\n") + if this.Identifier != nil { + s = append(s, "Identifier: "+fmt.Sprintf("%#v", this.Identifier)+",\n") + } if this.Samples != nil { s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") } s = append(s, "}") return strings.Join(s, "") } +func (this *PatternSeries_Pattern) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&logproto.PatternSeries_Pattern{` + + `Pattern:` + fmt.Sprintf("%#v", this.Pattern) + `}`}, ", ") + return s +} +func (this *PatternSeries_Labels) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&logproto.PatternSeries_Labels{` + + `Labels:` + fmt.Sprintf("%#v", this.Labels) + `}`}, ", ") + return s +} func (this *PatternSample) GoString() string { if this == nil { return "nil" @@ -715,6 +830,15 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Identifier != nil { + { + size := m.Identifier.Size() + i -= size + if _, err := m.Identifier.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } if len(m.Samples) > 0 { for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { { @@ -729,16 +853,35 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x12 } } - if len(m.Pattern) > 0 { - i -= len(m.Pattern) - copy(dAtA[i:], m.Pattern) - i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) - i-- - dAtA[i] = 0xa - } return len(dAtA) - i, nil } +func (m *PatternSeries_Pattern) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PatternSeries_Pattern) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Pattern) + copy(dAtA[i:], m.Pattern) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} +func (m *PatternSeries_Labels) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PatternSeries_Labels) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Labels) + copy(dAtA[i:], m.Labels) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Labels))) + i-- + dAtA[i] = 0x1a + return len(dAtA) - i, nil +} func (m *PatternSample) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -824,9 +967,8 @@ func (m *PatternSeries) Size() (n int) { } var l int _ = l - l = len(m.Pattern) - if l > 0 { - n += 1 + l + sovPattern(uint64(l)) + if m.Identifier != nil { + n += m.Identifier.Size() } if len(m.Samples) > 0 { for _, e := range m.Samples { @@ -837,6 +979,26 @@ func (m *PatternSeries) Size() (n int) { return n } +func (m *PatternSeries_Pattern) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Pattern) + n += 1 + l + sovPattern(uint64(l)) + return n +} +func (m *PatternSeries_Labels) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Labels) + n += 1 + l + sovPattern(uint64(l)) + return n +} func (m *PatternSample) Size() (n int) { if m == nil { return 0 @@ -896,12 +1058,32 @@ func (this *PatternSeries) String() string { } repeatedStringForSamples += "}" s := strings.Join([]string{`&PatternSeries{`, - `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, + `Identifier:` + fmt.Sprintf("%v", this.Identifier) + `,`, `Samples:` + repeatedStringForSamples + `,`, `}`, }, "") return s } +func (this *PatternSeries_Pattern) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PatternSeries_Pattern{`, + `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, + `}`, + }, "") + return s +} +func (this *PatternSeries_Labels) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PatternSeries_Labels{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `}`, + }, "") + return s +} func (this *PatternSample) String() string { if this == nil { return "nil" @@ -1237,7 +1419,7 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Pattern = string(dAtA[iNdEx:postIndex]) + m.Identifier = &PatternSeries_Pattern{string(dAtA[iNdEx:postIndex])} iNdEx = postIndex case 2: if wireType != 2 { @@ -1273,6 +1455,38 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + 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 ErrInvalidLengthPattern + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Identifier = &PatternSeries_Labels{string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPattern(dAtA[iNdEx:]) diff --git a/pkg/logproto/pattern.proto b/pkg/logproto/pattern.proto index e92a201b3a8b..fa03742b1af7 100644 --- a/pkg/logproto/pattern.proto +++ b/pkg/logproto/pattern.proto @@ -32,7 +32,10 @@ message QueryPatternsResponse { } message PatternSeries { - string pattern = 1; + oneof identifier { + string pattern = 1; + string labels = 3; + } repeated PatternSample samples = 2; } diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index e50d8739c30a..b44b134c8bdd 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -410,6 +410,20 @@ type VectorAggEvaluator struct { lb *labels.Builder } +func NewVectorAggEvaluator( + nextEvaluator StepEvaluator, + expr *syntax.VectorAggregationExpr, + buf []byte, + lb *labels.Builder, +) *VectorAggEvaluator { + return &VectorAggEvaluator{ + nextEvaluator: nextEvaluator, + expr: expr, + buf: buf, + lb: lb, + } +} + func (e *VectorAggEvaluator) Next() (bool, int64, StepResult) { next, ts, r := e.nextEvaluator.Next() @@ -684,9 +698,7 @@ func newRangeAggEvaluator( return nil, err } - return &RangeVectorEvaluator{ - iter: iter, - }, nil + return NewRangeVectorEvaluator(iter), nil } } @@ -696,6 +708,12 @@ type RangeVectorEvaluator struct { err error } +func NewRangeVectorEvaluator(iter RangeVectorIterator) *RangeVectorEvaluator { + return &RangeVectorEvaluator{ + iter: iter, + } +} + func (r *RangeVectorEvaluator) Next() (bool, int64, StepResult) { next := r.iter.Next() if !next { diff --git a/pkg/logql/range_vector.go b/pkg/logql/range_vector.go index 44a865157754..200f1480add7 100644 --- a/pkg/logql/range_vector.go +++ b/pkg/logql/range_vector.go @@ -75,21 +75,18 @@ func newRangeVectorIterator( if err != nil { return nil, err } - return &batchRangeVectorIterator{ - iter: it, - step: step, - end: end, - selRange: selRange, - metrics: map[string]labels.Labels{}, - window: map[string]*promql.Series{}, - agg: vectorAggregator, - current: start - step, // first loop iteration will set it to start - offset: offset, - }, nil -} - -//batch - + return NewBatchRangeVectorIterator( + it, + selRange, + step, + start, + end, + offset, + vectorAggregator, + ), nil +} + +// batch type batchRangeVectorIterator struct { iter iter.PeekingSampleIterator selRange, step, end, current, offset int64 @@ -99,14 +96,32 @@ type batchRangeVectorIterator struct { agg BatchRangeVectorAggregator } +func NewBatchRangeVectorIterator( + it iter.PeekingSampleIterator, + selRange, step, start, end, offset int64, + agg BatchRangeVectorAggregator, +) RangeVectorIterator { + return &batchRangeVectorIterator{ + iter: it, + selRange: selRange, + step: step, + end: end, + current: start - step, // first loop iteration will set it to start + offset: offset, + metrics: map[string]labels.Labels{}, + window: map[string]*promql.Series{}, + agg: agg, + } +} + func (r *batchRangeVectorIterator) Next() bool { // slides the range window to the next position - r.current = r.current + r.step + r.current = r.current + r.step // first current will be 5 min before start if r.current > r.end { return false } rangeEnd := r.current - rangeStart := rangeEnd - r.selRange + rangeStart := rangeEnd - r.selRange // in nanoseconds // load samples r.popBack(rangeStart) r.load(rangeStart, rangeEnd) diff --git a/pkg/pattern/chunk/util.go b/pkg/pattern/chunk/util.go new file mode 100644 index 000000000000..8cbde3fb0474 --- /dev/null +++ b/pkg/pattern/chunk/util.go @@ -0,0 +1,14 @@ +package chunk + +import ( + "time" + + "github.com/prometheus/common/model" +) + +const ( + TimeResolution = model.Time(int64(time.Second*10) / 1e6) + MaxChunkTime = 1 * time.Hour +) + +func TruncateTimestamp(ts, step model.Time) model.Time { return ts - ts%step } diff --git a/pkg/pattern/drain/chunk.go b/pkg/pattern/drain/chunk.go index 9b1e34e2e3a1..8be0dd64070f 100644 --- a/pkg/pattern/drain/chunk.go +++ b/pkg/pattern/drain/chunk.go @@ -7,15 +7,12 @@ import ( "github.com/prometheus/common/model" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/iter" ) const ( - TimeResolution = model.Time(int64(time.Second*10) / 1e6) - defaultVolumeSize = 500 - - maxChunkTime = 1 * time.Hour ) type Chunks []Chunk @@ -25,7 +22,7 @@ type Chunk struct { } func newChunk(ts model.Time) Chunk { - maxSize := int(maxChunkTime.Nanoseconds()/TimeResolution.UnixNano()) + 1 + maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 v := Chunk{Samples: make([]logproto.PatternSample, 1, maxSize)} v.Samples[0] = logproto.PatternSample{ Timestamp: ts, @@ -39,11 +36,11 @@ func (c Chunk) spaceFor(ts model.Time) bool { return true } - return ts.Sub(c.Samples[0].Timestamp) < maxChunkTime + return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime } // ForRange returns samples with only the values -// in the given range [start:end) and aggregates them by step duration. +// in the given range [start:end] and aggregates them by step duration. // start and end are in milliseconds since epoch. step is a duration in milliseconds. func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { if len(c.Samples) == 0 { @@ -51,7 +48,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { } first := c.Samples[0].Timestamp last := c.Samples[len(c.Samples)-1].Timestamp - if start >= end || first >= end || last < start { + if start >= end || first > end || last < start { return nil } var lo int @@ -61,17 +58,18 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { }) } hi := len(c.Samples) - if end < last { + + if end <= last { hi = sort.Search(len(c.Samples), func(i int) bool { - return c.Samples[i].Timestamp >= end + return c.Samples[i].Timestamp > end }) } - if step == TimeResolution { + if step == chunk.TimeResolution { return c.Samples[lo:hi] } // Re-scale samples into step-sized buckets - currentStep := truncateTimestamp(c.Samples[lo].Timestamp, step) + currentStep := chunk.TruncateTimestamp(c.Samples[lo].Timestamp, step) aggregatedSamples := make([]logproto.PatternSample, 0, ((c.Samples[hi-1].Timestamp-currentStep)/step)+1) aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ Timestamp: currentStep, @@ -79,7 +77,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { }) for _, sample := range c.Samples[lo:hi] { if sample.Timestamp >= currentStep+step { - stepForSample := truncateTimestamp(sample.Timestamp, step) + stepForSample := chunk.TruncateTimestamp(sample.Timestamp, step) for i := currentStep + step; i <= stepForSample; i += step { aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ Timestamp: i, @@ -95,7 +93,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { } func (c *Chunks) Add(ts model.Time) { - t := truncateTimestamp(ts, TimeResolution) + t := chunk.TruncateTimestamp(ts, chunk.TimeResolution) if len(*c) == 0 { *c = append(*c, newChunk(t)) @@ -123,9 +121,9 @@ func (c Chunks) Iterator(pattern string, from, through, step model.Time) iter.It if len(samples) == 0 { continue } - iters = append(iters, iter.NewSlice(pattern, samples)) + iters = append(iters, iter.NewPatternSlice(pattern, samples)) } - return iter.NewNonOverlappingIterator(pattern, iters) + return iter.NewNonOverlappingPatternIterator(pattern, iters) } func (c Chunks) samples() []*logproto.PatternSample { @@ -197,5 +195,3 @@ func (c *Chunks) size() int { } return size } - -func truncateTimestamp(ts, step model.Time) model.Time { return ts - ts%step } diff --git a/pkg/pattern/drain/chunk_test.go b/pkg/pattern/drain/chunk_test.go index 4863a6629729..e404a9b5da77 100644 --- a/pkg/pattern/drain/chunk_test.go +++ b/pkg/pattern/drain/chunk_test.go @@ -9,28 +9,29 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" ) func TestAdd(t *testing.T) { cks := Chunks{} - cks.Add(TimeResolution + 1) - cks.Add(TimeResolution + 2) - cks.Add(2*TimeResolution + 1) + cks.Add(chunk.TimeResolution + 1) + cks.Add(chunk.TimeResolution + 2) + cks.Add(2*chunk.TimeResolution + 1) require.Equal(t, 1, len(cks)) require.Equal(t, 2, len(cks[0].Samples)) - cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + TimeResolution + 1) + cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + chunk.TimeResolution + 1) require.Equal(t, 2, len(cks)) require.Equal(t, 1, len(cks[1].Samples)) } func TestIterator(t *testing.T) { cks := Chunks{} - cks.Add(TimeResolution + 1) - cks.Add(TimeResolution + 2) - cks.Add(2*TimeResolution + 1) - cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + TimeResolution + 1) + cks.Add(chunk.TimeResolution + 1) + cks.Add(chunk.TimeResolution + 2) + cks.Add(2*chunk.TimeResolution + 1) + cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + chunk.TimeResolution + 1) - it := cks.Iterator("test", model.Time(0), model.Time(time.Hour.Nanoseconds()), TimeResolution) + it := cks.Iterator("test", model.Time(0), model.Time(time.Hour.Nanoseconds()), chunk.TimeResolution) require.NotNil(t, it) var samples []logproto.PatternSample @@ -137,7 +138,7 @@ func TestForRange(t *testing.T) { }, }, { - name: "Start and End Before First Element", + name: "Start before First and End Inclusive of First Element", c: &Chunk{Samples: []logproto.PatternSample{ {Timestamp: 2, Value: 2}, {Timestamp: 4, Value: 4}, @@ -145,6 +146,17 @@ func TestForRange(t *testing.T) { }}, start: 0, end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start and End before First Element", + c: &Chunk{Samples: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }}, + start: 0, + end: 1, expected: nil, }, { diff --git a/pkg/pattern/flush_test.go b/pkg/pattern/flush_test.go index 9ee4bd436992..d2ed2384522d 100644 --- a/pkg/pattern/flush_test.go +++ b/pkg/pattern/flush_test.go @@ -62,7 +62,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 2, len(res.Series)) ing.sweepUsers(true, true) @@ -72,7 +72,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err = iter.ReadAll(it) + res, err = iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) } diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 1cb91a1cda29..e6174faffb6c 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -12,6 +12,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/tenant" @@ -21,6 +22,7 @@ import ( ring_client "github.com/grafana/dskit/ring/client" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/clientpool" "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/util" @@ -238,17 +240,77 @@ func (i *Ingester) Query(req *logproto.QueryPatternsRequest, stream logproto.Pat if err != nil { return err } - iterator, err := instance.Iterator(ctx, req) - if err != nil { - return err + + expr, err := syntax.ParseExpr(req.Query) + + switch e := expr.(type) { + case syntax.SampleExpr: + var err error + iterator, err := instance.QuerySample(ctx, e, req) // this is returning a first value of 0,0 + if err != nil { + return err + } + + // TODO(twhitney): query store + // if start, end, ok := buildStoreRequest(i.cfg, req.Start, req.End, time.Now()); ok { + // storeReq := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ + // Start: start, + // End: end, + // Selector: req.Selector, + // Shards: req.Shards, + // Deletes: req.Deletes, + // Plan: req.Plan, + // }} + // storeItr, err := i.store.SelectSamples(ctx, storeReq) + // if err != nil { + // util.LogErrorWithContext(ctx, "closing iterator", it.Close) + // return err + // } + + // it = iter.NewMergeSampleIterator(ctx, []iter.SampleIterator{it, storeItr}) + // } + + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendMetricsSample(ctx, iterator, stream) + case syntax.LogSelectorExpr: + var err error + iterator, err := instance.Iterator(ctx, req) + if err != nil { + return err + } + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendPatternSample(ctx, iterator, stream) + default: + return httpgrpc.Errorf( + http.StatusBadRequest, + fmt.Sprintf("unexpected type (%T): cannot evaluate", e), + ) } - defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) - return sendPatternSample(ctx, iterator, stream) } func sendPatternSample(ctx context.Context, it iter.Iterator, stream logproto.Pattern_QueryServer) error { for ctx.Err() == nil { - batch, err := iter.ReadBatch(it, readBatchSize) + batch, err := iter.ReadPatternsBatch(it, readBatchSize) + if err != nil { + return err + } + if err := stream.Send(batch); err != nil && err != context.Canceled { + return err + } + if len(batch.Series) == 0 { + return nil + } + } + return nil +} + +func sendMetricsSample( + ctx context.Context, + it iter.Iterator, + stream logproto.Pattern_QueryServer, +) error { + for ctx.Err() == nil { + batch, err := iter.ReadMetricsBatch(it, readBatchSize) if err != nil { return err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 13315b0a13f1..e9b42010a56e 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -2,6 +2,7 @@ package pattern import ( "context" + "errors" "math" "net/http" @@ -19,6 +20,8 @@ import ( // TODO(kolesnikovae): parametrise QueryPatternsRequest const minClusterSize = 30 +var ErrParseQuery = errors.New("only label matcher, byte_over_time, and count_over_time queries without filters are supported") + type IngesterQuerier struct { cfg Config logger log.Logger @@ -44,10 +47,24 @@ func NewIngesterQuerier( } func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { + // validate that a supported query was provided + // TODO(twhitney): validate metric queries don't have filters + var expr syntax.Expr _, err := syntax.ParseMatchers(req.Query, true) if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + expr, err = syntax.ParseSampleExpr(req.Query) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, ErrParseQuery.Error()) + } + + switch expr.(type) { + case *syntax.VectorAggregationExpr, *syntax.RangeAggregationExpr: + break + default: + return nil, ErrParseQuery + } } + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { return client.Query(ctx, req) }) @@ -58,18 +75,30 @@ func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatte for i := range resps { iterators[i] = iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) } - // TODO(kolesnikovae): Incorporate with pruning - resp, err := iter.ReadBatch(iter.NewMerge(iterators...), math.MaxInt32) - if err != nil { - return nil, err + switch expr.(type) { + case *syntax.VectorAggregationExpr, *syntax.RangeAggregationExpr: + resp, err := iter.ReadMetricsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err + } + return resp, nil + default: + // TODO(kolesnikovae): Incorporate with pruning + resp, err := iter.ReadPatternsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err + } + return prunePatterns(resp, minClusterSize), nil } - return prunePatterns(resp, minClusterSize), nil } -func prunePatterns(resp *logproto.QueryPatternsResponse, minClusterSize int) *logproto.QueryPatternsResponse { +func prunePatterns( + resp *logproto.QueryPatternsResponse, + minClusterSize int, +) *logproto.QueryPatternsResponse { d := drain.New(drain.DefaultConfig(), nil) for _, p := range resp.Series { - d.TrainPattern(p.Pattern, p.Samples) + d.TrainPattern(p.GetPattern(), p.Samples) } resp.Series = resp.Series[:0] @@ -81,10 +110,8 @@ func prunePatterns(resp *logproto.QueryPatternsResponse, minClusterSize int) *lo if pattern == "" { continue } - resp.Series = append(resp.Series, &logproto.PatternSeries{ - Pattern: pattern, - Samples: cluster.Samples(), - }) + resp.Series = append(resp.Series, + logproto.NewPatternSeriesWithPattern(pattern, cluster.Samples())) } return resp } diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index d1016b326df7..c83da105ec2c 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -18,9 +18,7 @@ func Test_prunePatterns(t *testing.T) { resp := new(logproto.QueryPatternsResponse) scanner := bufio.NewScanner(file) for scanner.Scan() { - resp.Series = append(resp.Series, &logproto.PatternSeries{ - Pattern: scanner.Text(), - }) + resp.Series = append(resp.Series, logproto.NewPatternSeriesWithPattern(scanner.Text(), []*logproto.PatternSample{})) } require.NoError(t, scanner.Err()) prunePatterns(resp, 0) @@ -36,7 +34,7 @@ func Test_prunePatterns(t *testing.T) { patterns := make([]string, 0, len(resp.Series)) for _, p := range resp.Series { - patterns = append(patterns, p.Pattern) + patterns = append(patterns, p.GetPattern()) } require.Equal(t, expectedPatterns, patterns) diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index eff054b9ec04..bc681e8faf02 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -11,53 +11,268 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/pkg/push" ) func TestInstancePushQuery(t *testing.T) { - lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test")) - require.NoError(t, err) - - err = inst.Push(context.Background(), &push.PushRequest{ - Streams: []push.Stream{ - { - Labels: lbs.String(), - Entries: []push.Entry{ + t.Run("test pattern samples", func(t *testing.T) { + lbs := labels.New(labels.Label{Name: "test", Value: "test"}) + inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test")) + require.NoError(t, err) + + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(20, 0), + Line: "ts=1 msg=hello", + }, + }, + }, + }, + }) + + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(30, 0), + Line: "ts=2 msg=hello", + }, + }, + }, + }, + }) + for i := 0; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ { - Timestamp: time.Unix(20, 0), - Line: "ts=1 msg=hello", + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(30, 0), + Line: "foo bar foo bar", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ + Query: "{test=\"test\"}", + Start: time.Unix(0, 0), + End: time.Unix(0, math.MaxInt64), + }) + require.NoError(t, err) + res, err := iter.ReadAllWithPatterns(it) + require.NoError(t, err) + require.Equal(t, 2, len(res.Series)) + + it, err = inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ + Query: "{test=\"test\"}", + Start: time.Unix(0, 0), + End: time.Unix(30, 0), + }) + require.NoError(t, err) + res, err = iter.ReadAllWithPatterns(it) + require.NoError(t, err) + // query should be inclusive of end time to match our + // existing metric query behavior + require.Equal(t, 2, len(res.Series)) + require.Equal(t, 2, len(res.Series[0].Samples)) + }) + + t.Run("test count_over_time samples", func(t *testing.T) { + lbs := labels.New(labels.Label{Name: "test", Value: "test"}) + inst, err := newInstance("foo", log.NewNopLogger()) + require.NoError(t, err) + + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=hello", + }, }, }, }, - }, + }) + for i := 1; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo bar foo bar", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + + expr, err := syntax.ParseSampleExpr(`count_over_time({test="test"}[20s])`) + require.NoError(t, err) + + it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints := ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + require.Equal(t, int64(1), res.Series[0].Samples[0].Value) + + expr, err = syntax.ParseSampleExpr(`count_over_time({test="test"}[80s])`) + require.NoError(t, err) + + it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints = ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + + // with a larger selection range of 80s, we expect to eventually get up to 4 per datapoint + // our pushes are spaced 20s apart, and there's 10s step, so we ecpect to see the value increase + // every 2 samples, maxing out and staying at 4 after 6 samples (since it starts a 1, not 0) + require.Equal(t, int64(1), res.Series[0].Samples[0].Value) + require.Equal(t, int64(1), res.Series[0].Samples[1].Value) + require.Equal(t, int64(2), res.Series[0].Samples[2].Value) + require.Equal(t, int64(2), res.Series[0].Samples[3].Value) + require.Equal(t, int64(3), res.Series[0].Samples[4].Value) + require.Equal(t, int64(3), res.Series[0].Samples[5].Value) + require.Equal(t, int64(4), res.Series[0].Samples[6].Value) + require.Equal(t, int64(4), res.Series[0].Samples[expectedDataPoints-1].Value) }) - for i := 0; i <= 30; i++ { + + t.Run("test bytes_over_time samples", func(t *testing.T) { + lbs := labels.New(labels.Label{Name: "test", Value: "test"}) + inst, err := newInstance("foo", log.NewNopLogger()) + require.NoError(t, err) + err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { Labels: lbs.String(), Entries: []push.Entry{ { - Timestamp: time.Unix(20, 0), - Line: "foo bar foo bar", + Timestamp: time.Unix(0, 0), + Line: "foo bar foo bars", }, }, }, }, }) + for i := 1; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo bar foo bars", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + + expr, err := syntax.ParseSampleExpr(`bytes_over_time({test="test"}[20s])`) + require.NoError(t, err) + + it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) require.NoError(t, err) - } - require.NoError(t, err) - it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ - Query: "{test=\"test\"}", - Start: time.Unix(0, 0), - End: time.Unix(0, math.MaxInt64), + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints := ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + require.Equal(t, int64(16), res.Series[0].Samples[0].Value) + + expr, err = syntax.ParseSampleExpr(`bytes_over_time({test="test"}[80s])`) + require.NoError(t, err) + + it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints = ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + + // with a larger selection range of 80s, we expect to eventually get up to 64 bytes + // as each pushe is 16 bytes and are spaced 20s apart. We query with 10s step, + // so we ecpect to see the value increase by 16 bytes every 2 samples, + // maxing out and staying at 64 after 6 samples (since it starts a 1, not 0) + require.Equal(t, int64(16), res.Series[0].Samples[0].Value) + require.Equal(t, int64(16), res.Series[0].Samples[1].Value) + require.Equal(t, int64(32), res.Series[0].Samples[2].Value) + require.Equal(t, int64(32), res.Series[0].Samples[3].Value) + require.Equal(t, int64(48), res.Series[0].Samples[4].Value) + require.Equal(t, int64(48), res.Series[0].Samples[5].Value) + require.Equal(t, int64(64), res.Series[0].Samples[6].Value) + require.Equal(t, int64(64), res.Series[0].Samples[expectedDataPoints-1].Value) }) - require.NoError(t, err) - res, err := iter.ReadAll(it) - require.NoError(t, err) - require.Equal(t, 2, len(res.Series)) } diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index f6efa7de0443..cafc8dec2354 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -15,8 +15,9 @@ import ( "github.com/grafana/loki/v3/pkg/ingester/index" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/pattern/drain" + "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" ) @@ -82,8 +83,8 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ } from, through := util.RoundToMilliseconds(req.Start, req.End) step := model.Time(req.Step) - if step < drain.TimeResolution { - step = drain.TimeResolution + if step < chunk.TimeResolution { + step = chunk.TimeResolution } var iters []iter.Iterator @@ -101,6 +102,56 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ return iter.NewMerge(iters...), nil } +func (i *instance) QuerySample( + ctx context.Context, + expr syntax.SampleExpr, + req *logproto.QueryPatternsRequest, +) (iter.Iterator, error) { + from, through := util.RoundToMilliseconds(req.Start, req.End) + step := model.Time(req.Step) + if step < chunk.TimeResolution { + step = chunk.TimeResolution + } + + selector, err := expr.Selector() + if err != nil { + return nil, err + } + + typ, err := metric.ExtractMetricType(expr) + if err != nil || typ == metric.Unsupported { + return nil, err + } + + var iters []iter.Iterator + err = i.forMatchingStreams( + selector.Matchers(), + func(stream *stream) error { + var iter iter.Iterator + var err error + if typ == metric.Bytes { + iter, err = stream.BytesIterator(ctx, expr, from, through, step) + } else if typ == metric.Count { + iter, err = stream.CountIterator(ctx, expr, from, through, step) + } else { + return fmt.Errorf("unsupported query operation") + } + + if err != nil { + return err + } + + iters = append(iters, iter) + return nil + }, + ) + if err != nil { + return nil, err + } + + return iter.NewMerge(iters...), nil +} + // forMatchingStreams will execute a function for each stream that matches the given matchers. func (i *instance) forMatchingStreams( matchers []*labels.Matcher, diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go new file mode 100644 index 000000000000..7edd5364e6f1 --- /dev/null +++ b/pkg/pattern/instance_test.go @@ -0,0 +1,115 @@ +package pattern + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/push" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestInstance_QuerySample(t *testing.T) { + ctx := context.Background() + thirtySeconds := int64(30000) + oneMin := int64(60000) + fiveMin := oneMin * 5 + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 + + mockReq := &logproto.QueryPatternsRequest{ + Start: time.Unix(then/1000, 0), + End: time.Now(), + Step: oneMin, + } + + instance, err := newInstance("test", log.NewNopLogger()) + require.NoError(t, err) + + labels := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + } + + lastTsMilli := (then + oneMin + oneMin) // 1715964095000 + + //TODO(twhitney): Add a few more pushes to this or another test + instance.Push(ctx, &logproto.PushRequest{ + Streams: []push.Stream{ + { + Labels: labels.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(then/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix((then+thirtySeconds)/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix((then+oneMin)/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix(lastTsMilli/1000, 0), + Line: "this=that color=blue", + }, + }, + Hash: uint64(labels.Fingerprint()), + }, + }, + }) + + t.Run("successful count over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[30s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + // start is request start minus range, which is 30s here + start := then - 30000 + require.True(t, start < lastTsMilli-30000) + secondPoint := start + oneMin + require.True(t, secondPoint < lastTsMilli-30000) + // this is the first point past the lastTsMilli + thirdPoint := secondPoint + oneMin + require.Equal(t, lastTsMilli-30000, thirdPoint) + + next := iter.Next() + require.True(t, next) + + sample := iter.At() + require.Equal(t, int64(4), sample.Value) + require.Equal(t, model.Time(thirdPoint), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) + + t.Run("successful bytes over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[30s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + next := iter.Next() + require.True(t, next) + + expctedTs := (then - 30000) + oneMin + oneMin + sample := iter.At() + require.Equal(t, int64(80), sample.Value) + require.Equal(t, model.Time(expctedTs), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) +} diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index 80ad1197c80a..159edf1d73d2 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -6,7 +6,33 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" ) -func ReadBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { +func ReadMetricsBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { + var ( + series = map[string][]*logproto.PatternSample{} + respSize int + ) + + for ; respSize < batchSize && it.Next(); respSize++ { + labels := it.Labels() + sample := it.At() + series[labels.String()] = append(series[labels.String()], &sample) + } + result := logproto.QueryPatternsResponse{ + Series: make([]*logproto.PatternSeries, 0, len(series)), + } + for id, samples := range series { + result.Series = append( + result.Series, + logproto.NewPatternSeriesWithLabels(id, samples), + ) + } + return &result, it.Error() +} + +func ReadPatternsBatch( + it Iterator, + batchSize int, +) (*logproto.QueryPatternsResponse, error) { var ( series = map[string][]*logproto.PatternSample{} respSize int @@ -20,15 +46,19 @@ func ReadBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, err result := logproto.QueryPatternsResponse{ Series: make([]*logproto.PatternSeries, 0, len(series)), } - for pattern, samples := range series { - result.Series = append(result.Series, &logproto.PatternSeries{ - Pattern: pattern, - Samples: samples, - }) + for id, samples := range series { + result.Series = append( + result.Series, + logproto.NewPatternSeriesWithPattern(id, samples), + ) } return &result, it.Error() } -func ReadAll(it Iterator) (*logproto.QueryPatternsResponse, error) { - return ReadBatch(it, math.MaxInt32) +func ReadAllWithPatterns(it Iterator) (*logproto.QueryPatternsResponse, error) { + return ReadPatternsBatch(it, math.MaxInt32) +} + +func ReadAllWithLabels(it Iterator) (*logproto.QueryPatternsResponse, error) { + return ReadMetricsBatch(it, math.MaxInt32) } diff --git a/pkg/pattern/iter/batch_test.go b/pkg/pattern/iter/batch_test.go index 7f544e23f417..d798583d1a2a 100644 --- a/pkg/pattern/iter/batch_test.go +++ b/pkg/pattern/iter/batch_test.go @@ -32,13 +32,13 @@ func TestReadBatch(t *testing.T) { batchSize: 2, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo", - Samples: []*logproto.PatternSample{ + logproto.NewPatternSeriesWithPattern( + "foo", + []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, }, - }, + ), }, }, }, @@ -49,14 +49,14 @@ func TestReadBatch(t *testing.T) { batchSize: 4, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo", - Samples: []*logproto.PatternSample{ + logproto.NewPatternSeriesWithPattern( + "foo", + []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, }, - }, + ), }, }, }, @@ -64,8 +64,8 @@ func TestReadBatch(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - it := NewSlice(tt.pattern, tt.samples) - got, err := ReadBatch(it, tt.batchSize) + it := NewPatternSlice(tt.pattern, tt.samples) + got, err := ReadPatternsBatch(it, tt.batchSize) require.NoError(t, err) require.Equal(t, tt.expected, got) }) diff --git a/pkg/pattern/iter/iterator.go b/pkg/pattern/iter/iterator.go index 5a277c0f2734..2f27104f0da0 100644 --- a/pkg/pattern/iter/iterator.go +++ b/pkg/pattern/iter/iterator.go @@ -1,7 +1,9 @@ package iter import ( + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/prometheus/model/labels" ) var Empty Iterator = &emptyIterator{} @@ -10,49 +12,75 @@ type Iterator interface { Next() bool Pattern() string + Labels() labels.Labels At() logproto.PatternSample Error() error Close() error } -func NewSlice(pattern string, s []logproto.PatternSample) Iterator { - return &sliceIterator{ +type SampleIterator interface { + Iterator + Sample() logproto.PatternSample +} + +type PeekingIterator interface { + SampleIterator + Peek() (string, logproto.PatternSample, bool) +} + +func NewPatternSlice(pattern string, s []logproto.PatternSample) Iterator { + return &patternSliceIterator{ values: s, pattern: pattern, + labels: labels.EmptyLabels(), i: -1, } } -type sliceIterator struct { +func NewLabelsSlice(lbls labels.Labels, s []logproto.PatternSample) Iterator { + return &patternSliceIterator{ + values: s, + labels: lbls, + i: -1, + } +} + +type patternSliceIterator struct { i int pattern string + labels labels.Labels values []logproto.PatternSample } -func (s *sliceIterator) Next() bool { +func (s *patternSliceIterator) Next() bool { s.i++ return s.i < len(s.values) } -func (s *sliceIterator) Pattern() string { +func (s *patternSliceIterator) Pattern() string { return s.pattern } -func (s *sliceIterator) At() logproto.PatternSample { +func (s *patternSliceIterator) Labels() labels.Labels { + return s.labels +} + +func (s *patternSliceIterator) At() logproto.PatternSample { return s.values[s.i] } -func (s *sliceIterator) Error() error { +func (s *patternSliceIterator) Error() error { return nil } -func (s *sliceIterator) Close() error { +func (s *patternSliceIterator) Close() error { return nil } type emptyIterator struct { pattern string + labels labels.Labels } func (e *emptyIterator) Next() bool { @@ -63,6 +91,10 @@ func (e *emptyIterator) Pattern() string { return e.pattern } +func (e *emptyIterator) Labels() labels.Labels { + return e.labels +} + func (e *emptyIterator) At() logproto.PatternSample { return logproto.PatternSample{} } @@ -79,16 +111,25 @@ type nonOverlappingIterator struct { iterators []Iterator curr Iterator pattern string + labels labels.Labels } -// NewNonOverlappingIterator gives a chained iterator over a list of iterators. -func NewNonOverlappingIterator(pattern string, iterators []Iterator) Iterator { +// NewNonOverlappingPatternIterator gives a chained iterator over a list of iterators. +func NewNonOverlappingPatternIterator(pattern string, iterators []Iterator) Iterator { return &nonOverlappingIterator{ iterators: iterators, pattern: pattern, } } +// NewNonOverlappingLabelsIterator gives a chained iterator over a list of iterators. +func NewNonOverlappingLabelsIterator(labels labels.Labels, iterators []Iterator) Iterator { + return &nonOverlappingIterator{ + iterators: iterators, + labels: labels, + } +} + func (i *nonOverlappingIterator) Next() bool { for i.curr == nil || !i.curr.Next() { if len(i.iterators) == 0 { @@ -114,6 +155,10 @@ func (i *nonOverlappingIterator) Pattern() string { return i.pattern } +func (i *nonOverlappingIterator) Labels() labels.Labels { + return i.labels +} + func (i *nonOverlappingIterator) Error() error { if i.curr == nil { return nil @@ -131,3 +176,114 @@ func (i *nonOverlappingIterator) Close() error { i.iterators = nil return nil } + +type peekingIterator struct { + iter Iterator + + cache *sampleWithLabels + next *sampleWithLabels + labels labels.Labels +} + +type sampleWithLabels struct { + logproto.PatternSample + labels labels.Labels +} + +func (s *sampleWithLabels) Sample() logproto.Sample { + return logproto.Sample{ + Timestamp: s.PatternSample.Timestamp.UnixNano(), // logproto.Sample expects nano seconds + Value: float64(s.PatternSample.Value), + Hash: 0, + } +} + +func NewPeekingSampleIterator(iter Iterator) iter.PeekingSampleIterator { + // initialize the next entry so we can peek right from the start. + var cache *sampleWithLabels + next := &sampleWithLabels{} + if iter.Next() { + cache = &sampleWithLabels{ + PatternSample: iter.At(), + labels: iter.Labels(), + } + next.PatternSample = cache.PatternSample + next.labels = cache.labels + } + + return &peekingIterator{ + iter: iter, + cache: cache, + next: next, + labels: iter.Labels(), + } +} + +func (it *peekingIterator) Close() error { + return it.iter.Close() +} + +func (it *peekingIterator) Labels() string { + return it.labels.String() +} + +func (it *peekingIterator) Next() bool { + if it.cache != nil { + it.next.PatternSample = it.cache.PatternSample + it.next.labels = it.cache.labels + it.cacheNext() + return true + } + return false +} + +func (it *peekingIterator) Sample() logproto.Sample { + if it.next != nil { + return logproto.Sample{ + Timestamp: it.next.PatternSample.Timestamp.UnixNano(), // expecting nano seconds + Value: float64(it.next.PatternSample.Value), + Hash: 0, + } + } + return logproto.Sample{} +} + +func (it *peekingIterator) At() logproto.PatternSample { + if it.next != nil { + return it.next.PatternSample + } + return logproto.PatternSample{} +} + +// cacheNext caches the next element if it exists. +func (it *peekingIterator) cacheNext() { + if it.iter.Next() { + it.cache.PatternSample = it.iter.At() + it.cache.labels = it.iter.Labels() + return + } + // nothing left, remove the cached entry + it.cache = nil +} + +func (it *peekingIterator) Pattern() logproto.PatternSample { + if it.next != nil { + return it.next.PatternSample + } + return logproto.PatternSample{} +} + +func (it *peekingIterator) Peek() (string, logproto.Sample, bool) { + if it.cache != nil { + return it.cache.labels.String(), it.cache.Sample(), true + } + return "", logproto.Sample{}, false +} + +func (it *peekingIterator) Error() error { + return it.iter.Error() +} + +func (it *peekingIterator) StreamHash() uint64 { + return 0 +} diff --git a/pkg/pattern/iter/iterator_test.go b/pkg/pattern/iter/iterator_test.go index b327800575b5..dd66361d635b 100644 --- a/pkg/pattern/iter/iterator_test.go +++ b/pkg/pattern/iter/iterator_test.go @@ -3,57 +3,111 @@ package iter import ( "testing" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestSliceIterator(t *testing.T) { - tests := []struct { - name string - pattern string - samples []logproto.PatternSample - want []patternSample - }{ - { - name: "1 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, + t.Run("samples with pattern", func(t *testing.T) { + tests := []struct { + name string + pattern string + samples []logproto.PatternSample + want []patternSample + }{ + { + name: "1 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + }, + want: []patternSample{ + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + }, }, - want: []patternSample{ - {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, + { + name: "3 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + want: []patternSample{ + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, }, - }, - { - name: "3 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - {Timestamp: 20, Value: 4}, - {Timestamp: 30, Value: 6}, + { + name: "empty", + pattern: "foo", + samples: nil, + want: nil, }, - want: []patternSample{ - {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"foo", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"foo", logproto.PatternSample{Timestamp: 30, Value: 6}}, + } + + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + got := slice(NewPatternSlice(tt.pattern, tt.samples)) + require.Equal(t, tt.want, got) + }) + } + }) + + t.Run("samples with labels", func(t *testing.T) { + stream := labels.Labels{ + {Name: "test", Value: "test"}, + } + tests := []struct { + name string + labels labels.Labels + samples []logproto.PatternSample + want []patternSample + }{ + { + name: "1 samples", + labels: stream, + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + }, + want: []patternSample{ + {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, + }, }, - }, - { - name: "empty", - pattern: "foo", - samples: nil, - want: nil, - }, - } + { + name: "3 samples", + labels: stream, + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + want: []patternSample{ + {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream, logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, + }, + { + name: "empty", + labels: labels.EmptyLabels(), + samples: nil, + want: nil, + }, + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - got := slice(NewSlice(tt.pattern, tt.samples)) - require.Equal(t, tt.want, got) - }) - } + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + got := slice(NewLabelsSlice(tt.labels, tt.samples)) + require.Equal(t, tt.want, got) + }) + } + }) } func slice(it Iterator) []patternSample { @@ -63,6 +117,7 @@ func slice(it Iterator) []patternSample { samples = append(samples, patternSample{ pattern: it.Pattern(), sample: it.At(), + labels: it.Labels(), }) } if it.Error() != nil { diff --git a/pkg/pattern/iter/merge.go b/pkg/pattern/iter/merge.go index 3b0e07e33b8a..0c7c19a633a9 100644 --- a/pkg/pattern/iter/merge.go +++ b/pkg/pattern/iter/merge.go @@ -5,6 +5,7 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/util/loser" + "github.com/prometheus/prometheus/model/labels" ) type mergeIterator struct { @@ -16,18 +17,22 @@ type mergeIterator struct { type patternSample struct { pattern string + labels labels.Labels sample logproto.PatternSample } var max = patternSample{ pattern: "", + labels: labels.Labels{}, sample: logproto.PatternSample{Timestamp: math.MaxInt64}, } func NewMerge(iters ...Iterator) Iterator { + // TODO: I need to call next here tree := loser.New(iters, max, func(s Iterator) patternSample { return patternSample{ pattern: s.Pattern(), + labels: s.Labels(), sample: s.At(), } }, func(e1, e2 patternSample) bool { @@ -57,10 +62,13 @@ func (m *mergeIterator) Next() bool { } m.current.pattern = m.tree.Winner().Pattern() + m.current.labels = m.tree.Winner().Labels() m.current.sample = m.tree.Winner().At() for m.tree.Next() { - if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || m.current.pattern != m.tree.Winner().Pattern() { + if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || + m.current.pattern != m.tree.Winner().Pattern() || + m.current.labels.String() != m.tree.Winner().Labels().String() { return true } m.current.sample.Value += m.tree.Winner().At().Value @@ -74,6 +82,10 @@ func (m *mergeIterator) Pattern() string { return m.current.pattern } +func (m *mergeIterator) Labels() labels.Labels { + return m.current.labels +} + func (m *mergeIterator) At() logproto.PatternSample { return m.current.sample } diff --git a/pkg/pattern/iter/merge_test.go b/pkg/pattern/iter/merge_test.go index a1d643a5a01c..d55f417062bd 100644 --- a/pkg/pattern/iter/merge_test.go +++ b/pkg/pattern/iter/merge_test.go @@ -3,76 +3,150 @@ package iter import ( "testing" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestMerge(t *testing.T) { - tests := []struct { - name string - iterators []Iterator - expected []patternSample - }{ - { - name: "Empty iterators", - iterators: []Iterator{}, - expected: nil, - }, - { - name: "Merge single iterator", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, - }), + t.Run("merging patterns", func(t *testing.T) { + tests := []struct { + name string + iterators []Iterator + expected []patternSample + }{ + { + name: "Empty iterators", + iterators: []Iterator{}, + expected: nil, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"a", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, + { + name: "Merge single iterator", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, + }), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, }, - }, - { - name: "Merge multiple iterators", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + { + name: "Merge multiple iterators", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, - {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, + { + name: "Merge multiple iterators with similar samples", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 4}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 12}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - }, - { - name: "Merge multiple iterators with similar samples", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + } + + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + it := NewMerge(tt.iterators...) + defer it.Close() + + var result []patternSample + for it.Next() { + result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) + } + + require.Equal(t, tt.expected, result) + }) + } + }) + + t.Run("merging label samples", func(t *testing.T) { + stream1 := labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "ying", Value: "yang"}} + stream2 := labels.Labels{labels.Label{Name: "foo", Value: "baz"}, labels.Label{Name: "ying", Value: "yang"}} + tests := []struct { + name string + iterators []Iterator + expected []patternSample + }{ + { + name: "Empty iterators", + iterators: []Iterator{}, + expected: nil, + }, + { + name: "Merge single iterator", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, + }), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream1, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, + }, + { + name: "Merge multiple iterators", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, + {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 4}}, - {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 12}}, - {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, + { + name: "Merge multiple iterators with similar samples", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 4}}, + {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 12}}, + {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - }, - } + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - it := NewMerge(tt.iterators...) - defer it.Close() + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + it := NewMerge(tt.iterators...) + defer it.Close() - var result []patternSample - for it.Next() { - result = append(result, patternSample{it.Pattern(), it.At()}) - } + var result []patternSample + for it.Next() { + result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) + } - require.Equal(t, tt.expected, result) - }) - } + require.Equal(t, tt.expected, result) + }) + } + }) } diff --git a/pkg/pattern/iter/query_client.go b/pkg/pattern/iter/query_client.go index f6c5c4fa9774..00ed407a39e9 100644 --- a/pkg/pattern/iter/query_client.go +++ b/pkg/pattern/iter/query_client.go @@ -4,6 +4,8 @@ import ( "io" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" ) type queryClientIterator struct { @@ -38,6 +40,10 @@ func (i *queryClientIterator) Pattern() string { return i.curr.Pattern() } +func (i *queryClientIterator) Labels() labels.Labels { + return i.curr.Labels() +} + func (i *queryClientIterator) At() logproto.PatternSample { return i.curr.At() } @@ -58,7 +64,17 @@ func NewQueryResponseIterator(resp *logproto.QueryPatternsResponse) Iterator { for j, sample := range s.Samples { samples[j] = *sample } - iters[i] = NewSlice(s.Pattern, samples) + + switch s.GetIdentifier().(type) { + case *logproto.PatternSeries_Labels: + ls, err := parser.ParseMetric(s.GetLabels()) + if err != nil { + ls = labels.Labels{} + } + iters[i] = NewLabelsSlice(ls, samples) + default: + iters[i] = NewPatternSlice(s.GetPattern(), samples) + } } return NewMerge(iters...) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go new file mode 100644 index 000000000000..c4716ba532dd --- /dev/null +++ b/pkg/pattern/metric/chunk.go @@ -0,0 +1,201 @@ +package metric + +import ( + "context" + "fmt" + "sort" + "time" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" + "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +type MetricType int + +const ( + Bytes MetricType = iota + Count + Unsupported +) + +type Chunks struct { + chunks []Chunk + labels labels.Labels +} + +func NewChunks(labels labels.Labels) *Chunks { + return &Chunks{ + chunks: make([]Chunk, 0), + labels: labels, + } +} + +func (c *Chunks) Observe(bytes, count uint64, ts model.Time) { + if len(c.chunks) == 0 { + c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + return + } + + last := &(c.chunks)[len(c.chunks)-1] + if !last.spaceFor(ts) { + c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + return + } + + last.AddSample(newSample(bytes, count, ts)) +} + +func (c *Chunks) Iterator( + ctx context.Context, + typ MetricType, + from, through, step model.Time, +) (iter.Iterator, error) { + if typ == Unsupported { + return nil, fmt.Errorf("unsupported metric type") + } + + iters := make([]iter.Iterator, 0, len(c.chunks)) + for _, chunk := range c.chunks { + samples, err := chunk.ForRangeAndType(typ, from, through, step) + if err != nil { + return nil, err + } + + if len(samples) == 0 { + continue + } + + iters = append(iters, iter.NewLabelsSlice(c.labels, samples)) + } + return iter.NewNonOverlappingLabelsIterator(c.labels, iters), nil +} + +// TODO(twhitney): These values should be float64s (to match prometheus samples) or int64s (to match pattern samples) +type MetricSample struct { + Timestamp model.Time + Bytes uint64 + Count uint64 +} + +func newSample(bytes, count uint64, ts model.Time) MetricSample { + return MetricSample{ + Timestamp: ts, + Bytes: bytes, + Count: count, + } +} + +type MetricSamples []MetricSample + +type Chunk struct { + Samples MetricSamples + mint, maxt int64 +} + +func (c *Chunk) Bounds() (fromT, toT time.Time) { + return time.Unix(0, c.mint), time.Unix(0, c.maxt) +} + +func (c *Chunk) AddSample(s MetricSample) { + c.Samples = append(c.Samples, s) + ts := int64(s.Timestamp) + + if ts < c.mint { + c.mint = ts + } + + if ts > c.maxt { + c.maxt = ts + } +} + +func newChunk(bytes, count uint64, ts model.Time) Chunk { + maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 + v := Chunk{Samples: make(MetricSamples, 1, maxSize)} + v.Samples[0] = newSample(bytes, count, ts) + return v +} + +func (c *Chunk) spaceFor(ts model.Time) bool { + if len(c.Samples) == 0 { + return true + } + + return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime +} + +//TODO(twhitney): any way to remove the duplication between this and the drain chunk ForRange method? +// ForRangeAndType returns samples with only the values +// in the given range [start:end] and aggregates them by step duration. +// start and end are in milliseconds since epoch. step is a duration in milliseconds. +func (c *Chunk) ForRangeAndType( + typ MetricType, + start, end, step model.Time, +) ([]logproto.PatternSample, error) { + if typ == Unsupported { + return nil, fmt.Errorf("unsupported metric type") + } + + if len(c.Samples) == 0 { + return nil, nil + } + + first := c.Samples[0].Timestamp // why is this in the future? + last := c.Samples[len(c.Samples)-1].Timestamp + startBeforeEnd := start >= end + samplesAreAfterRange := first > end + samplesAreBeforeRange := last < start + if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { + return nil, nil + } + + var lo int + if start > first { + lo = sort.Search(len(c.Samples), func(i int) bool { + return c.Samples[i].Timestamp >= start + }) + } + hi := len(c.Samples) + if end < last { + hi = sort.Search(len(c.Samples), func(i int) bool { + return c.Samples[i].Timestamp > end + }) + } + + // Re-scale samples into step-sized buckets + currentStep := chunk.TruncateTimestamp(c.Samples[lo].Timestamp, step) + numOfSteps := ((c.Samples[hi-1].Timestamp - currentStep) / step) + 1 + aggregatedSamples := make([]logproto.PatternSample, 0, numOfSteps) + aggregatedSamples = append(aggregatedSamples, + logproto.PatternSample{ + Timestamp: currentStep, + Value: 0, + }) + + for _, sample := range c.Samples[lo:hi] { + if sample.Timestamp >= currentStep+step { + stepForSample := chunk.TruncateTimestamp(sample.Timestamp, step) + for i := currentStep + step; i <= stepForSample; i += step { + aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ + Timestamp: i, + Value: 0, + }) + } + currentStep = stepForSample + } + + var v int64 + if typ == Bytes { + v = int64(sample.Bytes) + } else { + v = int64(sample.Count) + } + + aggregatedSamples[len(aggregatedSamples)-1].Value += v + } + + return aggregatedSamples, nil +} diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go new file mode 100644 index 000000000000..32b746b0d20d --- /dev/null +++ b/pkg/pattern/metric/chunk_test.go @@ -0,0 +1,329 @@ +package metric + +import ( + "reflect" + "testing" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" +) + +func TestForRangeAndType(t *testing.T) { + testCases := []struct { + name string + c *Chunk + metricType MetricType + start model.Time + end model.Time + expected []logproto.PatternSample + }{ + { + name: "Empty count", + c: &Chunk{}, + metricType: Count, + start: 1, + end: 10, + expected: nil, + }, + { + name: "Empty bytes", + c: &Chunk{}, + metricType: Bytes, + start: 1, + end: 10, + expected: nil, + }, + { + name: "No Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 10, + end: 20, + expected: nil, + }, + { + name: "No Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 10, + end: 20, + expected: nil, + }, + { + name: "Complete Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Complete Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Partial Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 3, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Partial Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 3, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Single Element in Range -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 4, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Single Element in Range -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 4, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Start Before First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 5, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + }, + }, + { + name: "Start Before First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 5, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + }, + }, + { + name: "End After Last Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 5, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "End After Last Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 5, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Start before First and End Inclusive of First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start before First and End Inclusive of First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start and End before First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 1, + expected: nil, + }, + { + name: "Start and End before First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 1, + expected: nil, + }, + { + name: "Higher resolution samples down-sampled to preceding step bucket -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Count: 2}, + {Timestamp: 2, Count: 4}, + {Timestamp: 3, Count: 6}, + {Timestamp: 4, Count: 8}, + {Timestamp: 5, Count: 10}, + {Timestamp: 6, Count: 12}, + }}, + metricType: Count, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 10}, + {Timestamp: 4, Value: 18}, + {Timestamp: 6, Value: 12}, + }, + }, + { + name: "Higher resolution samples down-sampled to preceding step bucket -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Bytes: 2}, + {Timestamp: 2, Bytes: 4}, + {Timestamp: 3, Bytes: 6}, + {Timestamp: 4, Bytes: 8}, + {Timestamp: 5, Bytes: 10}, + {Timestamp: 6, Bytes: 12}, + }}, + metricType: Bytes, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 10}, + {Timestamp: 4, Value: 18}, + {Timestamp: 6, Value: 12}, + }, + }, + { + name: "Low resolution samples insert 0 values for empty steps -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Count: 2}, + {Timestamp: 5, Count: 10}, + }}, + metricType: Count, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 0}, + {Timestamp: 4, Value: 10}, + }, + }, + { + name: "Low resolution samples insert 0 values for empty steps -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Bytes: 2}, + {Timestamp: 5, Bytes: 10}, + }}, + metricType: Bytes, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 0}, + {Timestamp: 4, Value: 10}, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result, err := tc.c.ForRangeAndType(tc.metricType, tc.start, tc.end, model.Time(2)) + require.NoError(t, err) + if !reflect.DeepEqual(result, tc.expected) { + t.Errorf("Expected %v, got %v", tc.expected, result) + } + require.Equal(t, len(result), cap(result), "Returned slice wasn't created at the correct capacity") + }) + } +} diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go new file mode 100644 index 000000000000..a1e301f31276 --- /dev/null +++ b/pkg/pattern/metric/evaluator.go @@ -0,0 +1,354 @@ +package metric + +import ( + "context" + "fmt" + "sort" + "time" + + loki_iter "github.com/grafana/loki/v3/pkg/iter" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" +) + +// TODO(twhitney): duplication with code in NewStepEvaluator +func ExtractMetricType(expr syntax.SampleExpr) (MetricType, error) { + var typ MetricType + switch e := expr.(type) { + case *syntax.VectorAggregationExpr: + if rangeExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { + if rangeExpr.Operation == syntax.OpRangeTypeCount { + typ = Count + } else if rangeExpr.Operation == syntax.OpRangeTypeBytes { + typ = Bytes + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + case *syntax.RangeAggregationExpr: + if e.Operation == syntax.OpRangeTypeCount { + typ = Count + } else if e.Operation == syntax.OpRangeTypeBytes { + typ = Bytes + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + default: + return Unsupported, fmt.Errorf("unexpected expression type %T", e) + } + return typ, nil +} + +type SampleEvaluatorFactory interface { + // NewStepEvaluator returns a NewStepEvaluator for a given SampleExpr. + // It's explicitly passed another NewStepEvaluator + // in order to enable arbitrary computation of embedded expressions. This allows more modular & extensible + // NewStepEvaluator implementations which can be composed. + NewStepEvaluator( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, + ) (logql.StepEvaluator, error) +} + +type SampleEvaluatorFunc func( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) + +func (s SampleEvaluatorFunc) NewStepEvaluator( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) { + return s(ctx, nextEvaluatorFactory, expr, typ, from, through, step) +} + +type DefaultEvaluatorFactory struct { + chunks *Chunks +} + +func NewDefaultEvaluatorFactory(chunks *Chunks) *DefaultEvaluatorFactory { + return &DefaultEvaluatorFactory{ + chunks: chunks, + } +} + +func (ev *DefaultEvaluatorFactory) NewStepEvaluator( + ctx context.Context, + evFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) { + switch e := expr.(type) { + case *syntax.VectorAggregationExpr: + if rangExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { + // if range expression is wrapped with a vector expression + // we should send the vector expression for allowing reducing labels at the source. + evFactory = SampleEvaluatorFunc( + func(ctx context.Context, + _ SampleEvaluatorFactory, + _ syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, + ) (logql.StepEvaluator, error) { + fromWithRangeAndOffset := from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset) + throughWithOffset := through.Add(-rangExpr.Left.Offset) + it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + if err != nil { + return nil, err + } + + params := NewParams( + e, + from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset), + through.Add(-rangExpr.Left.Offset), + step, + ) + return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), rangExpr, params, rangExpr.Left.Offset) + }) + } + + if e.Grouping == nil { + return nil, errors.Errorf("aggregation operator '%q' without grouping", e.Operation) + } + nextEvaluator, err := evFactory.NewStepEvaluator(ctx, evFactory, e.Left, typ, from, through, step) + if err != nil { + return nil, err + } + sort.Strings(e.Grouping.Groups) + + return logql.NewVectorAggEvaluator( + nextEvaluator, + e, + make([]byte, 0, 1024), + labels.NewBuilder(labels.Labels{}), + ), nil + + case *syntax.RangeAggregationExpr: + fromWithRangeAndOffset := from.Add(-e.Left.Interval).Add(-e.Left.Offset) + throughWithOffset := through.Add(-e.Left.Offset) + it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + if err != nil { + return nil, err + } + + params := NewParams( + e, + from.Add(-e.Left.Interval).Add(-e.Left.Offset), + through.Add(-e.Left.Offset), + step, // expecting nanoseconds + ) + return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), e, params, e.Left.Offset) + default: + return nil, errors.Errorf("unexpected expr type (%T)", e) + } +} + +// Need to create our own StepEvaluator since we only support bytes and count over time, +// and always sum to get those values. In order to accomplish this we need control over the +// aggregation operation.. +func NewPatternSampleRangeAggEvaluator( + it loki_iter.PeekingSampleIterator, + expr *syntax.RangeAggregationExpr, + q logql.Params, + o time.Duration, +) (logql.StepEvaluator, error) { + iter, err := newRangeVectorIterator( + it, expr, + expr.Left.Interval.Nanoseconds(), + q.Step().Nanoseconds(), + q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), + ) + if err != nil { + return nil, err + } + + return logql.NewRangeVectorEvaluator(iter), nil +} + +func newRangeVectorIterator( + it loki_iter.PeekingSampleIterator, + expr *syntax.RangeAggregationExpr, + selRange, step, start, end, offset int64, +) (logql.RangeVectorIterator, error) { + // forces at least one step. + if step == 0 { + step = 1 + } + if offset != 0 { + start = start - offset + end = end - offset + } + // TODO(twhitney): do I need a streaming aggregator? + // if so the aggregator is going to make this + // a bit of a bad time, as there's currently no + // way to provide a custom one. + // + // var overlap bool + // if selRange >= step && start != end { + // overlap = true + // } + // if !overlap { + // _, err := streamingAggregator(expr) + // if err != nil { + // return nil, err + // } + // return &streamRangeVectorIterator{ + // iter: it, + // step: step, + // end: end, + // selRange: selRange, + // metrics: map[string]labels.Labels{}, + // r: expr, + // current: start - step, // first loop iteration will set it to start + // offset: offset, + // }, nil + // } + + // always sum + aggregator := logql.BatchRangeVectorAggregator(func(samples []promql.FPoint) float64 { + sum := 0.0 + for _, v := range samples { + sum += v.F + } + return sum + }) + + return logql.NewBatchRangeVectorIterator( + it, + selRange, + step, + start, + end, + offset, + aggregator, + ), nil +} + +type SeriesToSampleIterator struct { + floats []promql.FPoint + curTs int64 + cur float64 + lbls labels.Labels +} + +func NewSeriesToSampleIterator(series *promql.Series) *SeriesToSampleIterator { + return &SeriesToSampleIterator{ + floats: series.Floats, + lbls: series.Metric, + } +} + +func (s *SeriesToSampleIterator) Next() bool { + if len(s.floats) == 0 { + return false + } + + current, rest := s.floats[0], s.floats[1:] + + //Is timestamp the correct unit here? + s.curTs = current.T + s.cur = current.F + + s.floats = rest + return true +} + +func (s *SeriesToSampleIterator) Pattern() string { + return "" +} + +func (s *SeriesToSampleIterator) Labels() labels.Labels { + return s.lbls +} + +func (s *SeriesToSampleIterator) At() logproto.PatternSample { + return logproto.PatternSample{ + Timestamp: model.Time(s.curTs), + Value: int64(s.cur), + } +} + +func (s *SeriesToSampleIterator) Error() error { + return nil +} + +func (s *SeriesToSampleIterator) Close() error { + return nil +} + +type paramCompat struct { + expr syntax.SampleExpr + from model.Time + through model.Time + step model.Time +} + +func NewParams( + expr syntax.SampleExpr, + from, through, step model.Time, +) *paramCompat { + return ¶mCompat{ + expr: expr, + from: from, + through: through, + step: step, + } +} + +func (p *paramCompat) QueryString() string { + return p.expr.String() +} + +func (p *paramCompat) Start() time.Time { + return p.from.Time() +} + +func (p *paramCompat) End() time.Time { + return p.through.Time() +} + +func (p *paramCompat) Step() time.Duration { + return time.Duration(p.step.UnixNano()) +} + +func (p *paramCompat) Interval() time.Duration { + return time.Duration(0) +} + +func (p *paramCompat) Limit() uint32 { + return 0 +} + +func (p *paramCompat) Direction() logproto.Direction { + return logproto.BACKWARD +} + +func (p *paramCompat) Shards() []string { + return []string{} +} + +func (p *paramCompat) GetExpression() syntax.Expr { + return p.expr +} + +func (p *paramCompat) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go new file mode 100644 index 000000000000..9f23cb5546e6 --- /dev/null +++ b/pkg/pattern/metric/evaluator_test.go @@ -0,0 +1,363 @@ +package metric + +import ( + "context" + "testing" + + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" +) + +func Test_SampleEvaluator(t *testing.T) { + fiveMin := int64(300000) + stream := labels.Labels{ + labels.Label{ + Name: "foo", + Value: "bar", + }, + labels.Label{ + Name: "level", + Value: "debug", + }, + } + + setup := func(chunks Chunks, now int64, query string) logql.StepEvaluator { + factory := NewDefaultEvaluatorFactory(&chunks) + + expr, err := syntax.ParseSampleExpr(query) + require.NoError(t, err) + + typ, err := ExtractMetricType(expr) + require.NoError(t, err) + + evaluator, err := factory.NewStepEvaluator( + context.Background(), + factory, + expr.(syntax.SampleExpr), + typ, + model.Time(now-fiveMin), model.Time(now), model.Time(fiveMin), + ) + + require.NoError(t, err) + return evaluator + } + + chunks := func(now, then, beforeThen int64) Chunks { + nowTime := model.Time(now) + thenTime := model.Time(then) + beforeThenTime := model.Time(beforeThen) + return Chunks{ + chunks: []Chunk{ + { + Samples: []MetricSample{ + { + Timestamp: beforeThenTime, + Bytes: 1, + Count: 1, + }, + { + Timestamp: thenTime, + Bytes: 3, + Count: 2, + }, + { + Timestamp: nowTime, + Bytes: 5, + Count: 3, + }, + }, + mint: thenTime.Unix(), + maxt: nowTime.Unix(), + }, + }, + labels: stream, + } + } + + t.Run("grouping", func(t *testing.T) { + group := labels.Labels{ + labels.Label{ + Name: "level", + Value: "debug", + }, + } + t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 -- 5m before n + beforeThen := then - fiveMin // 1715963675000 -- 5m before then + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(2), resultVals[1]) + require.Equal(t, float64(3), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(3), resultVals[1]) // TODO: got 2, expected 3 + require.Equal(t, float64(5), resultVals[2]) + }) + }) + + t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - 150000 // 1715964125000 -- 2.5m before now + beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(8), resultVals[2]) + }) + }) + }) + + t.Run("without grouping", func(t *testing.T) { + t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 -- 5m before n + beforeThen := then - fiveMin // 1715963675000 -- 5m before then + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + samples := r.SampleVector() + resultTs[i] = samples[0].T + resultVals[i] = samples[0].F + + require.Equal(t, stream, samples[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(2), resultVals[1]) + require.Equal(t, float64(3), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + samples := r.SampleVector() + resultTs[i] = samples[0].T + resultVals[i] = samples[0].F + + require.Equal(t, stream, samples[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(3), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + }) + + t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - 150000 // 1715964125000 -- 2.5m before now + beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, stream, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, stream, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(8), resultVals[2]) + }) + }) + }) +} diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index f3aad280250d..24c9e21c96f8 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -2,15 +2,21 @@ package pattern import ( "context" + "math" "sync" "time" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/pattern/drain" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" ) type stream struct { @@ -20,6 +26,9 @@ type stream struct { labelHash uint64 patterns *drain.Drain mtx sync.Mutex + metrics *metric.Chunks + + evaluator metric.SampleEvaluatorFactory lastTs int64 } @@ -29,6 +38,7 @@ func newStream( labels labels.Labels, metrics *ingesterMetrics, ) (*stream, error) { + chunks := metric.NewChunks(labels) return &stream{ fp: fp, labels: labels, @@ -38,6 +48,8 @@ func newStream( PatternsEvictedTotal: metrics.patternsDiscardedTotal, PatternsDetectedTotal: metrics.patternsDetectedTotal, }), + metrics: chunks, + evaluator: metric.NewDefaultEvaluatorFactory(chunks), }, nil } @@ -48,13 +60,20 @@ func (s *stream) Push( s.mtx.Lock() defer s.mtx.Unlock() + bytes := uint64(0) + count := uint64(len(entries)) for _, entry := range entries { if entry.Timestamp.UnixNano() < s.lastTs { continue } + + bytes += uint64(len(entry.Line)) + s.lastTs = entry.Timestamp.UnixNano() s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) } + + s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) return nil } @@ -75,6 +94,142 @@ func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (it return iter.NewMerge(iters...), nil } +// TODO(twhitney): duplication between bytes and count iterators +func (s *stream) BytesIterator( + ctx context.Context, + expr syntax.SampleExpr, + from, through, step model.Time, +) (iter.Iterator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + stepEvaluator, err := s.evaluator.NewStepEvaluator( + ctx, + s.evaluator, + expr, + metric.Bytes, + from, + through, + step, + ) + if err != nil { + return nil, err + } + + next, ts, r := stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + // TODO(twhitney): actually get max series from limits + // this is only 1 series since we're already on a stream + // this this limit needs to also be enforced higher up + maxSeries := 1000 + series, err := s.JoinSampleVector( + next, + ts, + r, + stepEvaluator, + maxSeries, + from, through, step) + if err != nil { + return nil, err + } + + return metric.NewSeriesToSampleIterator(series), nil +} + +func (s *stream) JoinSampleVector( + next bool, + ts int64, + r logql.StepResult, + stepEvaluator logql.StepEvaluator, + maxSeries int, + from, through, step model.Time, +) (*promql.Series, error) { + stepCount := int(math.Ceil(float64(through.Sub(from).Nanoseconds()) / float64(step.UnixNano()))) + if stepCount <= 0 { + stepCount = 1 + } + + series := &promql.Series{ + Metric: s.labels, + Floats: make([]promql.FPoint, 0, stepCount), + } + + vec := promql.Vector{} + if next { + vec = r.SampleVector() + } + + // fail fast for the first step or instant query + if len(vec) > maxSeries { + return nil, logqlmodel.NewSeriesLimitError(maxSeries) + } + + for next { + vec = r.SampleVector() + for _, p := range vec { + series.Floats = append(series.Floats, promql.FPoint{ + T: ts, + F: p.F, + }) + } + + next, ts, r = stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + } + + return series, stepEvaluator.Error() +} + +// TODO(twhitney): duplication between bytes and count iterators +func (s *stream) CountIterator( + ctx context.Context, + expr syntax.SampleExpr, + from, through, step model.Time, +) (iter.Iterator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + stepEvaluator, err := s.evaluator.NewStepEvaluator( + ctx, + s.evaluator, + expr, + metric.Count, + from, + through, + step, + ) + if err != nil { + return nil, err + } + + next, ts, r := stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + // TODO(twhitney): actually get max series from limits + // this is only 1 series since we're already on a stream + // this this limit needs to also be enforced higher up + maxSeries := 1000 + series, err := s.JoinSampleVector( + next, + ts, + r, + stepEvaluator, + maxSeries, + from, through, step) + if err != nil { + return nil, err + } + + return metric.NewSeriesToSampleIterator(series), nil +} + func (s *stream) prune(olderThan time.Duration) bool { s.mtx.Lock() defer s.mtx.Unlock() diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index f2218816b111..ab0564af9995 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -36,7 +36,7 @@ func TestAddStream(t *testing.T) { require.NoError(t, err) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(2), res.Series[0].Samples[0].Value) @@ -70,7 +70,7 @@ func TestPruneStream(t *testing.T) { require.Equal(t, false, stream.prune(time.Hour)) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 1a4d6701b1b1..609c7ede41d0 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -202,8 +202,14 @@ func WriteQueryPatternsResponseJSON(r *logproto.QueryPatternsResponse, w io.Writ if len(r.Series) > 0 { for i, series := range r.Series { s.WriteObjectStart() - s.WriteObjectField("pattern") - s.WriteStringWithHTMLEscaped(series.Pattern) + if pattern := series.GetPattern(); pattern != "" { + s.WriteObjectField("pattern") + s.WriteStringWithHTMLEscaped(pattern) + } + if labels := series.GetLabels(); labels != "" { + s.WriteObjectField("labels") + s.WriteStringWithHTMLEscaped(labels) + } s.WriteMore() s.WriteObjectField("samples") s.WriteArrayStart() diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index c749677f7702..4688388744c9 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -1077,13 +1077,11 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, - }, + logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, + ), }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]}]}`, @@ -1091,20 +1089,17 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, - }, + logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, - { - Pattern: "foo <*> buzz", - Samples: []*logproto.PatternSample{ + ), + logproto.NewPatternSeriesWithPattern("foo <*> buzz", + []*logproto.PatternSample{ {Timestamp: model.TimeFromUnix(3), Value: 1}, {Timestamp: model.TimeFromUnix(3), Value: 2}, }, - }, + ), }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]},{"pattern":"foo <*> buzz","samples":[[3,1],[3,2]]}]}`, @@ -1112,17 +1107,58 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{}, + logproto.NewPatternSeriesWithPattern("foo <*> bar", + []*logproto.PatternSample{}, + ), + logproto.NewPatternSeriesWithPattern("foo <*> buzz", + []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[]},{"pattern":"foo <*> buzz","samples":[]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, - { - Pattern: "foo <*> buzz", - Samples: []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, + ), + logproto.NewPatternSeriesWithLabels(`{foo="buzz"}`, + []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(3), Value: 1}, + {Timestamp: model.TimeFromUnix(3), Value: 2}, + }, + ), }, }, - `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[]},{"pattern":"foo <*> buzz","samples":[]}]}`, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]},{"labels":"{foo=\"buzz\"}","samples":[[3,1],[3,2]]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, + []*logproto.PatternSample{}, + ), + logproto.NewPatternSeriesWithPattern(`{foo="buzz"}`, + []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[]},{"pattern":"{foo=\"buzz\"}","samples":[]}]}`, }, } { tc := tc From f0d6a92cb6dc00b9a61bf604f392db587222ece4 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 23 May 2024 14:03:32 -0600 Subject: [PATCH 02/38] feat: reject filter queries to /patterns endpoint --- pkg/logql/range_vector.go | 8 +- pkg/loki/loki.go | 2 +- pkg/pattern/ingester_querier.go | 28 +++-- pkg/pattern/ingester_querier_test.go | 158 +++++++++++++++++++++++++++ pkg/pattern/ingester_test.go | 4 +- pkg/pattern/instance_test.go | 2 +- pkg/pattern/metric/evaluator.go | 18 +-- pkg/pattern/ring_client.go | 86 +++++++++++++-- pkg/pattern/tee.go | 12 +- pkg/querier/querier.go | 7 ++ 10 files changed, 285 insertions(+), 40 deletions(-) diff --git a/pkg/logql/range_vector.go b/pkg/logql/range_vector.go index 200f1480add7..141d4865c46c 100644 --- a/pkg/logql/range_vector.go +++ b/pkg/logql/range_vector.go @@ -108,20 +108,20 @@ func NewBatchRangeVectorIterator( end: end, current: start - step, // first loop iteration will set it to start offset: offset, - metrics: map[string]labels.Labels{}, - window: map[string]*promql.Series{}, + metrics: map[string]labels.Labels{}, + window: map[string]*promql.Series{}, agg: agg, } } func (r *batchRangeVectorIterator) Next() bool { // slides the range window to the next position - r.current = r.current + r.step // first current will be 5 min before start + r.current = r.current + r.step if r.current > r.end { return false } rangeEnd := r.current - rangeStart := rangeEnd - r.selRange // in nanoseconds + rangeStart := rangeEnd - r.selRange // load samples r.popBack(rangeStart) r.load(rangeStart, rangeEnd) diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 9446b351aab8..ce826f0752d4 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -331,7 +331,7 @@ type Loki struct { distributor *distributor.Distributor Ingester ingester.Interface PatternIngester *pattern.Ingester - PatternRingClient *pattern.RingClient + PatternRingClient pattern.RingClient Querier querier.Querier cacheGenerationLoader queryrangebase.CacheGenNumberLoader querierAPI *querier.QuerierAPI diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index e9b42010a56e..41fce2d12398 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -4,10 +4,8 @@ import ( "context" "errors" "math" - "net/http" "github.com/go-kit/log" - "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/prometheus/client_golang/prometheus" @@ -26,14 +24,14 @@ type IngesterQuerier struct { cfg Config logger log.Logger - ringClient *RingClient + ringClient RingClient registerer prometheus.Registerer } func NewIngesterQuerier( cfg Config, - ringClient *RingClient, + ringClient RingClient, metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, @@ -48,21 +46,31 @@ func NewIngesterQuerier( func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { // validate that a supported query was provided - // TODO(twhitney): validate metric queries don't have filters var expr syntax.Expr _, err := syntax.ParseMatchers(req.Query, true) if err != nil { expr, err = syntax.ParseSampleExpr(req.Query) if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, ErrParseQuery.Error()) + return nil, ErrParseQuery } + var selector syntax.LogSelectorExpr switch expr.(type) { - case *syntax.VectorAggregationExpr, *syntax.RangeAggregationExpr: - break + case *syntax.VectorAggregationExpr: + selector, err = expr.(*syntax.VectorAggregationExpr).Selector() + case *syntax.RangeAggregationExpr: + selector, err = expr.(*syntax.RangeAggregationExpr).Selector() default: return nil, ErrParseQuery } + + if err != nil { + return nil, err + } + + if selector == nil || selector.HasFilter() { + return nil, ErrParseQuery + } } resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { @@ -118,7 +126,7 @@ func prunePatterns( // ForAllIngesters runs f, in parallel, for all ingesters func (q *IngesterQuerier) forAllIngesters(ctx context.Context, f func(context.Context, logproto.PatternClient) (interface{}, error)) ([]ResponseFromIngesters, error) { - replicationSet, err := q.ringClient.ring.GetReplicationSetForOperation(ring.Read) + replicationSet, err := q.ringClient.Ring().GetReplicationSetForOperation(ring.Read) if err != nil { return nil, err } @@ -137,7 +145,7 @@ func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet // Nothing here } results, err := ring.DoUntilQuorum(ctx, replicationSet, cfg, func(ctx context.Context, ingester *ring.InstanceDesc) (ResponseFromIngesters, error) { - client, err := q.ringClient.pool.GetClientFor(ingester.Addr) + client, err := q.ringClient.Pool().GetClientFor(ingester.Addr) if err != nil { return ResponseFromIngesters{addr: ingester.Addr}, err } diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index c83da105ec2c..809059638401 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -2,11 +2,17 @@ package pattern import ( "bufio" + "context" "os" "testing" + "time" + "github.com/go-kit/log" "github.com/stretchr/testify/require" + "github.com/grafana/dskit/ring" + ring_client "github.com/grafana/dskit/ring/client" + "github.com/grafana/dskit/services" "github.com/grafana/loki/v3/pkg/logproto" ) @@ -39,3 +45,155 @@ func Test_prunePatterns(t *testing.T) { require.Equal(t, expectedPatterns, patterns) } + +func Test_Patterns(t *testing.T) { + t.Run("it rejects metric queries with filters", func(t *testing.T) { + q := &IngesterQuerier{ + cfg: Config{}, + logger: log.NewNopLogger(), + ringClient: &fakeRingClient{}, + registerer: nil, + } + for _, query := range []string{ + `count_over_time({foo="bar"} |= "baz" [5m])`, + `count_over_time({foo="bar"} != "baz" [5m])`, + `count_over_time({foo="bar"} =~ "baz" [5m])`, + `count_over_time({foo="bar"} !~ "baz" [5m])`, + `count_over_time({foo="bar"} | logfmt | color=blue [5m])`, + `sum(count_over_time({foo="bar"} |= "baz" [5m]))`, + `sum by label(count_over_time({foo="bar"} |= "baz" [5m]))`, + `bytes_over_time({foo="bar"} |= "baz" [5m])`, + } { + _, err := q.Patterns( + context.Background(), + &logproto.QueryPatternsRequest{ + Query: query, + }, + ) + require.Error(t, err, query) + require.ErrorIs(t, err, ErrParseQuery) + + } + }) + + t.Run("accepts log selector queries and count and bytes metric queries", func(t *testing.T) { + q := &IngesterQuerier{ + cfg: Config{}, + logger: log.NewNopLogger(), + ringClient: &fakeRingClient{}, + registerer: nil, + } + for _, query := range []string{ + `{foo="bar"}`, + `count_over_time({foo="bar"}[5m])`, + `bytes_over_time({foo="bar"}[5m])`, + `sum(count_over_time({foo="bar"}[5m]))`, + `sum(bytes_over_time({foo="bar"}[5m]))`, + `sum by (level)(count_over_time({foo="bar"}[5m]))`, + `sum by (level)(bytes_over_time({foo="bar"}[5m]))`, + } { + _, err := q.Patterns( + context.Background(), + &logproto.QueryPatternsRequest{ + Query: query, + }, + ) + require.NoError(t, err, query) + } + }) +} + +type fakeRingClient struct{} + +func (f *fakeRingClient) Pool() *ring_client.Pool { + panic("not implemented") +} + +func (f *fakeRingClient) StartAsync(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) AwaitRunning(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) StopAsync() { + panic("not implemented") +} + +func (f *fakeRingClient) AwaitTerminated(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) FailureCase() error { + panic("not implemented") +} + +func (f *fakeRingClient) State() services.State { + panic("not implemented") +} + +func (f *fakeRingClient) AddListener(listener services.Listener) { + panic("not implemented") +} + +func (f *fakeRingClient) Ring() ring.ReadRing { + return &fakeRing{} +} + +type fakeRing struct{} + +func (f *fakeRing) Get( + key uint32, + op ring.Operation, + bufDescs []ring.InstanceDesc, + bufHosts []string, + bufZones []string, +) (ring.ReplicationSet, error) { + panic("not implemented") +} + +func (f *fakeRing) GetAllHealthy(op ring.Operation) (ring.ReplicationSet, error) { + panic("not implemented") +} + +func (f *fakeRing) GetReplicationSetForOperation(op ring.Operation) (ring.ReplicationSet, error) { + return ring.ReplicationSet{}, nil +} + +func (f *fakeRing) ReplicationFactor() int { + panic("not implemented") +} + +func (f *fakeRing) InstancesCount() int { + panic("not implemented") +} + +func (f *fakeRing) ShuffleShard(identifier string, size int) ring.ReadRing { + panic("not implemented") +} + +func (f *fakeRing) GetInstanceState(instanceID string) (ring.InstanceState, error) { + panic("not implemented") +} + +func (f *fakeRing) ShuffleShardWithLookback( + identifier string, + size int, + lookbackPeriod time.Duration, + now time.Time, +) ring.ReadRing { + panic("not implemented") +} + +func (f *fakeRing) HasInstance(instanceID string) bool { + panic("not implemented") +} + +func (f *fakeRing) CleanupShuffleShardCache(identifier string) { + panic("not implemented") +} + +func (f *fakeRing) GetTokenRangesForInstance(instanceID string) (ring.TokenRanges, error) { + panic("not implemented") +} diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index bc681e8faf02..6d0ca3485678 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -93,7 +93,7 @@ func TestInstancePushQuery(t *testing.T) { t.Run("test count_over_time samples", func(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger()) + inst, err := newInstance("foo", log.NewNopLogger(), nil) require.NoError(t, err) err = inst.Push(context.Background(), &push.PushRequest{ @@ -185,7 +185,7 @@ func TestInstancePushQuery(t *testing.T) { t.Run("test bytes_over_time samples", func(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger()) + inst, err := newInstance("foo", log.NewNopLogger(), nil) require.NoError(t, err) err = inst.Push(context.Background(), &push.PushRequest{ diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 7edd5364e6f1..e40d512f969a 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -28,7 +28,7 @@ func TestInstance_QuerySample(t *testing.T) { Step: oneMin, } - instance, err := newInstance("test", log.NewNopLogger()) + instance, err := newInstance("test", log.NewNopLogger(), nil) require.NoError(t, err) labels := model.LabelSet{ diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index a1e301f31276..86a4c83870b9 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -11,6 +11,7 @@ import ( "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -48,9 +49,9 @@ func ExtractMetricType(expr syntax.SampleExpr) (MetricType, error) { } type SampleEvaluatorFactory interface { - // NewStepEvaluator returns a NewStepEvaluator for a given SampleExpr. - // It's explicitly passed another NewStepEvaluator - // in order to enable arbitrary computation of embedded expressions. This allows more modular & extensible + // NewStepEvaluator returns a NewStepEvaluator for a given SampleExpr. + // It's explicitly passed another NewStepEvaluator + // in order to enable arbitrary computation of embedded expressions. This allows more modular & extensible // NewStepEvaluator implementations which can be composed. NewStepEvaluator( ctx context.Context, @@ -161,7 +162,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( } } -// Need to create our own StepEvaluator since we only support bytes and count over time, +// Need to create our own StepEvaluator since we only support bytes and count over time, // and always sum to get those values. In order to accomplish this we need control over the // aggregation operation.. func NewPatternSampleRangeAggEvaluator( @@ -199,7 +200,7 @@ func newRangeVectorIterator( // TODO(twhitney): do I need a streaming aggregator? // if so the aggregator is going to make this // a bit of a bad time, as there's currently no - // way to provide a custom one. + // way to provide a custom one. // // var overlap bool // if selRange >= step && start != end { @@ -222,7 +223,7 @@ func newRangeVectorIterator( // }, nil // } - // always sum + // always sum aggregator := logql.BatchRangeVectorAggregator(func(samples []promql.FPoint) float64 { sum := 0.0 for _, v := range samples { @@ -263,7 +264,6 @@ func (s *SeriesToSampleIterator) Next() bool { current, rest := s.floats[0], s.floats[1:] - //Is timestamp the correct unit here? s.curTs = current.T s.cur = current.F @@ -352,3 +352,7 @@ func (p *paramCompat) GetExpression() syntax.Expr { func (p *paramCompat) GetStoreChunks() *logproto.ChunkRefGroup { return nil } + +func (p *paramCompat) CachingOptions() (res resultscache.CachingOptions) { + return +} diff --git a/pkg/pattern/ring_client.go b/pkg/pattern/ring_client.go index 3ceaf481a3b9..d1421b842422 100644 --- a/pkg/pattern/ring_client.go +++ b/pkg/pattern/ring_client.go @@ -13,7 +13,13 @@ import ( "github.com/grafana/loki/v3/pkg/pattern/clientpool" ) -type RingClient struct { +type RingClient interface { + services.Service + Ring() ring.ReadRing + Pool() *ring_client.Pool +} + +type ringClient struct { cfg Config logger log.Logger @@ -29,10 +35,10 @@ func NewRingClient( metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, -) (*RingClient, error) { +) (RingClient, error) { var err error registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) - ringClient := &RingClient{ + ringClient := &ringClient{ logger: log.With(logger, "component", "pattern-ring-client"), cfg: cfg, } @@ -59,19 +65,81 @@ func NewRingClient( return ringClient, nil } -func (q *RingClient) starting(ctx context.Context) error { - return services.StartManagerAndAwaitHealthy(ctx, q.subservices) +func (r *ringClient) starting(ctx context.Context) error { + return services.StartManagerAndAwaitHealthy(ctx, r.subservices) } -func (q *RingClient) running(ctx context.Context) error { +func (r *ringClient) running(ctx context.Context) error { select { case <-ctx.Done(): return nil - case err := <-q.subservicesWatcher.Chan(): + case err := <-r.subservicesWatcher.Chan(): return fmt.Errorf("pattern tee subservices failed: %w", err) } } -func (q *RingClient) stopping(_ error) error { - return services.StopManagerAndAwaitStopped(context.Background(), q.subservices) +func (r *ringClient) stopping(_ error) error { + return services.StopManagerAndAwaitStopped(context.Background(), r.subservices) +} + +func (r *ringClient) Ring() ring.ReadRing { + return r.ring +} + +func (r *ringClient) Pool() *ring_client.Pool { + return r.pool +} + +// StartAsync starts Service asynchronously. Service must be in New State, otherwise error is returned. +// Context is used as a parent context for service own context. +func (r *ringClient) StartAsync(ctx context.Context) error { + return r.StartAsync(ctx) +} + +// AwaitRunning waits until service gets into Running state. +// If service is in New or Starting state, this method is blocking. +// If service is already in Running state, returns immediately with no error. +// If service is in a state, from which it cannot get into Running state, error is returned immediately. +func (r *ringClient) AwaitRunning(ctx context.Context) error { + return r.AwaitRunning(ctx) +} + +// StopAsync tell the service to stop. This method doesn't block and can be called multiple times. +// If Service is New, it is Terminated without having been started nor stopped. +// If Service is in Starting or Running state, this initiates shutdown and returns immediately. +// If Service has already been stopped, this method returns immediately, without taking action. +func (r *ringClient) StopAsync() { + r.StopAsync() +} + +// AwaitTerminated waits for the service to reach Terminated or Failed state. If service is already in one of these states, +// when method is called, method returns immediately. +// If service enters Terminated state, this method returns nil. +// If service enters Failed state, or context is finished before reaching Terminated or Failed, error is returned. +func (r *ringClient) AwaitTerminated(ctx context.Context) error { + return r.AwaitTerminated(ctx) +} + +// FailureCase returns error if Service is in Failed state. +// If Service is not in Failed state, this method returns nil. +func (r *ringClient) FailureCase() error { + return r.FailureCase() +} + +// State returns current state of the service. +func (r *ringClient) State() services.State { + return r.State() +} + +// AddListener adds listener to this service. Listener will be notified on subsequent state transitions +// of the service. Previous state transitions are not replayed, so it is suggested to add listeners before +// service is started. +// +// AddListener guarantees execution ordering across calls to a given listener but not across calls to +// multiple listeners. Specifically, a given listener will have its callbacks invoked in the same order +// as the service enters those states. Additionally, at most one of the listener's callbacks will execute +// at once. However, multiple listeners' callbacks may execute concurrently, and listeners may execute +// in an order different from the one in which they were registered. +func (r *ringClient) AddListener(listener services.Listener) { + r.AddListener(listener) } diff --git a/pkg/pattern/tee.go b/pkg/pattern/tee.go index 70fb37e1b692..ed90d0dd478c 100644 --- a/pkg/pattern/tee.go +++ b/pkg/pattern/tee.go @@ -18,14 +18,14 @@ import ( type Tee struct { cfg Config logger log.Logger - ringClient *RingClient + ringClient RingClient ingesterAppends *prometheus.CounterVec } func NewTee( cfg Config, - ringClient *RingClient, + ringClient RingClient, metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, @@ -38,8 +38,8 @@ func NewTee( Name: "pattern_ingester_appends_total", Help: "The total number of batch appends sent to pattern ingesters.", }, []string{"ingester", "status"}), - cfg: cfg, - ringClient: ringClient, + cfg: cfg, + ringClient: ringClient, } return t, nil @@ -58,7 +58,7 @@ func (t *Tee) Duplicate(tenant string, streams []distributor.KeyedStream) { func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { var descs [1]ring.InstanceDesc - replicationSet, err := t.ringClient.ring.Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil) + replicationSet, err := t.ringClient.Ring().Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil) if err != nil { return err } @@ -66,7 +66,7 @@ func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { return errors.New("no instances found") } addr := replicationSet.Instances[0].Addr - client, err := t.ringClient.pool.GetClientFor(addr) + client, err := t.ringClient.Pool().GetClientFor(addr) if err != nil { return err } diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index f2fe80566b46..bfe023b90b56 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -34,6 +34,7 @@ import ( logql_log "github.com/grafana/loki/v3/pkg/logql/log" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/logqlmodel" + "github.com/grafana/loki/v3/pkg/pattern" querier_limits "github.com/grafana/loki/v3/pkg/querier/limits" "github.com/grafana/loki/v3/pkg/querier/plan" "github.com/grafana/loki/v3/pkg/storage" @@ -1047,6 +1048,12 @@ func (q *SingleTenantQuerier) Patterns(ctx context.Context, req *logproto.QueryP return nil, httpgrpc.Errorf(http.StatusNotFound, "") } res, err := q.patternQuerier.Patterns(ctx, req) + if err != nil { + if errors.Is(err, pattern.ErrParseQuery) { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + } + return res, err } From 68aa1886fc9cc7903e70234372c3968dc849ae6b Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 23 May 2024 17:03:32 -0600 Subject: [PATCH 03/38] feat: guard aggregation behavior behind a feature flag --- pkg/pattern/ingester.go | 6 +- pkg/pattern/ingester_querier.go | 92 +++++++++++++++++----------- pkg/pattern/ingester_querier_test.go | 21 +++++-- pkg/pattern/ingester_test.go | 29 ++++----- pkg/pattern/instance.go | 41 ++++++++----- pkg/pattern/instance_test.go | 5 +- pkg/pattern/metric/config.go | 16 +++++ pkg/pattern/stream.go | 25 +++++--- pkg/pattern/stream_test.go | 9 ++- pkg/querier/querier.go | 5 +- 10 files changed, 161 insertions(+), 88 deletions(-) create mode 100644 pkg/pattern/metric/config.go diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index e6174faffb6c..a3d6c4ef67a0 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -25,6 +25,7 @@ import ( "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/clientpool" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" util_log "github.com/grafana/loki/v3/pkg/util/log" ) @@ -38,6 +39,7 @@ type Config struct { ConcurrentFlushes int `yaml:"concurrent_flushes"` FlushCheckPeriod time.Duration `yaml:"flush_check_period"` + MetricAggregation metric.AggregationConfig `yaml:"metric_aggregation,omitempty" doc:"description=Configures the metric aggregation and storage behavior of the pattern ingester."` // For testing. factory ring_client.PoolFactory `yaml:"-"` } @@ -49,6 +51,8 @@ func (cfg *Config) RegisterFlags(fs *flag.FlagSet) { fs.BoolVar(&cfg.Enabled, "pattern-ingester.enabled", false, "Flag to enable or disable the usage of the pattern-ingester component.") fs.IntVar(&cfg.ConcurrentFlushes, "pattern-ingester.concurrent-flushes", 32, "How many flushes can happen concurrently from each stream.") fs.DurationVar(&cfg.FlushCheckPeriod, "pattern-ingester.flush-check-period", 30*time.Second, "How often should the ingester see if there are any blocks to flush. The first flush check is delayed by a random time up to 0.8x the flush check period. Additionally, there is +/- 1% jitter added to the interval.") + + cfg.MetricAggregation.RegisterFlagsWithPrefix(fs, "pattern-ingester.") } func (cfg *Config) Validate() error { @@ -335,7 +339,7 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { / inst, ok = i.instances[instanceID] if !ok { var err error - inst, err = newInstance(instanceID, i.logger, i.metrics) + inst, err = newInstance(instanceID, i.logger, i.metrics, i.cfg.MetricAggregation) if err != nil { return nil, err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 41fce2d12398..fc62c7cfdca4 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -45,34 +45,70 @@ func NewIngesterQuerier( } func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { - // validate that a supported query was provided - var expr syntax.Expr _, err := syntax.ParseMatchers(req.Query, true) if err != nil { - expr, err = syntax.ParseSampleExpr(req.Query) - if err != nil { - return nil, ErrParseQuery + // not a pattern query, so either a metric query or an error + if q.cfg.MetricAggregation.Enabled { + return q.queryMetricSamples(ctx, req) } - var selector syntax.LogSelectorExpr - switch expr.(type) { - case *syntax.VectorAggregationExpr: - selector, err = expr.(*syntax.VectorAggregationExpr).Selector() - case *syntax.RangeAggregationExpr: - selector, err = expr.(*syntax.RangeAggregationExpr).Selector() - default: - return nil, ErrParseQuery - } + return nil, err + } - if err != nil { - return nil, err - } + return q.queryPatternSamples(ctx, req) +} - if selector == nil || selector.HasFilter() { - return nil, ErrParseQuery - } +func (q *IngesterQuerier) queryPatternSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { + iterators, err := q.query(ctx, req) + if err != nil { + return nil, err + } + + // TODO(kolesnikovae): Incorporate with pruning + resp, err := iter.ReadPatternsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err + } + return prunePatterns(resp, minClusterSize), nil +} + +func (q *IngesterQuerier) queryMetricSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { + expr, err := syntax.ParseSampleExpr(req.Query) + if err != nil { + return nil, err + } + + var selector syntax.LogSelectorExpr + switch expr.(type) { + case *syntax.VectorAggregationExpr: + selector, err = expr.(*syntax.VectorAggregationExpr).Selector() + case *syntax.RangeAggregationExpr: + selector, err = expr.(*syntax.RangeAggregationExpr).Selector() + default: + return nil, ErrParseQuery } + if err != nil { + return nil, err + } + + if selector == nil || selector.HasFilter() { + return nil, ErrParseQuery + } + + iterators, err := q.query(ctx, req) + if err != nil { + return nil, err + } + + resp, err := iter.ReadMetricsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err + } + return resp, nil +} + +func (q *IngesterQuerier) query(ctx context.Context, req *logproto.QueryPatternsRequest) ([]iter.Iterator, error) { resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { return client.Query(ctx, req) }) @@ -83,21 +119,7 @@ func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatte for i := range resps { iterators[i] = iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) } - switch expr.(type) { - case *syntax.VectorAggregationExpr, *syntax.RangeAggregationExpr: - resp, err := iter.ReadMetricsBatch(iter.NewMerge(iterators...), math.MaxInt32) - if err != nil { - return nil, err - } - return resp, nil - default: - // TODO(kolesnikovae): Incorporate with pruning - resp, err := iter.ReadPatternsBatch(iter.NewMerge(iterators...), math.MaxInt32) - if err != nil { - return nil, err - } - return prunePatterns(resp, minClusterSize), nil - } + return iterators, nil } func prunePatterns( diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index 809059638401..20b3dcae83d7 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -14,6 +14,7 @@ import ( ring_client "github.com/grafana/dskit/ring/client" "github.com/grafana/dskit/services" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/metric" ) func Test_prunePatterns(t *testing.T) { @@ -49,7 +50,11 @@ func Test_prunePatterns(t *testing.T) { func Test_Patterns(t *testing.T) { t.Run("it rejects metric queries with filters", func(t *testing.T) { q := &IngesterQuerier{ - cfg: Config{}, + cfg: Config{ + MetricAggregation: metric.AggregationConfig{ + Enabled: true, + }, + }, logger: log.NewNopLogger(), ringClient: &fakeRingClient{}, registerer: nil, @@ -57,11 +62,11 @@ func Test_Patterns(t *testing.T) { for _, query := range []string{ `count_over_time({foo="bar"} |= "baz" [5m])`, `count_over_time({foo="bar"} != "baz" [5m])`, - `count_over_time({foo="bar"} =~ "baz" [5m])`, + `count_over_time({foo="bar"} |~ "baz" [5m])`, `count_over_time({foo="bar"} !~ "baz" [5m])`, - `count_over_time({foo="bar"} | logfmt | color=blue [5m])`, + `count_over_time({foo="bar"} | logfmt | color="blue" [5m])`, `sum(count_over_time({foo="bar"} |= "baz" [5m]))`, - `sum by label(count_over_time({foo="bar"} |= "baz" [5m]))`, + `sum by (label)(count_over_time({foo="bar"} |= "baz" [5m]))`, `bytes_over_time({foo="bar"} |= "baz" [5m])`, } { _, err := q.Patterns( @@ -71,14 +76,18 @@ func Test_Patterns(t *testing.T) { }, ) require.Error(t, err, query) - require.ErrorIs(t, err, ErrParseQuery) + require.ErrorIs(t, err, ErrParseQuery, query) } }) t.Run("accepts log selector queries and count and bytes metric queries", func(t *testing.T) { q := &IngesterQuerier{ - cfg: Config{}, + cfg: Config{ + MetricAggregation: metric.AggregationConfig{ + Enabled: true, + }, + }, logger: log.NewNopLogger(), ringClient: &fakeRingClient{}, registerer: nil, diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index 6d0ca3485678..9707837a4fab 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -13,17 +13,24 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/pkg/push" ) func TestInstancePushQuery(t *testing.T) { - t.Run("test pattern samples", func(t *testing.T) { - lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test")) + lbs := labels.New(labels.Label{Name: "test", Value: "test"}) + setup := func() *instance { + inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ + Enabled: true, + }) require.NoError(t, err) - err = inst.Push(context.Background(), &push.PushRequest{ + return inst + } + t.Run("test pattern samples", func(t *testing.T) { + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { Labels: lbs.String(), @@ -92,11 +99,8 @@ func TestInstancePushQuery(t *testing.T) { }) t.Run("test count_over_time samples", func(t *testing.T) { - lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger(), nil) - require.NoError(t, err) - - err = inst.Push(context.Background(), &push.PushRequest{ + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { Labels: lbs.String(), @@ -184,11 +188,8 @@ func TestInstancePushQuery(t *testing.T) { }) t.Run("test bytes_over_time samples", func(t *testing.T) { - lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger(), nil) - require.NoError(t, err) - - err = inst.Push(context.Background(), &push.PushRequest{ + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { Labels: lbs.String(), diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index cafc8dec2354..d98b412a85e2 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -25,27 +25,29 @@ const indexShards = 32 // instance is a tenant instance of the pattern ingester. type instance struct { - instanceID string - buf []byte // buffer used to compute fps. - mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free - streams *streamsMap - index *index.BitPrefixInvertedIndex - logger log.Logger - metrics *ingesterMetrics + instanceID string + buf []byte // buffer used to compute fps. + mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free + streams *streamsMap + index *index.BitPrefixInvertedIndex + logger log.Logger + metrics *ingesterMetrics + aggregationCfg metric.AggregationConfig } -func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics) (*instance, error) { +func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics, aggCfg metric.AggregationConfig) (*instance, error) { index, err := index.NewBitPrefixWithShards(indexShards) if err != nil { return nil, err } i := &instance{ - buf: make([]byte, 0, 1024), - logger: logger, - instanceID: instanceID, - streams: newStreamsMap(), - index: index, - metrics: metrics, + buf: make([]byte, 0, 1024), + logger: logger, + instanceID: instanceID, + streams: newStreamsMap(), + index: index, + metrics: metrics, + aggregationCfg: aggCfg, } i.mapper = ingester.NewFPMapper(i.getLabelsFromFingerprint) return i, nil @@ -60,7 +62,7 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels, func() (*stream, error) { // add stream - return i.createStream(ctx, reqStream) + return i.createStream(ctx, reqStream, i.aggregationCfg.Enabled) }, nil) if err != nil { appendErr.Add(err) @@ -107,6 +109,11 @@ func (i *instance) QuerySample( expr syntax.SampleExpr, req *logproto.QueryPatternsRequest, ) (iter.Iterator, error) { + if !i.aggregationCfg.Enabled { + // Should never get here, but this will prevent nil pointer panics in test + return iter.Empty, nil + } + from, through := util.RoundToMilliseconds(req.Start, req.End) step := model.Time(req.Step) if step < chunk.TimeResolution { @@ -184,14 +191,14 @@ outer: return nil } -func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream) (*stream, error) { +func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream, aggregateMetrics bool) (*stream, error) { labels, err := syntax.ParseLabels(pushReqStream.Labels) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } fp := i.getHashForLabels(labels) sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp) - s, err := newStream(fp, sortedLabels, i.metrics) + s, err := newStream(fp, sortedLabels, i.metrics, i.aggregationCfg.Enabled) if err != nil { return nil, fmt.Errorf("failed to create stream: %w", err) } diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index e40d512f969a..1a3699148920 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -9,6 +9,7 @@ import ( "github.com/grafana/loki/pkg/push" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -28,7 +29,7 @@ func TestInstance_QuerySample(t *testing.T) { Step: oneMin, } - instance, err := newInstance("test", log.NewNopLogger(), nil) + instance, err := newInstance("test", log.NewNopLogger(), nil, metric.AggregationConfig{}) require.NoError(t, err) labels := model.LabelSet{ @@ -37,7 +38,7 @@ func TestInstance_QuerySample(t *testing.T) { lastTsMilli := (then + oneMin + oneMin) // 1715964095000 - //TODO(twhitney): Add a few more pushes to this or another test + // TODO(twhitney): Add a few more pushes to this or another test instance.Push(ctx, &logproto.PushRequest{ Streams: []push.Stream{ { diff --git a/pkg/pattern/metric/config.go b/pkg/pattern/metric/config.go new file mode 100644 index 000000000000..8bdf85cdcb64 --- /dev/null +++ b/pkg/pattern/metric/config.go @@ -0,0 +1,16 @@ +package metric + +import "flag" + +type AggregationConfig struct { + Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` +} + +// RegisterFlags registers pattern ingester related flags. +func (cfg *AggregationConfig) RegisterFlags(fs *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix(fs, "") +} + +func (cfg *AggregationConfig) RegisterFlagsWithPrefix(fs *flag.FlagSet, prefix string) { + fs.BoolVar(&cfg.Enabled, prefix+"metric-aggregation.enabled", false, "Flag to enable or disable metric aggregation.") +} diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 24c9e21c96f8..0c08790dd708 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -26,7 +26,9 @@ type stream struct { labelHash uint64 patterns *drain.Drain mtx sync.Mutex - metrics *metric.Chunks + + aggregateMetrics bool + metrics *metric.Chunks evaluator metric.SampleEvaluatorFactory @@ -37,9 +39,9 @@ func newStream( fp model.Fingerprint, labels labels.Labels, metrics *ingesterMetrics, + aggregateMetrics bool, ) (*stream, error) { - chunks := metric.NewChunks(labels) - return &stream{ + stream := &stream{ fp: fp, labels: labels, labelsString: labels.String(), @@ -48,9 +50,16 @@ func newStream( PatternsEvictedTotal: metrics.patternsDiscardedTotal, PatternsDetectedTotal: metrics.patternsDetectedTotal, }), - metrics: chunks, - evaluator: metric.NewDefaultEvaluatorFactory(chunks), - }, nil + aggregateMetrics: aggregateMetrics, + } + + if aggregateMetrics { + chunks := metric.NewChunks(labels) + stream.metrics = chunks + stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) + } + + return stream, nil } func (s *stream) Push( @@ -73,7 +82,9 @@ func (s *stream) Push( s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) } - s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) + if s.aggregateMetrics && s.metrics != nil { + s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) + } return nil } diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index ab0564af9995..7f6f98d15b21 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -16,7 +16,7 @@ import ( func TestAddStream(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test")) + stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test"), false) require.NoError(t, err) err = stream.Push(context.Background(), []push.Entry{ @@ -44,7 +44,12 @@ func TestAddStream(t *testing.T) { func TestPruneStream(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test")) + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + false, + ) require.NoError(t, err) err = stream.Push(context.Background(), []push.Entry{ diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index bfe023b90b56..b05d9411faaf 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -34,7 +34,6 @@ import ( logql_log "github.com/grafana/loki/v3/pkg/logql/log" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/logqlmodel" - "github.com/grafana/loki/v3/pkg/pattern" querier_limits "github.com/grafana/loki/v3/pkg/querier/limits" "github.com/grafana/loki/v3/pkg/querier/plan" "github.com/grafana/loki/v3/pkg/storage" @@ -1049,9 +1048,7 @@ func (q *SingleTenantQuerier) Patterns(ctx context.Context, req *logproto.QueryP } res, err := q.patternQuerier.Patterns(ctx, req) if err != nil { - if errors.Is(err, pattern.ErrParseQuery) { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } return res, err From b897fc5fbeab92d304c7cdb5a66edd93c850fa5c Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 24 May 2024 13:36:00 -0600 Subject: [PATCH 04/38] fix: ring proxy methods on pattern ring_client --- pkg/pattern/ring_client.go | 40 +++++++------------------------------- 1 file changed, 7 insertions(+), 33 deletions(-) diff --git a/pkg/pattern/ring_client.go b/pkg/pattern/ring_client.go index d1421b842422..0d49cfe42379 100644 --- a/pkg/pattern/ring_client.go +++ b/pkg/pattern/ring_client.go @@ -90,56 +90,30 @@ func (r *ringClient) Pool() *ring_client.Pool { return r.pool } -// StartAsync starts Service asynchronously. Service must be in New State, otherwise error is returned. -// Context is used as a parent context for service own context. func (r *ringClient) StartAsync(ctx context.Context) error { - return r.StartAsync(ctx) + return r.ring.StartAsync(ctx) } -// AwaitRunning waits until service gets into Running state. -// If service is in New or Starting state, this method is blocking. -// If service is already in Running state, returns immediately with no error. -// If service is in a state, from which it cannot get into Running state, error is returned immediately. func (r *ringClient) AwaitRunning(ctx context.Context) error { - return r.AwaitRunning(ctx) + return r.ring.AwaitRunning(ctx) } -// StopAsync tell the service to stop. This method doesn't block and can be called multiple times. -// If Service is New, it is Terminated without having been started nor stopped. -// If Service is in Starting or Running state, this initiates shutdown and returns immediately. -// If Service has already been stopped, this method returns immediately, without taking action. func (r *ringClient) StopAsync() { - r.StopAsync() + r.ring.StopAsync() } -// AwaitTerminated waits for the service to reach Terminated or Failed state. If service is already in one of these states, -// when method is called, method returns immediately. -// If service enters Terminated state, this method returns nil. -// If service enters Failed state, or context is finished before reaching Terminated or Failed, error is returned. func (r *ringClient) AwaitTerminated(ctx context.Context) error { - return r.AwaitTerminated(ctx) + return r.ring.AwaitTerminated(ctx) } -// FailureCase returns error if Service is in Failed state. -// If Service is not in Failed state, this method returns nil. func (r *ringClient) FailureCase() error { - return r.FailureCase() + return r.ring.FailureCase() } -// State returns current state of the service. func (r *ringClient) State() services.State { - return r.State() + return r.ring.State() } -// AddListener adds listener to this service. Listener will be notified on subsequent state transitions -// of the service. Previous state transitions are not replayed, so it is suggested to add listeners before -// service is started. -// -// AddListener guarantees execution ordering across calls to a given listener but not across calls to -// multiple listeners. Specifically, a given listener will have its callbacks invoked in the same order -// as the service enters those states. Additionally, at most one of the listener's callbacks will execute -// at once. However, multiple listeners' callbacks may execute concurrently, and listeners may execute -// in an order different from the one in which they were registered. func (r *ringClient) AddListener(listener services.Listener) { - r.AddListener(listener) + r.ring.AddListener(listener) } From 258765772e7815d07849c70f8c774a0902079145 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 24 May 2024 17:35:15 -0600 Subject: [PATCH 05/38] fix: grouping --- cmd/loki/loki-local-config.yaml | 2 + pkg/pattern/ingester.go | 4 +- pkg/pattern/instance.go | 13 +--- pkg/pattern/metric/chunk.go | 19 ++++- pkg/pattern/metric/chunk_test.go | 110 +++++++++++++++++++++++++++ pkg/pattern/metric/evaluator.go | 22 +++--- pkg/pattern/metric/evaluator_test.go | 4 - pkg/pattern/stream.go | 73 +++++------------- 8 files changed, 163 insertions(+), 84 deletions(-) diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 5f717a3d6a81..913b9a4c1873 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -35,6 +35,8 @@ schema_config: pattern_ingester: enabled: true + metric_aggregation: + enabled: true ruler: alertmanager_url: http://localhost:9093 diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index a3d6c4ef67a0..5d3da8ce1227 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -48,11 +48,11 @@ type Config struct { func (cfg *Config) RegisterFlags(fs *flag.FlagSet) { cfg.LifecyclerConfig.RegisterFlagsWithPrefix("pattern-ingester.", fs, util_log.Logger) cfg.ClientConfig.RegisterFlags(fs) + cfg.MetricAggregation.RegisterFlagsWithPrefix(fs, "pattern-ingester.") + fs.BoolVar(&cfg.Enabled, "pattern-ingester.enabled", false, "Flag to enable or disable the usage of the pattern-ingester component.") fs.IntVar(&cfg.ConcurrentFlushes, "pattern-ingester.concurrent-flushes", 32, "How many flushes can happen concurrently from each stream.") fs.DurationVar(&cfg.FlushCheckPeriod, "pattern-ingester.flush-check-period", 30*time.Second, "How often should the ingester see if there are any blocks to flush. The first flush check is delayed by a random time up to 0.8x the flush check period. Additionally, there is +/- 1% jitter added to the interval.") - - cfg.MetricAggregation.RegisterFlagsWithPrefix(fs, "pattern-ingester.") } func (cfg *Config) Validate() error { diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index d98b412a85e2..c5b724e0f6fc 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -125,24 +125,13 @@ func (i *instance) QuerySample( return nil, err } - typ, err := metric.ExtractMetricType(expr) - if err != nil || typ == metric.Unsupported { - return nil, err - } - var iters []iter.Iterator err = i.forMatchingStreams( selector.Matchers(), func(stream *stream) error { var iter iter.Iterator var err error - if typ == metric.Bytes { - iter, err = stream.BytesIterator(ctx, expr, from, through, step) - } else if typ == metric.Count { - iter, err = stream.CountIterator(ctx, expr, from, through, step) - } else { - return fmt.Errorf("unsupported query operation") - } + iter, err = stream.SampleIterator(ctx, expr, from, through, step) if err != nil { return err diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index c4716ba532dd..cbc688461849 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -7,6 +7,7 @@ import ( "time" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/prometheus/common/model" @@ -51,12 +52,23 @@ func (c *Chunks) Observe(bytes, count uint64, ts model.Time) { func (c *Chunks) Iterator( ctx context.Context, typ MetricType, + grouping *syntax.Grouping, from, through, step model.Time, ) (iter.Iterator, error) { if typ == Unsupported { return nil, fmt.Errorf("unsupported metric type") } + lbls := c.labels + if grouping != nil { + sort.Strings(grouping.Groups) + lbls = make(labels.Labels, 0, len(grouping.Groups)) + for _, group := range grouping.Groups { + value := c.labels.Get(group) + lbls = append(lbls, labels.Label{Name: group, Value: value}) + } + } + iters := make([]iter.Iterator, 0, len(c.chunks)) for _, chunk := range c.chunks { samples, err := chunk.ForRangeAndType(typ, from, through, step) @@ -68,9 +80,10 @@ func (c *Chunks) Iterator( continue } - iters = append(iters, iter.NewLabelsSlice(c.labels, samples)) + iters = append(iters, iter.NewLabelsSlice(lbls, samples)) } - return iter.NewNonOverlappingLabelsIterator(c.labels, iters), nil + + return iter.NewNonOverlappingLabelsIterator(lbls, iters), nil } // TODO(twhitney): These values should be float64s (to match prometheus samples) or int64s (to match pattern samples) @@ -127,7 +140,7 @@ func (c *Chunk) spaceFor(ts model.Time) bool { return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime } -//TODO(twhitney): any way to remove the duplication between this and the drain chunk ForRange method? +// TODO(twhitney): any way to remove the duplication between this and the drain chunk ForRange method? // ForRangeAndType returns samples with only the values // in the given range [start:end] and aggregates them by step duration. // start and end are in milliseconds since epoch. step is a duration in milliseconds. diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 32b746b0d20d..4ae6a0575358 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -1,11 +1,15 @@ package metric import ( + "context" "reflect" "testing" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" ) @@ -327,3 +331,109 @@ func TestForRangeAndType(t *testing.T) { }) } } + +func Test_Chunks_Iterator(t *testing.T) { + ctx := context.Background() + lbls := labels.Labels{ + labels.Label{Name: "foo", Value: "bar"}, + labels.Label{Name: "container", Value: "jar"}, + } + chunks := Chunks{ + chunks: []Chunk{ + { + Samples: []MetricSample{ + {Timestamp: 2, Bytes: 2, Count: 1}, + {Timestamp: 4, Bytes: 4, Count: 3}, + {Timestamp: 6, Bytes: 6, Count: 5}, + }, + mint: 2, + maxt: 6, + }, + }, + labels: lbls, + } + + t.Run("without grouping", func(t *testing.T) { + it, err := chunks.Iterator(ctx, Bytes, nil, 0, 10, 2) + require.NoError(t, err) + + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) + + it, err = chunks.Iterator(ctx, Count, nil, 0, 10, 2) + require.NoError(t, err) + + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) + }) + + t.Run("grouping", func(t *testing.T) { + grouping := &syntax.Grouping{ + Groups: []string{"container"}, + Without: false, + } + + expectedLabels := labels.Labels{ + labels.Label{ + Name: "container", + Value: "jar", + }, + } + + it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) + require.NoError(t, err) + + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) + + it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) + require.NoError(t, err) + + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) + }) + + t.Run("grouping by a missing label", func(t *testing.T) { + grouping := &syntax.Grouping{ + Groups: []string{"missing"}, + Without: false, + } + + expectedLabels := labels.Labels{ + labels.Label{ + Name: "missing", + Value: "", + }, + } + + it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) + require.NoError(t, err) + + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) + + it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) + require.NoError(t, err) + + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) + }) +} diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index 86a4c83870b9..e4c7368a959e 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -19,7 +19,7 @@ import ( ) // TODO(twhitney): duplication with code in NewStepEvaluator -func ExtractMetricType(expr syntax.SampleExpr) (MetricType, error) { +func extractMetricType(expr syntax.SampleExpr) (MetricType, error) { var typ MetricType switch e := expr.(type) { case *syntax.VectorAggregationExpr: @@ -57,7 +57,6 @@ type SampleEvaluatorFactory interface { ctx context.Context, nextEvaluatorFactory SampleEvaluatorFactory, expr syntax.SampleExpr, - typ MetricType, from, through, step model.Time, ) (logql.StepEvaluator, error) } @@ -66,7 +65,6 @@ type SampleEvaluatorFunc func( ctx context.Context, nextEvaluatorFactory SampleEvaluatorFactory, expr syntax.SampleExpr, - typ MetricType, from, through, step model.Time, ) (logql.StepEvaluator, error) @@ -74,10 +72,9 @@ func (s SampleEvaluatorFunc) NewStepEvaluator( ctx context.Context, nextEvaluatorFactory SampleEvaluatorFactory, expr syntax.SampleExpr, - typ MetricType, from, through, step model.Time, ) (logql.StepEvaluator, error) { - return s(ctx, nextEvaluatorFactory, expr, typ, from, through, step) + return s(ctx, nextEvaluatorFactory, expr, from, through, step) } type DefaultEvaluatorFactory struct { @@ -94,9 +91,13 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( ctx context.Context, evFactory SampleEvaluatorFactory, expr syntax.SampleExpr, - typ MetricType, from, through, step model.Time, ) (logql.StepEvaluator, error) { + metricType, err := extractMetricType(expr) + if err != nil || metricType == Unsupported { + return nil, err + } + switch e := expr.(type) { case *syntax.VectorAggregationExpr: if rangExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { @@ -106,12 +107,11 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( func(ctx context.Context, _ SampleEvaluatorFactory, _ syntax.SampleExpr, - typ MetricType, from, through, step model.Time, ) (logql.StepEvaluator, error) { fromWithRangeAndOffset := from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset) throughWithOffset := through.Add(-rangExpr.Left.Offset) - it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + it, err := ev.chunks.Iterator(ctx, metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) if err != nil { return nil, err } @@ -129,7 +129,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( if e.Grouping == nil { return nil, errors.Errorf("aggregation operator '%q' without grouping", e.Operation) } - nextEvaluator, err := evFactory.NewStepEvaluator(ctx, evFactory, e.Left, typ, from, through, step) + nextEvaluator, err := evFactory.NewStepEvaluator(ctx, evFactory, e.Left, from, through, step) if err != nil { return nil, err } @@ -145,7 +145,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( case *syntax.RangeAggregationExpr: fromWithRangeAndOffset := from.Add(-e.Left.Interval).Add(-e.Left.Offset) throughWithOffset := through.Add(-e.Left.Offset) - it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + it, err := ev.chunks.Iterator(ctx, metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) if err != nil { return nil, err } @@ -250,6 +250,8 @@ type SeriesToSampleIterator struct { lbls labels.Labels } +// TODO: could this me a matrix iterator that returned multiple samples with +// different labels for the same timestamp? func NewSeriesToSampleIterator(series *promql.Series) *SeriesToSampleIterator { return &SeriesToSampleIterator{ floats: series.Floats, diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 9f23cb5546e6..62440a354a64 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -30,14 +30,10 @@ func Test_SampleEvaluator(t *testing.T) { expr, err := syntax.ParseSampleExpr(query) require.NoError(t, err) - typ, err := ExtractMetricType(expr) - require.NoError(t, err) - evaluator, err := factory.NewStepEvaluator( context.Background(), factory, expr.(syntax.SampleExpr), - typ, model.Time(now-fiveMin), model.Time(now), model.Time(fiveMin), ) diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 0c08790dd708..de3538d7b05b 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -2,6 +2,7 @@ package pattern import ( "context" + "errors" "math" "sync" "time" @@ -105,8 +106,7 @@ func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (it return iter.NewMerge(iters...), nil } -// TODO(twhitney): duplication between bytes and count iterators -func (s *stream) BytesIterator( +func (s *stream) SampleIterator( ctx context.Context, expr syntax.SampleExpr, from, through, step model.Time, @@ -118,7 +118,6 @@ func (s *stream) BytesIterator( ctx, s.evaluator, expr, - metric.Bytes, from, through, step, @@ -150,6 +149,7 @@ func (s *stream) BytesIterator( return metric.NewSeriesToSampleIterator(series), nil } +//TODO: should this join multiple series into a matrix, so we don't have the weird hack? func (s *stream) JoinSampleVector( next bool, ts int64, @@ -163,11 +163,6 @@ func (s *stream) JoinSampleVector( stepCount = 1 } - series := &promql.Series{ - Metric: s.labels, - Floats: make([]promql.FPoint, 0, stepCount), - } - vec := promql.Vector{} if next { vec = r.SampleVector() @@ -178,10 +173,22 @@ func (s *stream) JoinSampleVector( return nil, logqlmodel.NewSeriesLimitError(maxSeries) } + var seriesHash string + series := map[string]*promql.Series{} for next { vec = r.SampleVector() for _, p := range vec { - series.Floats = append(series.Floats, promql.FPoint{ + seriesHash = p.Metric.String() + s, ok := series[seriesHash] + if !ok { + s = &promql.Series{ + Metric: p.Metric, + Floats: make([]promql.FPoint, 0, stepCount), + } + series[p.Metric.String()] = s + } + + s.Floats = append(s.Floats, promql.FPoint{ T: ts, F: p.F, }) @@ -193,52 +200,12 @@ func (s *stream) JoinSampleVector( } } - return series, stepEvaluator.Error() -} - -// TODO(twhitney): duplication between bytes and count iterators -func (s *stream) CountIterator( - ctx context.Context, - expr syntax.SampleExpr, - from, through, step model.Time, -) (iter.Iterator, error) { - s.mtx.Lock() - defer s.mtx.Unlock() - - stepEvaluator, err := s.evaluator.NewStepEvaluator( - ctx, - s.evaluator, - expr, - metric.Count, - from, - through, - step, - ) - if err != nil { - return nil, err - } - - next, ts, r := stepEvaluator.Next() - if stepEvaluator.Error() != nil { - return nil, stepEvaluator.Error() + if len(series) > 1 { + // TODO: is this actually a problem? Should this just become a Matrix + return nil, errors.New("multiple series found in a single stream") } - // TODO(twhitney): actually get max series from limits - // this is only 1 series since we're already on a stream - // this this limit needs to also be enforced higher up - maxSeries := 1000 - series, err := s.JoinSampleVector( - next, - ts, - r, - stepEvaluator, - maxSeries, - from, through, step) - if err != nil { - return nil, err - } - - return metric.NewSeriesToSampleIterator(series), nil + return series[seriesHash], stepEvaluator.Error() } func (s *stream) prune(olderThan time.Duration) bool { From 6dd77ae08c9ce08a52e49425fcb150baad6e3787 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 12:56:05 -0600 Subject: [PATCH 06/38] feat: refactor metric samples to be it's own endpoint --- pkg/logproto/extensions.go | 18 +- pkg/logproto/extensions_test.go | 65 +- pkg/logproto/pattern.pb.go | 983 +++++++++++++++++++----- pkg/logproto/pattern.proto | 30 +- pkg/logql/range_vector.go | 1 + pkg/pattern/drain/chunk.go | 13 +- pkg/pattern/drain/chunk_test.go | 13 +- pkg/pattern/flush_test.go | 4 +- pkg/pattern/ingester.go | 108 +-- pkg/pattern/ingester_querier.go | 63 +- pkg/pattern/ingester_querier_test.go | 39 +- pkg/pattern/ingester_test.go | 201 +++-- pkg/pattern/instance.go | 22 +- pkg/pattern/instance_test.go | 68 +- pkg/pattern/iter/batch.go | 61 +- pkg/pattern/iter/batch_test.go | 8 +- pkg/pattern/iter/iterator.go | 176 +---- pkg/pattern/iter/iterator_test.go | 135 +--- pkg/pattern/iter/merge.go | 14 +- pkg/pattern/iter/merge_test.go | 188 ++--- pkg/pattern/iter/query_client.go | 69 +- pkg/pattern/metric/chunk.go | 134 ++-- pkg/pattern/metric/chunk_test.go | 179 ++--- pkg/pattern/metric/evaluator.go | 49 +- pkg/pattern/metric/evaluator_test.go | 247 +++--- pkg/pattern/stream.go | 67 +- pkg/pattern/stream_test.go | 4 +- pkg/querier/querier.go | 18 +- pkg/querier/querier_mock_test.go | 15 + pkg/querier/queryrange/codec.go | 4 + pkg/querier/queryrange/extensions.go | 27 +- pkg/querier/queryrange/queryrange.pb.go | 788 ++++++++++++++++--- pkg/querier/queryrange/queryrange.proto | 9 + pkg/util/marshal/marshal.go | 56 +- pkg/util/marshal/marshal_test.go | 181 +++-- 35 files changed, 2583 insertions(+), 1474 deletions(-) diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index bd726bbb0d35..9ca73b7ecbb4 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -161,8 +161,7 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { var v struct { Status string `json:"status"` Data []struct { - Pattern string `json:"pattern,omitempty"` - Labels string `json:"labels,omitempty"` + Pattern string `json:"pattern"` Samples [][]int64 `json:"samples"` } `json:"data"` } @@ -175,12 +174,7 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { for _, s := range d.Samples { samples = append(samples, &PatternSample{Timestamp: model.TimeFromUnix(s[0]), Value: s[1]}) } - - if pattern := d.Pattern; pattern != "" { - r.Series = append(r.Series, NewPatternSeriesWithPattern(pattern, samples)) - } else if labels := d.Labels; labels != "" { - r.Series = append(r.Series, NewPatternSeriesWithLabels(labels, samples)) - } + r.Series = append(r.Series, &PatternSeries{Pattern: d.Pattern, Samples: samples}) } return nil } @@ -195,10 +189,6 @@ func (m *ShardsResponse) Merge(other *ShardsResponse) { m.Statistics.Merge(other.Statistics) } -func NewPatternSeriesWithPattern(pattern string, samples []*PatternSample) *PatternSeries { - return &PatternSeries{Identifier: &PatternSeries_Pattern{pattern}, Samples: samples} -} - -func NewPatternSeriesWithLabels(labels string, samples []*PatternSample) *PatternSeries { - return &PatternSeries{Identifier: &PatternSeries_Labels{labels}, Samples: samples} +func NewPatternSeries(pattern string, samples []*PatternSample) *PatternSeries { + return &PatternSeries{Pattern: pattern, Samples: samples} } diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go index de8e84e34cd3..95afc117bb9b 100644 --- a/pkg/logproto/extensions_test.go +++ b/pkg/logproto/extensions_test.go @@ -43,8 +43,7 @@ func TestShard_SpaceFor(t *testing.T) { } func TestQueryPatternsResponse_UnmarshalJSON(t *testing.T) { - t.Run("unmarshals patterns", func(t *testing.T) { - mockData := []byte(`{ + mockData := []byte(`{ "status": "success", "data": [ { @@ -58,54 +57,20 @@ func TestQueryPatternsResponse_UnmarshalJSON(t *testing.T) { ] }`) - expectedSeries := []*PatternSeries{ - NewPatternSeriesWithPattern("foo <*> bar", []*PatternSample{ - {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, - {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, - }), - NewPatternSeriesWithPattern("foo <*> buzz", []*PatternSample{ - {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, - {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, - }), - } - - r := &QueryPatternsResponse{} - err := r.UnmarshalJSON(mockData) - - require.Nil(t, err) - require.Equal(t, expectedSeries, r.Series) - }) - - t.Run("unmarshals labels", func(t *testing.T) { - mockData := []byte(`{ - "status": "success", - "data": [ - { - "labels": "{foo=\"bar\"}", - "samples": [[1609459200, 10], [1609545600, 15]] - }, - { - "labels": "{foo=\"buzz\"}", - "samples": [[1609459200, 20], [1609545600, 25]] - } - ] - }`) - - expectedSeries := []*PatternSeries{ - NewPatternSeriesWithLabels(`{foo="bar"}`, []*PatternSample{ - {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, - {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, - }), - NewPatternSeriesWithLabels(`{foo="buzz"}`, []*PatternSample{ - {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, - {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, - }), - } + expectedSeries := []*PatternSeries{ + NewPatternSeries("foo <*> bar", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, + }), + NewPatternSeries("foo <*> buzz", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, + }), + } - r := &QueryPatternsResponse{} - err := r.UnmarshalJSON(mockData) + r := &QueryPatternsResponse{} + err := r.UnmarshalJSON(mockData) - require.Nil(t, err) - require.Equal(t, expectedSeries, r.Series) - }) + require.Nil(t, err) + require.Equal(t, expectedSeries, r.Series) } diff --git a/pkg/logproto/pattern.pb.go b/pkg/logproto/pattern.pb.go index facf0b4bfa90..9fb0cd5d73e2 100644 --- a/pkg/logproto/pattern.pb.go +++ b/pkg/logproto/pattern.pb.go @@ -11,6 +11,7 @@ import ( _ "github.com/gogo/protobuf/types" github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" push "github.com/grafana/loki/pkg/push" + stats "github.com/grafana/loki/v3/pkg/logqlmodel/stats" github_com_prometheus_common_model "github.com/prometheus/common/model" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" @@ -146,11 +147,8 @@ func (m *QueryPatternsResponse) GetSeries() []*PatternSeries { } type PatternSeries struct { - // Types that are valid to be assigned to Identifier: - // *PatternSeries_Pattern - // *PatternSeries_Labels - Identifier isPatternSeries_Identifier `protobuf_oneof:"identifier"` - Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` + Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3" json:"pattern,omitempty"` + Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` } func (m *PatternSeries) Reset() { *m = PatternSeries{} } @@ -185,40 +183,9 @@ func (m *PatternSeries) XXX_DiscardUnknown() { var xxx_messageInfo_PatternSeries proto.InternalMessageInfo -type isPatternSeries_Identifier interface { - isPatternSeries_Identifier() - Equal(interface{}) bool - MarshalTo([]byte) (int, error) - Size() int -} - -type PatternSeries_Pattern struct { - Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3,oneof"` -} -type PatternSeries_Labels struct { - Labels string `protobuf:"bytes,3,opt,name=labels,proto3,oneof"` -} - -func (*PatternSeries_Pattern) isPatternSeries_Identifier() {} -func (*PatternSeries_Labels) isPatternSeries_Identifier() {} - -func (m *PatternSeries) GetIdentifier() isPatternSeries_Identifier { - if m != nil { - return m.Identifier - } - return nil -} - func (m *PatternSeries) GetPattern() string { - if x, ok := m.GetIdentifier().(*PatternSeries_Pattern); ok { - return x.Pattern - } - return "" -} - -func (m *PatternSeries) GetLabels() string { - if x, ok := m.GetIdentifier().(*PatternSeries_Labels); ok { - return x.Labels + if m != nil { + return m.Pattern } return "" } @@ -230,14 +197,6 @@ func (m *PatternSeries) GetSamples() []*PatternSample { return nil } -// XXX_OneofWrappers is for the internal use of the proto package. -func (*PatternSeries) XXX_OneofWrappers() []interface{} { - return []interface{}{ - (*PatternSeries_Pattern)(nil), - (*PatternSeries_Labels)(nil), - } -} - type PatternSample struct { Timestamp github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=timestamp,proto3,customtype=github.com/prometheus/common/model.Time" json:"timestamp"` Value int64 `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` @@ -282,49 +241,176 @@ func (m *PatternSample) GetValue() int64 { return 0 } +type QuerySamplesRequest struct { + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` + Start time.Time `protobuf:"bytes,2,opt,name=start,proto3,stdtime" json:"start"` + End time.Time `protobuf:"bytes,3,opt,name=end,proto3,stdtime" json:"end"` + Step int64 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` +} + +func (m *QuerySamplesRequest) Reset() { *m = QuerySamplesRequest{} } +func (*QuerySamplesRequest) ProtoMessage() {} +func (*QuerySamplesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_aaf4192acc66a4ea, []int{4} +} +func (m *QuerySamplesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QuerySamplesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_QuerySamplesRequest.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 *QuerySamplesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QuerySamplesRequest.Merge(m, src) +} +func (m *QuerySamplesRequest) XXX_Size() int { + return m.Size() +} +func (m *QuerySamplesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_QuerySamplesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_QuerySamplesRequest proto.InternalMessageInfo + +func (m *QuerySamplesRequest) GetQuery() string { + if m != nil { + return m.Query + } + return "" +} + +func (m *QuerySamplesRequest) GetStart() time.Time { + if m != nil { + return m.Start + } + return time.Time{} +} + +func (m *QuerySamplesRequest) GetEnd() time.Time { + if m != nil { + return m.End + } + return time.Time{} +} + +func (m *QuerySamplesRequest) GetStep() int64 { + if m != nil { + return m.Step + } + return 0 +} + +type QuerySamplesResponse struct { + Series []Series `protobuf:"bytes,1,rep,name=series,proto3,customtype=Series" json:"series,omitempty"` + Stats stats.Ingester `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats"` + Warnings []string `protobuf:"bytes,3,rep,name=warnings,proto3" json:"warnings,omitempty"` +} + +func (m *QuerySamplesResponse) Reset() { *m = QuerySamplesResponse{} } +func (*QuerySamplesResponse) ProtoMessage() {} +func (*QuerySamplesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_aaf4192acc66a4ea, []int{5} +} +func (m *QuerySamplesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QuerySamplesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_QuerySamplesResponse.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 *QuerySamplesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QuerySamplesResponse.Merge(m, src) +} +func (m *QuerySamplesResponse) XXX_Size() int { + return m.Size() +} +func (m *QuerySamplesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_QuerySamplesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_QuerySamplesResponse proto.InternalMessageInfo + +func (m *QuerySamplesResponse) GetStats() stats.Ingester { + if m != nil { + return m.Stats + } + return stats.Ingester{} +} + +func (m *QuerySamplesResponse) GetWarnings() []string { + if m != nil { + return m.Warnings + } + return nil +} + func init() { proto.RegisterType((*QueryPatternsRequest)(nil), "logproto.QueryPatternsRequest") proto.RegisterType((*QueryPatternsResponse)(nil), "logproto.QueryPatternsResponse") proto.RegisterType((*PatternSeries)(nil), "logproto.PatternSeries") proto.RegisterType((*PatternSample)(nil), "logproto.PatternSample") + proto.RegisterType((*QuerySamplesRequest)(nil), "logproto.QuerySamplesRequest") + proto.RegisterType((*QuerySamplesResponse)(nil), "logproto.QuerySamplesResponse") } func init() { proto.RegisterFile("pkg/logproto/pattern.proto", fileDescriptor_aaf4192acc66a4ea) } var fileDescriptor_aaf4192acc66a4ea = []byte{ - // 511 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0xbd, 0x6e, 0xd4, 0x40, - 0x10, 0xf6, 0xc6, 0xf7, 0x93, 0x6c, 0xa0, 0x59, 0x2e, 0x60, 0x19, 0x69, 0x7d, 0x72, 0xc3, 0x55, - 0x5e, 0xb8, 0x48, 0x20, 0x51, 0x5e, 0x95, 0x02, 0xa4, 0x60, 0xa8, 0x90, 0x28, 0x7c, 0xb9, 0x39, - 0xdb, 0x8a, 0xed, 0x75, 0xbc, 0xeb, 0x48, 0x74, 0x54, 0xd4, 0xf7, 0x18, 0x3c, 0x00, 0x0f, 0x91, - 0xf2, 0xca, 0x88, 0x22, 0x70, 0xbe, 0x86, 0x32, 0x8f, 0x80, 0xbc, 0x6b, 0xe7, 0x2e, 0x11, 0x29, - 0xd2, 0xd8, 0x33, 0xf3, 0x7d, 0x33, 0xfb, 0xed, 0xcc, 0x2c, 0xb6, 0xf3, 0xd3, 0x90, 0x25, 0x3c, - 0xcc, 0x0b, 0x2e, 0x39, 0xcb, 0x03, 0x29, 0xa1, 0xc8, 0x3c, 0xe5, 0x91, 0xdd, 0x36, 0x6e, 0x0f, - 0x42, 0x1e, 0x72, 0x4d, 0xa9, 0x2d, 0x8d, 0xdb, 0x4e, 0xc8, 0x79, 0x98, 0x00, 0x53, 0xde, 0xb4, - 0x9c, 0x33, 0x19, 0xa7, 0x20, 0x64, 0x90, 0xe6, 0x0d, 0xe1, 0xf9, 0xad, 0xe2, 0xad, 0xd1, 0x80, - 0x4f, 0x6a, 0x30, 0x2f, 0x45, 0xa4, 0x3e, 0x3a, 0xe8, 0xfe, 0x44, 0x78, 0xf0, 0xa1, 0x84, 0xe2, - 0xeb, 0xb1, 0x56, 0x22, 0x7c, 0x38, 0x2b, 0x41, 0x48, 0x32, 0xc0, 0xdd, 0xb3, 0x3a, 0x6e, 0xa1, - 0x21, 0x1a, 0xed, 0xf9, 0xda, 0x21, 0x6f, 0x71, 0x57, 0xc8, 0xa0, 0x90, 0xd6, 0xce, 0x10, 0x8d, - 0xf6, 0xc7, 0xb6, 0xa7, 0x15, 0x79, 0xad, 0x22, 0xef, 0x53, 0xab, 0x68, 0xb2, 0x7b, 0x71, 0xe5, - 0x18, 0x8b, 0xdf, 0x0e, 0xf2, 0x75, 0x0a, 0x79, 0x8d, 0x4d, 0xc8, 0x66, 0x96, 0xf9, 0x80, 0xcc, - 0x3a, 0x81, 0x10, 0xdc, 0x11, 0x12, 0x72, 0xab, 0x33, 0x44, 0x23, 0xd3, 0x57, 0xb6, 0x7b, 0x84, - 0x0f, 0xee, 0xa8, 0x16, 0x39, 0xcf, 0x04, 0x10, 0x86, 0x7b, 0x02, 0x8a, 0x18, 0x84, 0x85, 0x86, - 0xe6, 0x68, 0x7f, 0xfc, 0xcc, 0xbb, 0xe9, 0x42, 0xc3, 0xfd, 0xa8, 0x60, 0xbf, 0xa1, 0xb9, 0xdf, - 0x11, 0x7e, 0x7c, 0x0b, 0x21, 0x36, 0xee, 0x37, 0x63, 0xd1, 0x77, 0x3f, 0x32, 0xfc, 0x36, 0x40, - 0x2c, 0xdc, 0x4b, 0x82, 0x29, 0x24, 0x42, 0x5d, 0xa3, 0x86, 0x1a, 0x9f, 0xbc, 0xc2, 0x7d, 0x11, - 0xa4, 0x79, 0x02, 0xc2, 0xda, 0xb9, 0xef, 0x64, 0x85, 0xfb, 0x2d, 0x6f, 0xf2, 0x08, 0xe3, 0x78, - 0x06, 0x99, 0x8c, 0xe7, 0x31, 0x14, 0xae, 0xdc, 0xe8, 0x50, 0x38, 0x79, 0x8f, 0xf7, 0x6e, 0xe6, - 0xab, 0x94, 0x98, 0x13, 0x56, 0x77, 0xe6, 0xd7, 0x95, 0xf3, 0x22, 0x8c, 0x65, 0x54, 0x4e, 0xbd, - 0x13, 0x9e, 0xd6, 0xcb, 0x90, 0x82, 0x8c, 0xa0, 0x14, 0xec, 0x84, 0xa7, 0x29, 0xcf, 0x58, 0xca, - 0x67, 0x90, 0xa8, 0x7e, 0xfa, 0x9b, 0x0a, 0xf5, 0x40, 0xcf, 0x83, 0xa4, 0x04, 0x35, 0x3a, 0xd3, - 0xd7, 0xce, 0x78, 0x81, 0x70, 0xbf, 0x39, 0x96, 0xbc, 0xc1, 0x9d, 0xe3, 0x52, 0x44, 0xe4, 0x60, - 0x4b, 0x79, 0x29, 0xa2, 0x66, 0x23, 0xec, 0xa7, 0x77, 0xc3, 0xba, 0xe5, 0xae, 0x41, 0xde, 0xe1, - 0xae, 0x9a, 0x06, 0xa1, 0x1b, 0xca, 0xff, 0x96, 0xca, 0x76, 0xee, 0xc5, 0xdb, 0x5a, 0x2f, 0xd1, - 0xe4, 0xcb, 0x72, 0x45, 0x8d, 0xcb, 0x15, 0x35, 0xae, 0x57, 0x14, 0x7d, 0xab, 0x28, 0xfa, 0x51, - 0x51, 0x74, 0x51, 0x51, 0xb4, 0xac, 0x28, 0xfa, 0x53, 0x51, 0xf4, 0xb7, 0xa2, 0xc6, 0x75, 0x45, - 0xd1, 0x62, 0x4d, 0x8d, 0xe5, 0x9a, 0x1a, 0x97, 0x6b, 0x6a, 0x7c, 0xde, 0x6e, 0x49, 0x58, 0x04, - 0xf3, 0x20, 0x0b, 0x58, 0xc2, 0x4f, 0x63, 0x76, 0x7e, 0xc8, 0xb6, 0x5f, 0xc5, 0xb4, 0xa7, 0x7e, - 0x87, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x1c, 0x3a, 0x0e, 0x65, 0x89, 0x03, 0x00, 0x00, + // 597 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x54, 0xbd, 0x6e, 0xd4, 0x40, + 0x10, 0xf6, 0xc6, 0x97, 0xbf, 0x8d, 0xf8, 0xd1, 0x26, 0x01, 0xcb, 0x88, 0xb5, 0xe5, 0x86, 0x93, + 0x90, 0xbc, 0x70, 0x91, 0x00, 0x51, 0x5e, 0x05, 0x12, 0x48, 0x87, 0xa1, 0x42, 0x50, 0xf8, 0x92, + 0x8d, 0xef, 0x14, 0xdb, 0xeb, 0xf3, 0xae, 0x83, 0xe8, 0x78, 0x84, 0xb4, 0xbc, 0x01, 0x3d, 0x3c, + 0xc4, 0x95, 0x57, 0x46, 0x29, 0x0e, 0xce, 0xd7, 0x50, 0xe6, 0x11, 0x90, 0x77, 0xd7, 0xf7, 0x27, + 0xae, 0xa0, 0xa4, 0x59, 0xef, 0xcc, 0xf7, 0xcd, 0x78, 0x66, 0xbe, 0xb1, 0xa1, 0x9d, 0x9d, 0x45, + 0x24, 0x66, 0x51, 0x96, 0x33, 0xc1, 0x48, 0x16, 0x0a, 0x41, 0xf3, 0xd4, 0x97, 0x16, 0xda, 0xa9, + 0xfd, 0xf6, 0x41, 0xc4, 0x22, 0xa6, 0x28, 0xd5, 0x4d, 0xe1, 0xb6, 0x13, 0x31, 0x16, 0xc5, 0x94, + 0x48, 0xab, 0x5b, 0x9c, 0x12, 0xd1, 0x4f, 0x28, 0x17, 0x61, 0x92, 0x69, 0xc2, 0xbd, 0xa5, 0xe4, + 0xf5, 0x45, 0x83, 0xfb, 0x15, 0x98, 0x15, 0xbc, 0x27, 0x0f, 0xed, 0x74, 0x75, 0xc4, 0x20, 0x4e, + 0xd8, 0x09, 0x8d, 0x09, 0x17, 0xa1, 0xe0, 0xea, 0x54, 0x0c, 0xef, 0x07, 0x80, 0x07, 0x6f, 0x0a, + 0x9a, 0x7f, 0xee, 0xa8, 0x5a, 0x79, 0x40, 0x07, 0x05, 0xe5, 0x02, 0x1d, 0xc0, 0xcd, 0x41, 0xe5, + 0xb7, 0x80, 0x0b, 0x9a, 0xbb, 0x81, 0x32, 0xd0, 0x73, 0xb8, 0xc9, 0x45, 0x98, 0x0b, 0x6b, 0xc3, + 0x05, 0xcd, 0xbd, 0x96, 0xed, 0xab, 0x9a, 0xfd, 0xba, 0x66, 0xff, 0x5d, 0x5d, 0x73, 0x7b, 0x67, + 0x38, 0x76, 0x8c, 0x8b, 0x9f, 0x0e, 0x08, 0x54, 0x08, 0x7a, 0x02, 0x4d, 0x9a, 0x9e, 0x58, 0xe6, + 0x3f, 0x44, 0x56, 0x01, 0x08, 0xc1, 0x06, 0x17, 0x34, 0xb3, 0x1a, 0x2e, 0x68, 0x9a, 0x81, 0xbc, + 0x7b, 0x2f, 0xe0, 0xe1, 0x4a, 0xd5, 0x3c, 0x63, 0x29, 0xa7, 0x88, 0xc0, 0x2d, 0x4e, 0xf3, 0x3e, + 0xe5, 0x16, 0x70, 0xcd, 0xe6, 0x5e, 0xeb, 0xae, 0x3f, 0x9b, 0x93, 0xe6, 0xbe, 0x95, 0x70, 0xa0, + 0x69, 0xde, 0x07, 0x78, 0x63, 0x09, 0x40, 0x16, 0xdc, 0xd6, 0xba, 0xe9, 0xd6, 0x6b, 0x13, 0x3d, + 0x86, 0xdb, 0x3c, 0x4c, 0xb2, 0x98, 0x72, 0x6b, 0x63, 0x5d, 0x72, 0x89, 0x07, 0x35, 0xcf, 0x13, + 0xf3, 0xec, 0xd2, 0x83, 0x5e, 0xc3, 0xdd, 0x99, 0xac, 0x32, 0xbf, 0xd9, 0x26, 0x55, 0xbb, 0x57, + 0x63, 0xe7, 0x41, 0xd4, 0x17, 0xbd, 0xa2, 0xeb, 0x1f, 0xb3, 0xa4, 0xda, 0x81, 0x84, 0x8a, 0x1e, + 0x2d, 0x38, 0x39, 0x66, 0x49, 0xc2, 0x52, 0x22, 0x05, 0x94, 0x43, 0x0a, 0xe6, 0x19, 0x2a, 0x95, + 0xce, 0xc3, 0xb8, 0xa0, 0x52, 0x0f, 0x33, 0x50, 0x86, 0xf7, 0x1d, 0xc0, 0x7d, 0x39, 0x1e, 0xf5, + 0xd2, 0xff, 0x44, 0xd3, 0xaf, 0xf5, 0x2a, 0xce, 0xaa, 0xd6, 0x9a, 0x3e, 0x5b, 0xd1, 0xf4, 0xf6, + 0x7c, 0xec, 0x4a, 0xb3, 0xf6, 0xcd, 0xe1, 0xd8, 0x01, 0x57, 0x63, 0x67, 0x6b, 0x59, 0x5c, 0xf4, + 0x50, 0xb6, 0x26, 0xb8, 0x6e, 0xed, 0x96, 0xaf, 0x56, 0xff, 0x65, 0x1a, 0x51, 0x2e, 0x68, 0xde, + 0x6e, 0x54, 0x55, 0x05, 0x8a, 0x83, 0x6c, 0xb8, 0xf3, 0x29, 0xcc, 0xd3, 0x7e, 0x1a, 0x71, 0xcb, + 0x74, 0xcd, 0xe6, 0x6e, 0x30, 0xb3, 0x5b, 0x25, 0x80, 0xdb, 0x5a, 0x48, 0xf4, 0x14, 0x36, 0x3a, + 0x05, 0xef, 0xa1, 0xc3, 0x05, 0xf5, 0x0b, 0xde, 0xd3, 0x43, 0xb6, 0xef, 0xac, 0xba, 0x55, 0x17, + 0x9e, 0x81, 0x5e, 0xc1, 0x4d, 0xd9, 0x1f, 0xc2, 0x73, 0xca, 0xdf, 0xbe, 0x3d, 0xdb, 0x59, 0x8b, + 0xd7, 0xb9, 0x1e, 0x01, 0xd4, 0x81, 0x7b, 0x0b, 0xd3, 0x42, 0xf7, 0x57, 0x62, 0x96, 0xa5, 0xb7, + 0xf1, 0x3a, 0x78, 0x9e, 0xb1, 0xfd, 0x71, 0x34, 0xc1, 0xc6, 0xe5, 0x04, 0x1b, 0xd7, 0x13, 0x0c, + 0xbe, 0x94, 0x18, 0x7c, 0x2b, 0x31, 0x18, 0x96, 0x18, 0x8c, 0x4a, 0x0c, 0x7e, 0x95, 0x18, 0xfc, + 0x2e, 0xb1, 0x71, 0x5d, 0x62, 0x70, 0x31, 0xc5, 0xc6, 0x68, 0x8a, 0x8d, 0xcb, 0x29, 0x36, 0xde, + 0x2f, 0xae, 0x6d, 0x94, 0x87, 0xa7, 0x61, 0x1a, 0x92, 0x98, 0x9d, 0xf5, 0xc9, 0xf9, 0x11, 0x59, + 0xfc, 0x61, 0x75, 0xb7, 0xe4, 0xe3, 0xe8, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x3e, 0x65, 0x57, + 0x8c, 0x24, 0x05, 0x00, 0x00, } func (this *QueryPatternsRequest) Equal(that interface{}) bool { @@ -408,13 +494,7 @@ func (this *PatternSeries) Equal(that interface{}) bool { } else if this == nil { return false } - if that1.Identifier == nil { - if this.Identifier != nil { - return false - } - } else if this.Identifier == nil { - return false - } else if !this.Identifier.Equal(that1.Identifier) { + if this.Pattern != that1.Pattern { return false } if len(this.Samples) != len(that1.Samples) { @@ -427,14 +507,14 @@ func (this *PatternSeries) Equal(that interface{}) bool { } return true } -func (this *PatternSeries_Pattern) Equal(that interface{}) bool { +func (this *PatternSample) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*PatternSeries_Pattern) + that1, ok := that.(*PatternSample) if !ok { - that2, ok := that.(PatternSeries_Pattern) + that2, ok := that.(PatternSample) if ok { that1 = &that2 } else { @@ -446,19 +526,22 @@ func (this *PatternSeries_Pattern) Equal(that interface{}) bool { } else if this == nil { return false } - if this.Pattern != that1.Pattern { + if !this.Timestamp.Equal(that1.Timestamp) { + return false + } + if this.Value != that1.Value { return false } return true } -func (this *PatternSeries_Labels) Equal(that interface{}) bool { +func (this *QuerySamplesRequest) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*PatternSeries_Labels) + that1, ok := that.(*QuerySamplesRequest) if !ok { - that2, ok := that.(PatternSeries_Labels) + that2, ok := that.(QuerySamplesRequest) if ok { that1 = &that2 } else { @@ -470,19 +553,28 @@ func (this *PatternSeries_Labels) Equal(that interface{}) bool { } else if this == nil { return false } - if this.Labels != that1.Labels { + if this.Query != that1.Query { + return false + } + if !this.Start.Equal(that1.Start) { + return false + } + if !this.End.Equal(that1.End) { + return false + } + if this.Step != that1.Step { return false } return true } -func (this *PatternSample) Equal(that interface{}) bool { +func (this *QuerySamplesResponse) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*PatternSample) + that1, ok := that.(*QuerySamplesResponse) if !ok { - that2, ok := that.(PatternSample) + that2, ok := that.(QuerySamplesResponse) if ok { that1 = &that2 } else { @@ -494,12 +586,25 @@ func (this *PatternSample) Equal(that interface{}) bool { } else if this == nil { return false } - if !this.Timestamp.Equal(that1.Timestamp) { + if len(this.Series) != len(that1.Series) { return false } - if this.Value != that1.Value { + for i := range this.Series { + if !this.Series[i].Equal(that1.Series[i]) { + return false + } + } + if !this.Stats.Equal(&that1.Stats) { + 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 + } + } return true } func (this *QueryPatternsRequest) GoString() string { @@ -531,41 +636,48 @@ func (this *PatternSeries) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 7) + s := make([]string, 0, 6) s = append(s, "&logproto.PatternSeries{") - if this.Identifier != nil { - s = append(s, "Identifier: "+fmt.Sprintf("%#v", this.Identifier)+",\n") - } + s = append(s, "Pattern: "+fmt.Sprintf("%#v", this.Pattern)+",\n") if this.Samples != nil { s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") } s = append(s, "}") return strings.Join(s, "") } -func (this *PatternSeries_Pattern) GoString() string { +func (this *PatternSample) GoString() string { if this == nil { return "nil" } - s := strings.Join([]string{`&logproto.PatternSeries_Pattern{` + - `Pattern:` + fmt.Sprintf("%#v", this.Pattern) + `}`}, ", ") - return s + s := make([]string, 0, 6) + s = append(s, "&logproto.PatternSample{") + s = append(s, "Timestamp: "+fmt.Sprintf("%#v", this.Timestamp)+",\n") + s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s = append(s, "}") + return strings.Join(s, "") } -func (this *PatternSeries_Labels) GoString() string { +func (this *QuerySamplesRequest) GoString() string { if this == nil { return "nil" } - s := strings.Join([]string{`&logproto.PatternSeries_Labels{` + - `Labels:` + fmt.Sprintf("%#v", this.Labels) + `}`}, ", ") - return s + s := make([]string, 0, 8) + s = append(s, "&logproto.QuerySamplesRequest{") + s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") + s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") + s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") + s = append(s, "Step: "+fmt.Sprintf("%#v", this.Step)+",\n") + s = append(s, "}") + return strings.Join(s, "") } -func (this *PatternSample) GoString() string { +func (this *QuerySamplesResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) - s = append(s, "&logproto.PatternSample{") - s = append(s, "Timestamp: "+fmt.Sprintf("%#v", this.Timestamp)+",\n") - s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s := make([]string, 0, 7) + s = append(s, "&logproto.QuerySamplesResponse{") + s = append(s, "Series: "+fmt.Sprintf("%#v", this.Series)+",\n") + s = append(s, "Stats: "+strings.Replace(this.Stats.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -592,6 +704,7 @@ const _ = grpc.SupportPackageIsVersion4 type PatternClient interface { Push(ctx context.Context, in *push.PushRequest, opts ...grpc.CallOption) (*push.PushResponse, error) Query(ctx context.Context, in *QueryPatternsRequest, opts ...grpc.CallOption) (Pattern_QueryClient, error) + QuerySample(ctx context.Context, in *QuerySamplesRequest, opts ...grpc.CallOption) (Pattern_QuerySampleClient, error) } type patternClient struct { @@ -643,10 +756,43 @@ func (x *patternQueryClient) Recv() (*QueryPatternsResponse, error) { return m, nil } +func (c *patternClient) QuerySample(ctx context.Context, in *QuerySamplesRequest, opts ...grpc.CallOption) (Pattern_QuerySampleClient, error) { + stream, err := c.cc.NewStream(ctx, &_Pattern_serviceDesc.Streams[1], "/logproto.Pattern/QuerySample", opts...) + if err != nil { + return nil, err + } + x := &patternQuerySampleClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Pattern_QuerySampleClient interface { + Recv() (*QuerySamplesResponse, error) + grpc.ClientStream +} + +type patternQuerySampleClient struct { + grpc.ClientStream +} + +func (x *patternQuerySampleClient) Recv() (*QuerySamplesResponse, error) { + m := new(QuerySamplesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // PatternServer is the server API for Pattern service. type PatternServer interface { Push(context.Context, *push.PushRequest) (*push.PushResponse, error) Query(*QueryPatternsRequest, Pattern_QueryServer) error + QuerySample(*QuerySamplesRequest, Pattern_QuerySampleServer) error } // UnimplementedPatternServer can be embedded to have forward compatible implementations. @@ -659,6 +805,9 @@ func (*UnimplementedPatternServer) Push(ctx context.Context, req *push.PushReque func (*UnimplementedPatternServer) Query(req *QueryPatternsRequest, srv Pattern_QueryServer) error { return status.Errorf(codes.Unimplemented, "method Query not implemented") } +func (*UnimplementedPatternServer) QuerySample(req *QuerySamplesRequest, srv Pattern_QuerySampleServer) error { + return status.Errorf(codes.Unimplemented, "method QuerySample not implemented") +} func RegisterPatternServer(s *grpc.Server, srv PatternServer) { s.RegisterService(&_Pattern_serviceDesc, srv) @@ -703,6 +852,27 @@ func (x *patternQueryServer) Send(m *QueryPatternsResponse) error { return x.ServerStream.SendMsg(m) } +func _Pattern_QuerySample_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(QuerySamplesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(PatternServer).QuerySample(m, &patternQuerySampleServer{stream}) +} + +type Pattern_QuerySampleServer interface { + Send(*QuerySamplesResponse) error + grpc.ServerStream +} + +type patternQuerySampleServer struct { + grpc.ServerStream +} + +func (x *patternQuerySampleServer) Send(m *QuerySamplesResponse) error { + return x.ServerStream.SendMsg(m) +} + var _Pattern_serviceDesc = grpc.ServiceDesc{ ServiceName: "logproto.Pattern", HandlerType: (*PatternServer)(nil), @@ -718,6 +888,11 @@ var _Pattern_serviceDesc = grpc.ServiceDesc{ Handler: _Pattern_Query_Handler, ServerStreams: true, }, + { + StreamName: "QuerySample", + Handler: _Pattern_QuerySample_Handler, + ServerStreams: true, + }, }, Metadata: "pkg/logproto/pattern.proto", } @@ -830,15 +1005,6 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Identifier != nil { - { - size := m.Identifier.Size() - i -= size - if _, err := m.Identifier.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } - } if len(m.Samples) > 0 { for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { { @@ -853,35 +1019,16 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x12 } } + if len(m.Pattern) > 0 { + i -= len(m.Pattern) + copy(dAtA[i:], m.Pattern) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) + i-- + dAtA[i] = 0xa + } return len(dAtA) - i, nil } -func (m *PatternSeries_Pattern) MarshalTo(dAtA []byte) (int, error) { - return m.MarshalToSizedBuffer(dAtA[:m.Size()]) -} - -func (m *PatternSeries_Pattern) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - i -= len(m.Pattern) - copy(dAtA[i:], m.Pattern) - i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) - i-- - dAtA[i] = 0xa - return len(dAtA) - i, nil -} -func (m *PatternSeries_Labels) MarshalTo(dAtA []byte) (int, error) { - return m.MarshalToSizedBuffer(dAtA[:m.Size()]) -} - -func (m *PatternSeries_Labels) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - i -= len(m.Labels) - copy(dAtA[i:], m.Labels) - i = encodeVarintPattern(dAtA, i, uint64(len(m.Labels))) - i-- - dAtA[i] = 0x1a - return len(dAtA) - i, nil -} func (m *PatternSample) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -915,26 +1062,133 @@ func (m *PatternSample) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func encodeVarintPattern(dAtA []byte, offset int, v uint64) int { - offset -= sovPattern(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *QuerySamplesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return base + return dAtA[:n], nil } -func (m *QueryPatternsRequest) Size() (n int) { - if m == nil { - return 0 - } + +func (m *QuerySamplesRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QuerySamplesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Query) - if l > 0 { - n += 1 + l + sovPattern(uint64(l)) + if m.Step != 0 { + i = encodeVarintPattern(dAtA, i, uint64(m.Step)) + i-- + dAtA[i] = 0x20 + } + n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err3 != nil { + return 0, err3 + } + i -= n3 + i = encodeVarintPattern(dAtA, i, uint64(n3)) + i-- + dAtA[i] = 0x1a + n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err4 != nil { + return 0, err4 + } + i -= n4 + i = encodeVarintPattern(dAtA, i, uint64(n4)) + i-- + dAtA[i] = 0x12 + if len(m.Query) > 0 { + i -= len(m.Query) + copy(dAtA[i:], m.Query) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Query))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *QuerySamplesResponse) 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 *QuerySamplesResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QuerySamplesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + 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 = encodeVarintPattern(dAtA, i, uint64(len(m.Warnings[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + { + size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPattern(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + if len(m.Series) > 0 { + for iNdEx := len(m.Series) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Series[iNdEx].Size() + i -= size + if _, err := m.Series[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintPattern(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func encodeVarintPattern(dAtA []byte, offset int, v uint64) int { + offset -= sovPattern(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *QueryPatternsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Query) + if l > 0 { + n += 1 + l + sovPattern(uint64(l)) } l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Start) n += 1 + l + sovPattern(uint64(l)) @@ -967,8 +1221,9 @@ func (m *PatternSeries) Size() (n int) { } var l int _ = l - if m.Identifier != nil { - n += m.Identifier.Size() + l = len(m.Pattern) + if l > 0 { + n += 1 + l + sovPattern(uint64(l)) } if len(m.Samples) > 0 { for _, e := range m.Samples { @@ -979,37 +1234,60 @@ func (m *PatternSeries) Size() (n int) { return n } -func (m *PatternSeries_Pattern) Size() (n int) { +func (m *PatternSample) Size() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Pattern) - n += 1 + l + sovPattern(uint64(l)) + if m.Timestamp != 0 { + n += 1 + sovPattern(uint64(m.Timestamp)) + } + if m.Value != 0 { + n += 1 + sovPattern(uint64(m.Value)) + } return n } -func (m *PatternSeries_Labels) Size() (n int) { + +func (m *QuerySamplesRequest) Size() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Labels) + l = len(m.Query) + if l > 0 { + n += 1 + l + sovPattern(uint64(l)) + } + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Start) n += 1 + l + sovPattern(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.End) + n += 1 + l + sovPattern(uint64(l)) + if m.Step != 0 { + n += 1 + sovPattern(uint64(m.Step)) + } return n } -func (m *PatternSample) Size() (n int) { + +func (m *QuerySamplesResponse) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.Timestamp != 0 { - n += 1 + sovPattern(uint64(m.Timestamp)) + if len(m.Series) > 0 { + for _, e := range m.Series { + l = e.Size() + n += 1 + l + sovPattern(uint64(l)) + } } - if m.Value != 0 { - n += 1 + sovPattern(uint64(m.Value)) + l = m.Stats.Size() + n += 1 + l + sovPattern(uint64(l)) + if len(m.Warnings) > 0 { + for _, s := range m.Warnings { + l = len(s) + n += 1 + l + sovPattern(uint64(l)) + } } return n } @@ -1058,39 +1336,44 @@ func (this *PatternSeries) String() string { } repeatedStringForSamples += "}" s := strings.Join([]string{`&PatternSeries{`, - `Identifier:` + fmt.Sprintf("%v", this.Identifier) + `,`, + `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, `Samples:` + repeatedStringForSamples + `,`, `}`, }, "") return s } -func (this *PatternSeries_Pattern) String() string { +func (this *PatternSample) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&PatternSeries_Pattern{`, - `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, + s := strings.Join([]string{`&PatternSample{`, + `Timestamp:` + fmt.Sprintf("%v", this.Timestamp) + `,`, + `Value:` + fmt.Sprintf("%v", this.Value) + `,`, `}`, }, "") return s } -func (this *PatternSeries_Labels) String() string { +func (this *QuerySamplesRequest) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&PatternSeries_Labels{`, - `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + s := strings.Join([]string{`&QuerySamplesRequest{`, + `Query:` + fmt.Sprintf("%v", this.Query) + `,`, + `Start:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Start), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, + `End:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.End), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, + `Step:` + fmt.Sprintf("%v", this.Step) + `,`, `}`, }, "") return s } -func (this *PatternSample) String() string { +func (this *QuerySamplesResponse) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&PatternSample{`, - `Timestamp:` + fmt.Sprintf("%v", this.Timestamp) + `,`, - `Value:` + fmt.Sprintf("%v", this.Value) + `,`, + s := strings.Join([]string{`&QuerySamplesResponse{`, + `Series:` + fmt.Sprintf("%v", this.Series) + `,`, + `Stats:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Stats), "Ingester", "stats.Ingester", 1), `&`, ``, 1) + `,`, + `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, `}`, }, "") return s @@ -1419,7 +1702,7 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Identifier = &PatternSeries_Pattern{string(dAtA[iNdEx:postIndex])} + m.Pattern = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { @@ -1455,9 +1738,153 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: + default: + iNdEx = preIndex + skippy, err := skipPattern(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPattern + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthPattern + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PatternSample) 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 ErrIntOverflowPattern + } + 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: PatternSample: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PatternSample: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + m.Value = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Value |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipPattern(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPattern + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthPattern + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *QuerySamplesRequest) 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 ErrIntOverflowPattern + } + 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: QuerySamplesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QuerySamplesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -1485,8 +1912,93 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Identifier = &PatternSeries_Labels{string(dAtA[iNdEx:postIndex])} + m.Query = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPattern + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Start, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPattern + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.End, dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Step", wireType) + } + m.Step = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Step |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPattern(dAtA[iNdEx:]) @@ -1511,7 +2023,7 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { } return nil } -func (m *PatternSample) Unmarshal(dAtA []byte) error { +func (m *QuerySamplesResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1534,17 +2046,17 @@ func (m *PatternSample) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PatternSample: wiretype end group for non-group") + return fmt.Errorf("proto: QuerySamplesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PatternSample: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QuerySamplesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Series", wireType) } - m.Timestamp = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPattern @@ -1554,16 +2066,31 @@ func (m *PatternSample) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Timestamp |= github_com_prometheus_common_model.Time(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthPattern + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Series = append(m.Series, Series{}) + if err := m.Series[len(m.Series)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) } - m.Value = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPattern @@ -1573,11 +2100,57 @@ func (m *PatternSample) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Value |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPattern + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Stats.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 ErrIntOverflowPattern + } + 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 ErrInvalidLengthPattern + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPattern(dAtA[iNdEx:]) diff --git a/pkg/logproto/pattern.proto b/pkg/logproto/pattern.proto index fa03742b1af7..419cfa8194a1 100644 --- a/pkg/logproto/pattern.proto +++ b/pkg/logproto/pattern.proto @@ -6,12 +6,14 @@ import "gogoproto/gogo.proto"; import "google/protobuf/timestamp.proto"; import "pkg/logproto/logproto.proto"; import "pkg/push/push.proto"; +import "pkg/logqlmodel/stats/stats.proto"; option go_package = "github.com/grafana/loki/v3/pkg/logproto"; service Pattern { rpc Push(PushRequest) returns (PushResponse) {} rpc Query(QueryPatternsRequest) returns (stream QueryPatternsResponse) {} + rpc QuerySample(QuerySamplesRequest) returns (stream QuerySamplesResponse) {} } message QueryPatternsRequest { @@ -32,10 +34,7 @@ message QueryPatternsResponse { } message PatternSeries { - oneof identifier { - string pattern = 1; - string labels = 3; - } + string pattern = 1; repeated PatternSample samples = 2; } @@ -46,3 +45,26 @@ message PatternSample { ]; int64 value = 2; } + +message QuerySamplesRequest { + string query = 1; + google.protobuf.Timestamp start = 2 [ + (gogoproto.stdtime) = true, + (gogoproto.nullable) = false + ]; + google.protobuf.Timestamp end = 3 [ + (gogoproto.stdtime) = true, + (gogoproto.nullable) = false + ]; + int64 step = 4; +} + + +message QuerySamplesResponse { + repeated Series series = 1 [ + (gogoproto.customtype) = "Series", + (gogoproto.nullable) = true + ]; + stats.Ingester stats = 2 [(gogoproto.nullable) = false]; + repeated string warnings = 3; +} diff --git a/pkg/logql/range_vector.go b/pkg/logql/range_vector.go index 141d4865c46c..180a1bde27ca 100644 --- a/pkg/logql/range_vector.go +++ b/pkg/logql/range_vector.go @@ -193,6 +193,7 @@ func (r *batchRangeVectorIterator) load(start, end int64) { series.Metric = metric r.window[lbs] = series } + // TODO(twhitney): Everywhere else, an FPoint.T is in milliseconds, but here it's in nanoseconds. p := promql.FPoint{ T: sample.Timestamp, F: sample.Value, diff --git a/pkg/pattern/drain/chunk.go b/pkg/pattern/drain/chunk.go index 8be0dd64070f..91983eb0ff9a 100644 --- a/pkg/pattern/drain/chunk.go +++ b/pkg/pattern/drain/chunk.go @@ -40,7 +40,7 @@ func (c Chunk) spaceFor(ts model.Time) bool { } // ForRange returns samples with only the values -// in the given range [start:end] and aggregates them by step duration. +// in the given range [start:end) and aggregates them by step duration. // start and end are in milliseconds since epoch. step is a duration in milliseconds. func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { if len(c.Samples) == 0 { @@ -48,7 +48,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { } first := c.Samples[0].Timestamp last := c.Samples[len(c.Samples)-1].Timestamp - if start >= end || first > end || last < start { + if start >= end || first >= end || last < start { return nil } var lo int @@ -58,10 +58,9 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { }) } hi := len(c.Samples) - - if end <= last { + if end < last { hi = sort.Search(len(c.Samples), func(i int) bool { - return c.Samples[i].Timestamp > end + return c.Samples[i].Timestamp >= end }) } if step == chunk.TimeResolution { @@ -121,9 +120,9 @@ func (c Chunks) Iterator(pattern string, from, through, step model.Time) iter.It if len(samples) == 0 { continue } - iters = append(iters, iter.NewPatternSlice(pattern, samples)) + iters = append(iters, iter.NewSlice(pattern, samples)) } - return iter.NewNonOverlappingPatternIterator(pattern, iters) + return iter.NewNonOverlappingIterator(pattern, iters) } func (c Chunks) samples() []*logproto.PatternSample { diff --git a/pkg/pattern/drain/chunk_test.go b/pkg/pattern/drain/chunk_test.go index e404a9b5da77..aa39fb7e8f59 100644 --- a/pkg/pattern/drain/chunk_test.go +++ b/pkg/pattern/drain/chunk_test.go @@ -138,7 +138,7 @@ func TestForRange(t *testing.T) { }, }, { - name: "Start before First and End Inclusive of First Element", + name: "Start and End Before First Element", c: &Chunk{Samples: []logproto.PatternSample{ {Timestamp: 2, Value: 2}, {Timestamp: 4, Value: 4}, @@ -146,17 +146,6 @@ func TestForRange(t *testing.T) { }}, start: 0, end: 2, - expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, - }, - { - name: "Start and End before First Element", - c: &Chunk{Samples: []logproto.PatternSample{ - {Timestamp: 2, Value: 2}, - {Timestamp: 4, Value: 4}, - {Timestamp: 6, Value: 6}, - }}, - start: 0, - end: 1, expected: nil, }, { diff --git a/pkg/pattern/flush_test.go b/pkg/pattern/flush_test.go index d2ed2384522d..9ee4bd436992 100644 --- a/pkg/pattern/flush_test.go +++ b/pkg/pattern/flush_test.go @@ -62,7 +62,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err := iter.ReadAllWithPatterns(it) + res, err := iter.ReadAll(it) require.NoError(t, err) require.Equal(t, 2, len(res.Series)) ing.sweepUsers(true, true) @@ -72,7 +72,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err = iter.ReadAllWithPatterns(it) + res, err = iter.ReadAll(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) } diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 5d3da8ce1227..755ad668af6a 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -12,7 +12,6 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/tenant" @@ -24,10 +23,12 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/clientpool" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" util_log "github.com/grafana/loki/v3/pkg/util/log" + + loki_iter "github.com/grafana/loki/v3/pkg/iter" + pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" ) const readBatchSize = 1024 @@ -244,57 +245,64 @@ func (i *Ingester) Query(req *logproto.QueryPatternsRequest, stream logproto.Pat if err != nil { return err } + iterator, err := instance.Iterator(ctx, req) + if err != nil { + return err + } + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendPatternSample(ctx, iterator, stream) +} - expr, err := syntax.ParseExpr(req.Query) +func (i *Ingester) QuerySample( + req *logproto.QuerySamplesRequest, + stream logproto.Pattern_QuerySampleServer, +) error { + ctx := stream.Context() + instanceID, err := tenant.TenantID(ctx) + if err != nil { + return err + } + instance, err := i.GetOrCreateInstance(instanceID) + if err != nil { + return err + } - switch e := expr.(type) { - case syntax.SampleExpr: - var err error - iterator, err := instance.QuerySample(ctx, e, req) // this is returning a first value of 0,0 - if err != nil { - return err - } + expr, err := syntax.ParseSampleExpr(req.Query) + if err != nil { + return err + } - // TODO(twhitney): query store - // if start, end, ok := buildStoreRequest(i.cfg, req.Start, req.End, time.Now()); ok { - // storeReq := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ - // Start: start, - // End: end, - // Selector: req.Selector, - // Shards: req.Shards, - // Deletes: req.Deletes, - // Plan: req.Plan, - // }} - // storeItr, err := i.store.SelectSamples(ctx, storeReq) - // if err != nil { - // util.LogErrorWithContext(ctx, "closing iterator", it.Close) - // return err - // } - - // it = iter.NewMergeSampleIterator(ctx, []iter.SampleIterator{it, storeItr}) - // } - - defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) - return sendMetricsSample(ctx, iterator, stream) - case syntax.LogSelectorExpr: - var err error - iterator, err := instance.Iterator(ctx, req) - if err != nil { - return err - } - defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) - return sendPatternSample(ctx, iterator, stream) - default: - return httpgrpc.Errorf( - http.StatusBadRequest, - fmt.Sprintf("unexpected type (%T): cannot evaluate", e), - ) + iterator, err := instance.QuerySample(ctx, expr, req) // this is returning a first value of 0,0 + if err != nil { + return err } + + // TODO(twhitney): query store + // if start, end, ok := buildStoreRequest(i.cfg, req.Start, req.End, time.Now()); ok { + // storeReq := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ + // Start: start, + // End: end, + // Selector: req.Selector, + // Shards: req.Shards, + // Deletes: req.Deletes, + // Plan: req.Plan, + // }} + // storeItr, err := i.store.SelectSamples(ctx, storeReq) + // if err != nil { + // util.LogErrorWithContext(ctx, "closing iterator", it.Close) + // return err + // } + + // it = iter.NewMergeSampleIterator(ctx, []iter.SampleIterator{it, storeItr}) + // } + + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendMetricSamples(ctx, iterator, stream) } -func sendPatternSample(ctx context.Context, it iter.Iterator, stream logproto.Pattern_QueryServer) error { +func sendPatternSample(ctx context.Context, it pattern_iter.Iterator, stream logproto.Pattern_QueryServer) error { for ctx.Err() == nil { - batch, err := iter.ReadPatternsBatch(it, readBatchSize) + batch, err := pattern_iter.ReadBatch(it, readBatchSize) if err != nil { return err } @@ -308,13 +316,13 @@ func sendPatternSample(ctx context.Context, it iter.Iterator, stream logproto.Pa return nil } -func sendMetricsSample( +func sendMetricSamples( ctx context.Context, - it iter.Iterator, - stream logproto.Pattern_QueryServer, + it loki_iter.SampleIterator, + stream logproto.Pattern_QuerySampleServer, ) error { for ctx.Err() == nil { - batch, err := iter.ReadMetricsBatch(it, readBatchSize) + batch, err := pattern_iter.ReadMetricsBatch(it, readBatchSize) if err != nil { return err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index fc62c7cfdca4..60eacb657d5a 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -4,21 +4,25 @@ import ( "context" "errors" "math" + "net/http" "github.com/go-kit/log" + "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/prometheus/client_golang/prometheus" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/drain" - "github.com/grafana/loki/v3/pkg/pattern/iter" + + loki_iter "github.com/grafana/loki/v3/pkg/iter" + pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" ) // TODO(kolesnikovae): parametrise QueryPatternsRequest const minClusterSize = 30 -var ErrParseQuery = errors.New("only label matcher, byte_over_time, and count_over_time queries without filters are supported") +var ErrParseQuery = errors.New("only byte_over_time and count_over_time queries without filters are supported") type IngesterQuerier struct { cfg Config @@ -47,32 +51,30 @@ func NewIngesterQuerier( func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { _, err := syntax.ParseMatchers(req.Query, true) if err != nil { - // not a pattern query, so either a metric query or an error - if q.cfg.MetricAggregation.Enabled { - return q.queryMetricSamples(ctx, req) - } - - return nil, err + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - - return q.queryPatternSamples(ctx, req) -} - -func (q *IngesterQuerier) queryPatternSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { - iterators, err := q.query(ctx, req) + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { + return client.Query(ctx, req) + }) if err != nil { return nil, err } - + iterators := make([]pattern_iter.Iterator, len(resps)) + for i := range resps { + iterators[i] = pattern_iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) + } // TODO(kolesnikovae): Incorporate with pruning - resp, err := iter.ReadPatternsBatch(iter.NewMerge(iterators...), math.MaxInt32) + resp, err := pattern_iter.ReadBatch(pattern_iter.NewMerge(iterators...), math.MaxInt32) if err != nil { return nil, err } return prunePatterns(resp, minClusterSize), nil } -func (q *IngesterQuerier) queryMetricSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { +func (q *IngesterQuerier) Samples( + ctx context.Context, + req *logproto.QuerySamplesRequest, +) (*logproto.QuerySamplesResponse, error) { expr, err := syntax.ParseSampleExpr(req.Query) if err != nil { return nil, err @@ -96,28 +98,43 @@ func (q *IngesterQuerier) queryMetricSamples(ctx context.Context, req *logproto. return nil, ErrParseQuery } - iterators, err := q.query(ctx, req) + iterators, err := q.querySample(ctx, req) if err != nil { return nil, err } - resp, err := iter.ReadMetricsBatch(iter.NewMerge(iterators...), math.MaxInt32) + // TODO: what should batch size be here? + resp, err := pattern_iter.ReadMetricsBatch(loki_iter.NewSortSampleIterator(iterators), math.MaxInt32) if err != nil { return nil, err } return resp, nil } -func (q *IngesterQuerier) query(ctx context.Context, req *logproto.QueryPatternsRequest) ([]iter.Iterator, error) { +func (q *IngesterQuerier) queryPattern(ctx context.Context, req *logproto.QueryPatternsRequest) ([]pattern_iter.Iterator, error) { resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { return client.Query(ctx, req) }) if err != nil { return nil, err } - iterators := make([]iter.Iterator, len(resps)) + iterators := make([]pattern_iter.Iterator, len(resps)) + for i := range resps { + iterators[i] = pattern_iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) + } + return iterators, nil +} + +func (q *IngesterQuerier) querySample(ctx context.Context, req *logproto.QuerySamplesRequest) ([]loki_iter.SampleIterator, error) { + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { + return client.QuerySample(ctx, req) + }) + if err != nil { + return nil, err + } + iterators := make([]loki_iter.SampleIterator, len(resps)) for i := range resps { - iterators[i] = iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) + iterators[i] = pattern_iter.NewQuerySamplesClientIterator(resps[i].response.(logproto.Pattern_QuerySampleClient)) } return iterators, nil } @@ -141,7 +158,7 @@ func prunePatterns( continue } resp.Series = append(resp.Series, - logproto.NewPatternSeriesWithPattern(pattern, cluster.Samples())) + logproto.NewPatternSeries(pattern, cluster.Samples())) } return resp } diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index 20b3dcae83d7..706265855888 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -25,7 +25,7 @@ func Test_prunePatterns(t *testing.T) { resp := new(logproto.QueryPatternsResponse) scanner := bufio.NewScanner(file) for scanner.Scan() { - resp.Series = append(resp.Series, logproto.NewPatternSeriesWithPattern(scanner.Text(), []*logproto.PatternSample{})) + resp.Series = append(resp.Series, logproto.NewPatternSeries(scanner.Text(), []*logproto.PatternSample{})) } require.NoError(t, scanner.Err()) prunePatterns(resp, 0) @@ -47,7 +47,7 @@ func Test_prunePatterns(t *testing.T) { require.Equal(t, expectedPatterns, patterns) } -func Test_Patterns(t *testing.T) { +func Test_Samples(t *testing.T) { t.Run("it rejects metric queries with filters", func(t *testing.T) { q := &IngesterQuerier{ cfg: Config{ @@ -69,19 +69,18 @@ func Test_Patterns(t *testing.T) { `sum by (label)(count_over_time({foo="bar"} |= "baz" [5m]))`, `bytes_over_time({foo="bar"} |= "baz" [5m])`, } { - _, err := q.Patterns( + _, err := q.Samples( context.Background(), - &logproto.QueryPatternsRequest{ + &logproto.QuerySamplesRequest{ Query: query, }, ) require.Error(t, err, query) require.ErrorIs(t, err, ErrParseQuery, query) - } }) - t.Run("accepts log selector queries and count and bytes metric queries", func(t *testing.T) { + t.Run("it rejects log selector queries", func(t *testing.T) { q := &IngesterQuerier{ cfg: Config{ MetricAggregation: metric.AggregationConfig{ @@ -94,6 +93,30 @@ func Test_Patterns(t *testing.T) { } for _, query := range []string{ `{foo="bar"}`, + } { + _, err := q.Samples( + context.Background(), + &logproto.QuerySamplesRequest{ + Query: query, + }, + ) + require.Error(t, err, query) + require.Equal(t, "only sample expression supported", err.Error(), query) + } + }) + + t.Run("accepts count and bytes metric queries", func(t *testing.T) { + q := &IngesterQuerier{ + cfg: Config{ + MetricAggregation: metric.AggregationConfig{ + Enabled: true, + }, + }, + logger: log.NewNopLogger(), + ringClient: &fakeRingClient{}, + registerer: nil, + } + for _, query := range []string{ `count_over_time({foo="bar"}[5m])`, `bytes_over_time({foo="bar"}[5m])`, `sum(count_over_time({foo="bar"}[5m]))`, @@ -101,9 +124,9 @@ func Test_Patterns(t *testing.T) { `sum by (level)(count_over_time({foo="bar"}[5m]))`, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`, } { - _, err := q.Patterns( + _, err := q.Samples( context.Background(), - &logproto.QueryPatternsRequest{ + &logproto.QuerySamplesRequest{ Query: query, }, ) diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index 9707837a4fab..3eb162114091 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -18,92 +18,81 @@ import ( "github.com/grafana/loki/pkg/push" ) -func TestInstancePushQuery(t *testing.T) { - lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - setup := func() *instance { - inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ - Enabled: true, - }) - require.NoError(t, err) +var lbls = labels.New(labels.Label{Name: "test", Value: "test"}) - return inst - } - t.Run("test pattern samples", func(t *testing.T) { - inst := setup() - err := inst.Push(context.Background(), &push.PushRequest{ - Streams: []push.Stream{ - { - Labels: lbs.String(), - Entries: []push.Entry{ - { - Timestamp: time.Unix(20, 0), - Line: "ts=1 msg=hello", - }, +func setup(t *testing.T) *instance { + inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ + Enabled: true, + }) + require.NoError(t, err) + + return inst +} + +func TestInstancePushQuery(t *testing.T) { + inst := setup(t) + err := inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbls.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(20, 0), + Line: "ts=1 msg=hello", }, }, }, - }) + }, + }) + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbls.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(30, 0), + Line: "ts=2 msg=hello", + }, + }, + }, + }, + }) + for i := 0; i <= 30; i++ { err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { - Labels: lbs.String(), + Labels: lbls.String(), Entries: []push.Entry{ { Timestamp: time.Unix(30, 0), - Line: "ts=2 msg=hello", + Line: "foo bar foo bar", }, }, }, }, }) - for i := 0; i <= 30; i++ { - err = inst.Push(context.Background(), &push.PushRequest{ - Streams: []push.Stream{ - { - Labels: lbs.String(), - Entries: []push.Entry{ - { - Timestamp: time.Unix(30, 0), - Line: "foo bar foo bar", - }, - }, - }, - }, - }) - require.NoError(t, err) - } - require.NoError(t, err) - it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ - Query: "{test=\"test\"}", - Start: time.Unix(0, 0), - End: time.Unix(0, math.MaxInt64), - }) - require.NoError(t, err) - res, err := iter.ReadAllWithPatterns(it) - require.NoError(t, err) - require.Equal(t, 2, len(res.Series)) - - it, err = inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ - Query: "{test=\"test\"}", - Start: time.Unix(0, 0), - End: time.Unix(30, 0), - }) require.NoError(t, err) - res, err = iter.ReadAllWithPatterns(it) - require.NoError(t, err) - // query should be inclusive of end time to match our - // existing metric query behavior - require.Equal(t, 2, len(res.Series)) - require.Equal(t, 2, len(res.Series[0].Samples)) + } + require.NoError(t, err) + it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ + Query: "{test=\"test\"}", + Start: time.Unix(0, 0), + End: time.Unix(0, math.MaxInt64), }) + require.NoError(t, err) + res, err := iter.ReadAll(it) + require.NoError(t, err) + require.Equal(t, 2, len(res.Series)) +} +func TestInstancePushQuerySamples(t *testing.T) { t.Run("test count_over_time samples", func(t *testing.T) { - inst := setup() + inst := setup(t) err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { - Labels: lbs.String(), + Labels: lbls.String(), Entries: []push.Entry{ { Timestamp: time.Unix(0, 0), @@ -117,7 +106,7 @@ func TestInstancePushQuery(t *testing.T) { err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { - Labels: lbs.String(), + Labels: lbls.String(), Entries: []push.Entry{ { Timestamp: time.Unix(int64(20*i), 0), @@ -134,65 +123,69 @@ func TestInstancePushQuery(t *testing.T) { expr, err := syntax.ParseSampleExpr(`count_over_time({test="test"}[20s])`) require.NoError(t, err) - it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + it, err := inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ Query: expr.String(), Start: time.Unix(0, 0), End: time.Unix(int64(20*30), 0), Step: 10000, }) require.NoError(t, err) - res, err := iter.ReadAllWithLabels(it) + res, err := iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) - require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) // end - start / step -- (start is 0, step is 10s) - // plus one because end is actually inclusive for metric queries - expectedDataPoints := ((20 * 30) / 10) + 1 + expectedDataPoints := ((20 * 30) / 10) require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) - require.Equal(t, int64(1), res.Series[0].Samples[0].Value) + require.Equal(t, float64(1), res.Series[0].Samples[0].Value) expr, err = syntax.ParseSampleExpr(`count_over_time({test="test"}[80s])`) require.NoError(t, err) - it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + // it, err := inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ + // Query: expr.String(), + // Start: time.Unix(0, 0), + // End: time.Unix(int64(20*30), 0), + // Step: 10000, + // }) + it, err = inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ Query: expr.String(), Start: time.Unix(0, 0), End: time.Unix(int64(20*30), 0), Step: 10000, }) require.NoError(t, err) - res, err = iter.ReadAllWithLabels(it) + res, err = iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) - require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) // end - start / step -- (start is 0, step is 10s) - // plus one because end is actually inclusive for metric queries - expectedDataPoints = ((20 * 30) / 10) + 1 + expectedDataPoints = ((20 * 30) / 10) require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) // with a larger selection range of 80s, we expect to eventually get up to 4 per datapoint // our pushes are spaced 20s apart, and there's 10s step, so we ecpect to see the value increase // every 2 samples, maxing out and staying at 4 after 6 samples (since it starts a 1, not 0) - require.Equal(t, int64(1), res.Series[0].Samples[0].Value) - require.Equal(t, int64(1), res.Series[0].Samples[1].Value) - require.Equal(t, int64(2), res.Series[0].Samples[2].Value) - require.Equal(t, int64(2), res.Series[0].Samples[3].Value) - require.Equal(t, int64(3), res.Series[0].Samples[4].Value) - require.Equal(t, int64(3), res.Series[0].Samples[5].Value) - require.Equal(t, int64(4), res.Series[0].Samples[6].Value) - require.Equal(t, int64(4), res.Series[0].Samples[expectedDataPoints-1].Value) + require.Equal(t, float64(1), res.Series[0].Samples[0].Value) + require.Equal(t, float64(1), res.Series[0].Samples[1].Value) + require.Equal(t, float64(2), res.Series[0].Samples[2].Value) + require.Equal(t, float64(2), res.Series[0].Samples[3].Value) + require.Equal(t, float64(3), res.Series[0].Samples[4].Value) + require.Equal(t, float64(3), res.Series[0].Samples[5].Value) + require.Equal(t, float64(4), res.Series[0].Samples[6].Value) + require.Equal(t, float64(4), res.Series[0].Samples[expectedDataPoints-1].Value) }) t.Run("test bytes_over_time samples", func(t *testing.T) { - inst := setup() + inst := setup(t) err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { - Labels: lbs.String(), + Labels: lbls.String(), Entries: []push.Entry{ { Timestamp: time.Unix(0, 0), @@ -206,7 +199,7 @@ func TestInstancePushQuery(t *testing.T) { err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { - Labels: lbs.String(), + Labels: lbls.String(), Entries: []push.Entry{ { Timestamp: time.Unix(int64(20*i), 0), @@ -223,57 +216,55 @@ func TestInstancePushQuery(t *testing.T) { expr, err := syntax.ParseSampleExpr(`bytes_over_time({test="test"}[20s])`) require.NoError(t, err) - it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + it, err := inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ Query: expr.String(), Start: time.Unix(0, 0), End: time.Unix(int64(20*30), 0), Step: 10000, }) require.NoError(t, err) - res, err := iter.ReadAllWithLabels(it) + res, err := iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) - require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) // end - start / step -- (start is 0, step is 10s) - // plus one because end is actually inclusive for metric queries - expectedDataPoints := ((20 * 30) / 10) + 1 + expectedDataPoints := ((20 * 30) / 10) require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) - require.Equal(t, int64(16), res.Series[0].Samples[0].Value) + require.Equal(t, float64(16), res.Series[0].Samples[0].Value) expr, err = syntax.ParseSampleExpr(`bytes_over_time({test="test"}[80s])`) require.NoError(t, err) - it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + it, err = inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ Query: expr.String(), Start: time.Unix(0, 0), End: time.Unix(int64(20*30), 0), Step: 10000, }) require.NoError(t, err) - res, err = iter.ReadAllWithLabels(it) + res, err = iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) - require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) // end - start / step -- (start is 0, step is 10s) - // plus one because end is actually inclusive for metric queries - expectedDataPoints = ((20 * 30) / 10) + 1 + expectedDataPoints = ((20 * 30) / 10) require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) // with a larger selection range of 80s, we expect to eventually get up to 64 bytes // as each pushe is 16 bytes and are spaced 20s apart. We query with 10s step, // so we ecpect to see the value increase by 16 bytes every 2 samples, // maxing out and staying at 64 after 6 samples (since it starts a 1, not 0) - require.Equal(t, int64(16), res.Series[0].Samples[0].Value) - require.Equal(t, int64(16), res.Series[0].Samples[1].Value) - require.Equal(t, int64(32), res.Series[0].Samples[2].Value) - require.Equal(t, int64(32), res.Series[0].Samples[3].Value) - require.Equal(t, int64(48), res.Series[0].Samples[4].Value) - require.Equal(t, int64(48), res.Series[0].Samples[5].Value) - require.Equal(t, int64(64), res.Series[0].Samples[6].Value) - require.Equal(t, int64(64), res.Series[0].Samples[expectedDataPoints-1].Value) + require.Equal(t, float64(16), res.Series[0].Samples[0].Value) + require.Equal(t, float64(16), res.Series[0].Samples[1].Value) + require.Equal(t, float64(32), res.Series[0].Samples[2].Value) + require.Equal(t, float64(32), res.Series[0].Samples[3].Value) + require.Equal(t, float64(48), res.Series[0].Samples[4].Value) + require.Equal(t, float64(48), res.Series[0].Samples[5].Value) + require.Equal(t, float64(64), res.Series[0].Samples[6].Value) + require.Equal(t, float64(64), res.Series[0].Samples[expectedDataPoints-1].Value) }) } diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index c5b724e0f6fc..a9309d88d819 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -16,9 +16,11 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/chunk" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" + + loki_iter "github.com/grafana/loki/v3/pkg/iter" + pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" ) const indexShards = 32 @@ -78,7 +80,7 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { } // Iterator returns an iterator of pattern samples matching the given query patterns request. -func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequest) (iter.Iterator, error) { +func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequest) (pattern_iter.Iterator, error) { matchers, err := syntax.ParseMatchers(req.Query, true) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) @@ -89,7 +91,7 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ step = chunk.TimeResolution } - var iters []iter.Iterator + var iters []pattern_iter.Iterator err = i.forMatchingStreams(matchers, func(s *stream) error { iter, err := s.Iterator(ctx, from, through, step) if err != nil { @@ -101,17 +103,17 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ if err != nil { return nil, err } - return iter.NewMerge(iters...), nil + return pattern_iter.NewMerge(iters...), nil } func (i *instance) QuerySample( ctx context.Context, expr syntax.SampleExpr, - req *logproto.QueryPatternsRequest, -) (iter.Iterator, error) { + req *logproto.QuerySamplesRequest, +) (loki_iter.SampleIterator, error) { if !i.aggregationCfg.Enabled { // Should never get here, but this will prevent nil pointer panics in test - return iter.Empty, nil + return loki_iter.NoopIterator, nil } from, through := util.RoundToMilliseconds(req.Start, req.End) @@ -125,11 +127,11 @@ func (i *instance) QuerySample( return nil, err } - var iters []iter.Iterator + var iters []loki_iter.SampleIterator err = i.forMatchingStreams( selector.Matchers(), func(stream *stream) error { - var iter iter.Iterator + var iter loki_iter.SampleIterator var err error iter, err = stream.SampleIterator(ctx, expr, from, through, step) @@ -145,7 +147,7 @@ func (i *instance) QuerySample( return nil, err } - return iter.NewMerge(iters...), nil + return loki_iter.NewSortSampleIterator(iters), nil } // forMatchingStreams will execute a function for each stream that matches the given matchers. diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 1a3699148920..67befa7ab5a8 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -17,26 +17,29 @@ import ( func TestInstance_QuerySample(t *testing.T) { ctx := context.Background() + thirtySeconds := int64(30000) oneMin := int64(60000) fiveMin := oneMin * 5 - now := int64(1715964275000) - then := now - fiveMin // 1715963975000 + now := fiveMin + then := int64(0) - mockReq := &logproto.QueryPatternsRequest{ - Start: time.Unix(then/1000, 0), - End: time.Now(), + mockReq := &logproto.QuerySamplesRequest{ + Start: time.UnixMilli(then), + End: time.UnixMilli(now + 1e4), Step: oneMin, } - instance, err := newInstance("test", log.NewNopLogger(), nil, metric.AggregationConfig{}) + instance, err := newInstance("test", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ + Enabled: true, + }) require.NoError(t, err) labels := model.LabelSet{ model.LabelName("foo"): model.LabelValue("bar"), } - lastTsMilli := (then + oneMin + oneMin) // 1715964095000 + lastTsMilli := (then + oneMin + thirtySeconds) // 0 + 60000 + 30000 = 90000 // TODO(twhitney): Add a few more pushes to this or another test instance.Push(ctx, &logproto.PushRequest{ @@ -45,19 +48,19 @@ func TestInstance_QuerySample(t *testing.T) { Labels: labels.String(), Entries: []push.Entry{ { - Timestamp: time.Unix(then/1000, 0), + Timestamp: time.UnixMilli(then), Line: "this=that color=blue", }, { - Timestamp: time.Unix((then+thirtySeconds)/1000, 0), + Timestamp: time.UnixMilli(then + thirtySeconds), Line: "this=that color=blue", }, { - Timestamp: time.Unix((then+oneMin)/1000, 0), + Timestamp: time.UnixMilli(then + oneMin), Line: "this=that color=blue", }, { - Timestamp: time.Unix(lastTsMilli/1000, 0), + Timestamp: time.UnixMilli(lastTsMilli), Line: "this=that color=blue", }, }, @@ -66,36 +69,44 @@ func TestInstance_QuerySample(t *testing.T) { }, }) + // 5 min query range + // 1 min step + // 1 min selection range + + // first: -60000 to 0 + // second: 0 to 60000 + // third: 60000 to 120000 + // fourth: 120000 to 180000 + // fifth: 180000 to 240000 + // sixth: 240000 to 300000 + + // lastTsMilli is 90000 + // would expect it in the 3rd bucket + start := then + secondPoint := start + oneMin + thirdPoint := secondPoint + oneMin + t.Run("successful count over time query", func(t *testing.T) { - expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[30s])`) + expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[60s])`) require.NoError(t, err) iter, err := instance.QuerySample(ctx, expr, mockReq) assert.NoError(t, err) assert.NotNil(t, iter) - // start is request start minus range, which is 30s here - start := then - 30000 - require.True(t, start < lastTsMilli-30000) - secondPoint := start + oneMin - require.True(t, secondPoint < lastTsMilli-30000) - // this is the first point past the lastTsMilli - thirdPoint := secondPoint + oneMin - require.Equal(t, lastTsMilli-30000, thirdPoint) - next := iter.Next() require.True(t, next) - sample := iter.At() - require.Equal(t, int64(4), sample.Value) - require.Equal(t, model.Time(thirdPoint), sample.Timestamp) + sample := iter.Sample() + require.Equal(t, float64(4), sample.Value) + require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) next = iter.Next() require.False(t, next) }) t.Run("successful bytes over time query", func(t *testing.T) { - expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[30s])`) + expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[60s])`) require.NoError(t, err) iter, err := instance.QuerySample(ctx, expr, mockReq) @@ -105,10 +116,9 @@ func TestInstance_QuerySample(t *testing.T) { next := iter.Next() require.True(t, next) - expctedTs := (then - 30000) + oneMin + oneMin - sample := iter.At() - require.Equal(t, int64(80), sample.Value) - require.Equal(t, model.Time(expctedTs), sample.Timestamp) + sample := iter.Sample() + require.Equal(t, float64(80), sample.Value) + require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) next = iter.Next() require.False(t, next) diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index 159edf1d73d2..d40004b667de 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -3,62 +3,67 @@ package iter import ( "math" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) -func ReadMetricsBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { +func ReadBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { var ( series = map[string][]*logproto.PatternSample{} respSize int ) for ; respSize < batchSize && it.Next(); respSize++ { - labels := it.Labels() + pattern := it.Pattern() sample := it.At() - series[labels.String()] = append(series[labels.String()], &sample) + series[pattern] = append(series[pattern], &sample) } result := logproto.QueryPatternsResponse{ Series: make([]*logproto.PatternSeries, 0, len(series)), } - for id, samples := range series { - result.Series = append( - result.Series, - logproto.NewPatternSeriesWithLabels(id, samples), - ) + for pattern, samples := range series { + result.Series = append(result.Series, &logproto.PatternSeries{ + Pattern: pattern, + Samples: samples, + }) } return &result, it.Error() } -func ReadPatternsBatch( - it Iterator, - batchSize int, -) (*logproto.QueryPatternsResponse, error) { +func ReadAll(it Iterator) (*logproto.QueryPatternsResponse, error) { + return ReadBatch(it, math.MaxInt32) +} + +func ReadMetricsBatch(it iter.SampleIterator, batchSize int) (*logproto.QuerySamplesResponse, error) { var ( - series = map[string][]*logproto.PatternSample{} + series = map[uint64]*logproto.Series{} respSize int ) for ; respSize < batchSize && it.Next(); respSize++ { - pattern := it.Pattern() - sample := it.At() - series[pattern] = append(series[pattern], &sample) + hash := it.StreamHash() + s, ok := series[hash] + if !ok { + s = &logproto.Series{ + Labels: it.Labels(), + Samples: []logproto.Sample{}, + StreamHash: hash, + } + series[hash] = s + } + + s.Samples = append(s.Samples, it.Sample()) } - result := logproto.QueryPatternsResponse{ - Series: make([]*logproto.PatternSeries, 0, len(series)), + + result := logproto.QuerySamplesResponse{ + Series: make([]logproto.Series, 0, len(series)), } - for id, samples := range series { - result.Series = append( - result.Series, - logproto.NewPatternSeriesWithPattern(id, samples), - ) + for _, s := range series { + result.Series = append(result.Series, *s) } return &result, it.Error() } -func ReadAllWithPatterns(it Iterator) (*logproto.QueryPatternsResponse, error) { - return ReadPatternsBatch(it, math.MaxInt32) -} - -func ReadAllWithLabels(it Iterator) (*logproto.QueryPatternsResponse, error) { +func ReadAllSamples(it iter.SampleIterator) (*logproto.QuerySamplesResponse, error) { return ReadMetricsBatch(it, math.MaxInt32) } diff --git a/pkg/pattern/iter/batch_test.go b/pkg/pattern/iter/batch_test.go index d798583d1a2a..3ee565892da4 100644 --- a/pkg/pattern/iter/batch_test.go +++ b/pkg/pattern/iter/batch_test.go @@ -32,7 +32,7 @@ func TestReadBatch(t *testing.T) { batchSize: 2, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithPattern( + logproto.NewPatternSeries( "foo", []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, @@ -49,7 +49,7 @@ func TestReadBatch(t *testing.T) { batchSize: 4, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithPattern( + logproto.NewPatternSeries( "foo", []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, @@ -64,8 +64,8 @@ func TestReadBatch(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - it := NewPatternSlice(tt.pattern, tt.samples) - got, err := ReadPatternsBatch(it, tt.batchSize) + it := NewSlice(tt.pattern, tt.samples) + got, err := ReadBatch(it, tt.batchSize) require.NoError(t, err) require.Equal(t, tt.expected, got) }) diff --git a/pkg/pattern/iter/iterator.go b/pkg/pattern/iter/iterator.go index 2f27104f0da0..5a277c0f2734 100644 --- a/pkg/pattern/iter/iterator.go +++ b/pkg/pattern/iter/iterator.go @@ -1,9 +1,7 @@ package iter import ( - "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" - "github.com/prometheus/prometheus/model/labels" ) var Empty Iterator = &emptyIterator{} @@ -12,75 +10,49 @@ type Iterator interface { Next() bool Pattern() string - Labels() labels.Labels At() logproto.PatternSample Error() error Close() error } -type SampleIterator interface { - Iterator - Sample() logproto.PatternSample -} - -type PeekingIterator interface { - SampleIterator - Peek() (string, logproto.PatternSample, bool) -} - -func NewPatternSlice(pattern string, s []logproto.PatternSample) Iterator { - return &patternSliceIterator{ +func NewSlice(pattern string, s []logproto.PatternSample) Iterator { + return &sliceIterator{ values: s, pattern: pattern, - labels: labels.EmptyLabels(), i: -1, } } -func NewLabelsSlice(lbls labels.Labels, s []logproto.PatternSample) Iterator { - return &patternSliceIterator{ - values: s, - labels: lbls, - i: -1, - } -} - -type patternSliceIterator struct { +type sliceIterator struct { i int pattern string - labels labels.Labels values []logproto.PatternSample } -func (s *patternSliceIterator) Next() bool { +func (s *sliceIterator) Next() bool { s.i++ return s.i < len(s.values) } -func (s *patternSliceIterator) Pattern() string { +func (s *sliceIterator) Pattern() string { return s.pattern } -func (s *patternSliceIterator) Labels() labels.Labels { - return s.labels -} - -func (s *patternSliceIterator) At() logproto.PatternSample { +func (s *sliceIterator) At() logproto.PatternSample { return s.values[s.i] } -func (s *patternSliceIterator) Error() error { +func (s *sliceIterator) Error() error { return nil } -func (s *patternSliceIterator) Close() error { +func (s *sliceIterator) Close() error { return nil } type emptyIterator struct { pattern string - labels labels.Labels } func (e *emptyIterator) Next() bool { @@ -91,10 +63,6 @@ func (e *emptyIterator) Pattern() string { return e.pattern } -func (e *emptyIterator) Labels() labels.Labels { - return e.labels -} - func (e *emptyIterator) At() logproto.PatternSample { return logproto.PatternSample{} } @@ -111,25 +79,16 @@ type nonOverlappingIterator struct { iterators []Iterator curr Iterator pattern string - labels labels.Labels } -// NewNonOverlappingPatternIterator gives a chained iterator over a list of iterators. -func NewNonOverlappingPatternIterator(pattern string, iterators []Iterator) Iterator { +// NewNonOverlappingIterator gives a chained iterator over a list of iterators. +func NewNonOverlappingIterator(pattern string, iterators []Iterator) Iterator { return &nonOverlappingIterator{ iterators: iterators, pattern: pattern, } } -// NewNonOverlappingLabelsIterator gives a chained iterator over a list of iterators. -func NewNonOverlappingLabelsIterator(labels labels.Labels, iterators []Iterator) Iterator { - return &nonOverlappingIterator{ - iterators: iterators, - labels: labels, - } -} - func (i *nonOverlappingIterator) Next() bool { for i.curr == nil || !i.curr.Next() { if len(i.iterators) == 0 { @@ -155,10 +114,6 @@ func (i *nonOverlappingIterator) Pattern() string { return i.pattern } -func (i *nonOverlappingIterator) Labels() labels.Labels { - return i.labels -} - func (i *nonOverlappingIterator) Error() error { if i.curr == nil { return nil @@ -176,114 +131,3 @@ func (i *nonOverlappingIterator) Close() error { i.iterators = nil return nil } - -type peekingIterator struct { - iter Iterator - - cache *sampleWithLabels - next *sampleWithLabels - labels labels.Labels -} - -type sampleWithLabels struct { - logproto.PatternSample - labels labels.Labels -} - -func (s *sampleWithLabels) Sample() logproto.Sample { - return logproto.Sample{ - Timestamp: s.PatternSample.Timestamp.UnixNano(), // logproto.Sample expects nano seconds - Value: float64(s.PatternSample.Value), - Hash: 0, - } -} - -func NewPeekingSampleIterator(iter Iterator) iter.PeekingSampleIterator { - // initialize the next entry so we can peek right from the start. - var cache *sampleWithLabels - next := &sampleWithLabels{} - if iter.Next() { - cache = &sampleWithLabels{ - PatternSample: iter.At(), - labels: iter.Labels(), - } - next.PatternSample = cache.PatternSample - next.labels = cache.labels - } - - return &peekingIterator{ - iter: iter, - cache: cache, - next: next, - labels: iter.Labels(), - } -} - -func (it *peekingIterator) Close() error { - return it.iter.Close() -} - -func (it *peekingIterator) Labels() string { - return it.labels.String() -} - -func (it *peekingIterator) Next() bool { - if it.cache != nil { - it.next.PatternSample = it.cache.PatternSample - it.next.labels = it.cache.labels - it.cacheNext() - return true - } - return false -} - -func (it *peekingIterator) Sample() logproto.Sample { - if it.next != nil { - return logproto.Sample{ - Timestamp: it.next.PatternSample.Timestamp.UnixNano(), // expecting nano seconds - Value: float64(it.next.PatternSample.Value), - Hash: 0, - } - } - return logproto.Sample{} -} - -func (it *peekingIterator) At() logproto.PatternSample { - if it.next != nil { - return it.next.PatternSample - } - return logproto.PatternSample{} -} - -// cacheNext caches the next element if it exists. -func (it *peekingIterator) cacheNext() { - if it.iter.Next() { - it.cache.PatternSample = it.iter.At() - it.cache.labels = it.iter.Labels() - return - } - // nothing left, remove the cached entry - it.cache = nil -} - -func (it *peekingIterator) Pattern() logproto.PatternSample { - if it.next != nil { - return it.next.PatternSample - } - return logproto.PatternSample{} -} - -func (it *peekingIterator) Peek() (string, logproto.Sample, bool) { - if it.cache != nil { - return it.cache.labels.String(), it.cache.Sample(), true - } - return "", logproto.Sample{}, false -} - -func (it *peekingIterator) Error() error { - return it.iter.Error() -} - -func (it *peekingIterator) StreamHash() uint64 { - return 0 -} diff --git a/pkg/pattern/iter/iterator_test.go b/pkg/pattern/iter/iterator_test.go index dd66361d635b..b327800575b5 100644 --- a/pkg/pattern/iter/iterator_test.go +++ b/pkg/pattern/iter/iterator_test.go @@ -3,111 +3,57 @@ package iter import ( "testing" - "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestSliceIterator(t *testing.T) { - t.Run("samples with pattern", func(t *testing.T) { - tests := []struct { - name string - pattern string - samples []logproto.PatternSample - want []patternSample - }{ - { - name: "1 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - }, - want: []patternSample{ - {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, - }, + tests := []struct { + name string + pattern string + samples []logproto.PatternSample + want []patternSample + }{ + { + name: "1 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, }, - { - name: "3 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - {Timestamp: 20, Value: 4}, - {Timestamp: 30, Value: 6}, - }, - want: []patternSample{ - {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, - }, + want: []patternSample{ + {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, }, - { - name: "empty", - pattern: "foo", - samples: nil, - want: nil, + }, + { + name: "3 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, }, - } - - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - got := slice(NewPatternSlice(tt.pattern, tt.samples)) - require.Equal(t, tt.want, got) - }) - } - }) - - t.Run("samples with labels", func(t *testing.T) { - stream := labels.Labels{ - {Name: "test", Value: "test"}, - } - tests := []struct { - name string - labels labels.Labels - samples []logproto.PatternSample - want []patternSample - }{ - { - name: "1 samples", - labels: stream, - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - }, - want: []patternSample{ - {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, - }, - }, - { - name: "3 samples", - labels: stream, - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - {Timestamp: 20, Value: 4}, - {Timestamp: 30, Value: 6}, - }, - want: []patternSample{ - {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"", stream, logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"", stream, logproto.PatternSample{Timestamp: 30, Value: 6}}, - }, - }, - { - name: "empty", - labels: labels.EmptyLabels(), - samples: nil, - want: nil, + want: []patternSample{ + {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"foo", logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"foo", logproto.PatternSample{Timestamp: 30, Value: 6}}, }, - } + }, + { + name: "empty", + pattern: "foo", + samples: nil, + want: nil, + }, + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - got := slice(NewLabelsSlice(tt.labels, tt.samples)) - require.Equal(t, tt.want, got) - }) - } - }) + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + got := slice(NewSlice(tt.pattern, tt.samples)) + require.Equal(t, tt.want, got) + }) + } } func slice(it Iterator) []patternSample { @@ -117,7 +63,6 @@ func slice(it Iterator) []patternSample { samples = append(samples, patternSample{ pattern: it.Pattern(), sample: it.At(), - labels: it.Labels(), }) } if it.Error() != nil { diff --git a/pkg/pattern/iter/merge.go b/pkg/pattern/iter/merge.go index 0c7c19a633a9..3b0e07e33b8a 100644 --- a/pkg/pattern/iter/merge.go +++ b/pkg/pattern/iter/merge.go @@ -5,7 +5,6 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/util/loser" - "github.com/prometheus/prometheus/model/labels" ) type mergeIterator struct { @@ -17,22 +16,18 @@ type mergeIterator struct { type patternSample struct { pattern string - labels labels.Labels sample logproto.PatternSample } var max = patternSample{ pattern: "", - labels: labels.Labels{}, sample: logproto.PatternSample{Timestamp: math.MaxInt64}, } func NewMerge(iters ...Iterator) Iterator { - // TODO: I need to call next here tree := loser.New(iters, max, func(s Iterator) patternSample { return patternSample{ pattern: s.Pattern(), - labels: s.Labels(), sample: s.At(), } }, func(e1, e2 patternSample) bool { @@ -62,13 +57,10 @@ func (m *mergeIterator) Next() bool { } m.current.pattern = m.tree.Winner().Pattern() - m.current.labels = m.tree.Winner().Labels() m.current.sample = m.tree.Winner().At() for m.tree.Next() { - if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || - m.current.pattern != m.tree.Winner().Pattern() || - m.current.labels.String() != m.tree.Winner().Labels().String() { + if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || m.current.pattern != m.tree.Winner().Pattern() { return true } m.current.sample.Value += m.tree.Winner().At().Value @@ -82,10 +74,6 @@ func (m *mergeIterator) Pattern() string { return m.current.pattern } -func (m *mergeIterator) Labels() labels.Labels { - return m.current.labels -} - func (m *mergeIterator) At() logproto.PatternSample { return m.current.sample } diff --git a/pkg/pattern/iter/merge_test.go b/pkg/pattern/iter/merge_test.go index d55f417062bd..a1d643a5a01c 100644 --- a/pkg/pattern/iter/merge_test.go +++ b/pkg/pattern/iter/merge_test.go @@ -3,150 +3,76 @@ package iter import ( "testing" - "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestMerge(t *testing.T) { - t.Run("merging patterns", func(t *testing.T) { - tests := []struct { - name string - iterators []Iterator - expected []patternSample - }{ - { - name: "Empty iterators", - iterators: []Iterator{}, - expected: nil, + tests := []struct { + name string + iterators []Iterator + expected []patternSample + }{ + { + name: "Empty iterators", + iterators: []Iterator{}, + expected: nil, + }, + { + name: "Merge single iterator", + iterators: []Iterator{ + NewSlice("a", []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, + }), }, - { - name: "Merge single iterator", - iterators: []Iterator{ - NewPatternSlice("a", []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, - }), - }, - expected: []patternSample{ - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, - }, + expected: []patternSample{ + {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"a", logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, }, - { - name: "Merge multiple iterators", - iterators: []Iterator{ - NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), - }, - expected: []patternSample{ - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, - {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, - }, + }, + { + name: "Merge multiple iterators", + iterators: []Iterator{ + NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), }, - { - name: "Merge multiple iterators with similar samples", - iterators: []Iterator{ - NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), - }, - expected: []patternSample{ - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 4}}, - {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 12}}, - {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, - }, + expected: []patternSample{ + {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, + {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, }, - } - - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - it := NewMerge(tt.iterators...) - defer it.Close() - - var result []patternSample - for it.Next() { - result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) - } - - require.Equal(t, tt.expected, result) - }) - } - }) - - t.Run("merging label samples", func(t *testing.T) { - stream1 := labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "ying", Value: "yang"}} - stream2 := labels.Labels{labels.Label{Name: "foo", Value: "baz"}, labels.Label{Name: "ying", Value: "yang"}} - tests := []struct { - name string - iterators []Iterator - expected []patternSample - }{ - { - name: "Empty iterators", - iterators: []Iterator{}, - expected: nil, - }, - { - name: "Merge single iterator", - iterators: []Iterator{ - NewLabelsSlice(stream1, []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, - }), - }, - expected: []patternSample{ - {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"", stream1, logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, - }, - }, - { - name: "Merge multiple iterators", - iterators: []Iterator{ - NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), - }, - expected: []patternSample{ - {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, - {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, - }, + }, + { + name: "Merge multiple iterators with similar samples", + iterators: []Iterator{ + NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), }, - { - name: "Merge multiple iterators with similar samples", - iterators: []Iterator{ - NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), - }, - expected: []patternSample{ - {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 4}}, - {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 12}}, - {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, - }, + expected: []patternSample{ + {"a", logproto.PatternSample{Timestamp: 10, Value: 4}}, + {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", logproto.PatternSample{Timestamp: 30, Value: 12}}, + {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, }, - } + }, + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - it := NewMerge(tt.iterators...) - defer it.Close() + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + it := NewMerge(tt.iterators...) + defer it.Close() - var result []patternSample - for it.Next() { - result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) - } + var result []patternSample + for it.Next() { + result = append(result, patternSample{it.Pattern(), it.At()}) + } - require.Equal(t, tt.expected, result) - }) - } - }) + require.Equal(t, tt.expected, result) + }) + } } diff --git a/pkg/pattern/iter/query_client.go b/pkg/pattern/iter/query_client.go index 00ed407a39e9..f471999d62e3 100644 --- a/pkg/pattern/iter/query_client.go +++ b/pkg/pattern/iter/query_client.go @@ -3,9 +3,8 @@ package iter import ( "io" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/promql/parser" ) type queryClientIterator struct { @@ -40,10 +39,6 @@ func (i *queryClientIterator) Pattern() string { return i.curr.Pattern() } -func (i *queryClientIterator) Labels() labels.Labels { - return i.curr.Labels() -} - func (i *queryClientIterator) At() logproto.PatternSample { return i.curr.At() } @@ -64,17 +59,59 @@ func NewQueryResponseIterator(resp *logproto.QueryPatternsResponse) Iterator { for j, sample := range s.Samples { samples[j] = *sample } + iters[i] = NewSlice(s.Pattern, samples) + } + return NewMerge(iters...) +} + +type querySamplesClientIterator struct { + client logproto.Pattern_QuerySampleClient + err error + curr iter.SampleIterator +} + +// NewQueryClientIterator returns an iterator over a QueryClient. +func NewQuerySamplesClientIterator(client logproto.Pattern_QuerySampleClient) iter.SampleIterator { + return &querySamplesClientIterator{ + client: client, + } +} - switch s.GetIdentifier().(type) { - case *logproto.PatternSeries_Labels: - ls, err := parser.ParseMetric(s.GetLabels()) - if err != nil { - ls = labels.Labels{} - } - iters[i] = NewLabelsSlice(ls, samples) - default: - iters[i] = NewPatternSlice(s.GetPattern(), samples) +func (i *querySamplesClientIterator) Next() bool { + for i.curr == nil || !i.curr.Next() { + batch, err := i.client.Recv() + if err == io.EOF { + return false + } else if err != nil { + i.err = err + return false } + i.curr = NewQuerySamplesResponseIterator(batch) } - return NewMerge(iters...) + + return true +} + +func (i *querySamplesClientIterator) Sample() logproto.Sample { + return i.curr.Sample() +} + +func (i *querySamplesClientIterator) StreamHash() uint64 { + return i.curr.StreamHash() +} + +func (i *querySamplesClientIterator) Labels() string { + return i.curr.Labels() +} + +func (i *querySamplesClientIterator) Error() error { + return i.err +} + +func (i *querySamplesClientIterator) Close() error { + return i.client.CloseSend() +} + +func NewQuerySamplesResponseIterator(resp *logproto.QuerySamplesResponse) iter.SampleIterator { + return iter.NewMultiSeriesIterator(resp.Series) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index cbc688461849..0198a91a6164 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -3,15 +3,17 @@ package metric import ( "context" "fmt" + "slices" "sort" "time" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/chunk" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/iter" ) type MetricType int @@ -34,7 +36,7 @@ func NewChunks(labels labels.Labels) *Chunks { } } -func (c *Chunks) Observe(bytes, count uint64, ts model.Time) { +func (c *Chunks) Observe(bytes, count float64, ts model.Time) { if len(c.chunks) == 0 { c.chunks = append(c.chunks, newChunk(bytes, count, ts)) return @@ -54,14 +56,14 @@ func (c *Chunks) Iterator( typ MetricType, grouping *syntax.Grouping, from, through, step model.Time, -) (iter.Iterator, error) { +) (iter.SampleIterator, error) { if typ == Unsupported { return nil, fmt.Errorf("unsupported metric type") } lbls := c.labels if grouping != nil { - sort.Strings(grouping.Groups) + sort.Strings(grouping.Groups) lbls = make(labels.Labels, 0, len(grouping.Groups)) for _, group := range grouping.Groups { value := c.labels.Get(group) @@ -69,31 +71,44 @@ func (c *Chunks) Iterator( } } - iters := make([]iter.Iterator, 0, len(c.chunks)) + // could have up to through-from/step steps for each chunk + maximumSteps := int64(((through-from)/step)+1) * int64(len(c.chunks)) + samples := make([]logproto.Sample, 0, maximumSteps) for _, chunk := range c.chunks { - samples, err := chunk.ForRangeAndType(typ, from, through, step) + ss, err := chunk.ForTypeAndRange(typ, from, through) if err != nil { return nil, err } - if len(samples) == 0 { + if len(ss) == 0 { continue } - iters = append(iters, iter.NewLabelsSlice(lbls, samples)) + samples = append(samples, ss...) } - return iter.NewNonOverlappingLabelsIterator(lbls, iters), nil + slices.SortFunc(samples, func(i, j logproto.Sample) int { + if i.Timestamp < j.Timestamp { + return -1 + } + + if i.Timestamp > j.Timestamp { + return 1 + } + return 0 + }) + + series := logproto.Series{Labels: lbls.String(), Samples: samples, StreamHash: lbls.Hash()} + return iter.NewSeriesIterator(series), nil } -// TODO(twhitney): These values should be float64s (to match prometheus samples) or int64s (to match pattern samples) type MetricSample struct { Timestamp model.Time - Bytes uint64 - Count uint64 + Bytes float64 + Count float64 } -func newSample(bytes, count uint64, ts model.Time) MetricSample { +func newSample(bytes, count float64, ts model.Time) MetricSample { return MetricSample{ Timestamp: ts, Bytes: bytes, @@ -125,7 +140,7 @@ func (c *Chunk) AddSample(s MetricSample) { } } -func newChunk(bytes, count uint64, ts model.Time) Chunk { +func newChunk(bytes, count float64, ts model.Time) Chunk { maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 v := Chunk{Samples: make(MetricSamples, 1, maxSize)} v.Samples[0] = newSample(bytes, count, ts) @@ -140,14 +155,14 @@ func (c *Chunk) spaceFor(ts model.Time) bool { return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime } -// TODO(twhitney): any way to remove the duplication between this and the drain chunk ForRange method? -// ForRangeAndType returns samples with only the values -// in the given range [start:end] and aggregates them by step duration. -// start and end are in milliseconds since epoch. step is a duration in milliseconds. -func (c *Chunk) ForRangeAndType( +// ForTypeAndRange returns samples with only the values +// in the given range [start:end), with no aggregation as that will be done in +// the step evaluator. start and end are in milliseconds since epoch. +// step is a duration in milliseconds. +func (c *Chunk) ForTypeAndRange( typ MetricType, - start, end, step model.Time, -) ([]logproto.PatternSample, error) { + start, end model.Time, +) ([]logproto.Sample, error) { if typ == Unsupported { return nil, fmt.Errorf("unsupported metric type") } @@ -156,58 +171,55 @@ func (c *Chunk) ForRangeAndType( return nil, nil } - first := c.Samples[0].Timestamp // why is this in the future? + first := c.Samples[0].Timestamp last := c.Samples[len(c.Samples)-1].Timestamp startBeforeEnd := start >= end - samplesAreAfterRange := first > end + samplesAreAfterRange := first >= end samplesAreBeforeRange := last < start if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { return nil, nil } - var lo int - if start > first { - lo = sort.Search(len(c.Samples), func(i int) bool { - return c.Samples[i].Timestamp >= start - }) - } + lo := 0 hi := len(c.Samples) - if end < last { - hi = sort.Search(len(c.Samples), func(i int) bool { - return c.Samples[i].Timestamp > end - }) - } - - // Re-scale samples into step-sized buckets - currentStep := chunk.TruncateTimestamp(c.Samples[lo].Timestamp, step) - numOfSteps := ((c.Samples[hi-1].Timestamp - currentStep) / step) + 1 - aggregatedSamples := make([]logproto.PatternSample, 0, numOfSteps) - aggregatedSamples = append(aggregatedSamples, - logproto.PatternSample{ - Timestamp: currentStep, - Value: 0, - }) - - for _, sample := range c.Samples[lo:hi] { - if sample.Timestamp >= currentStep+step { - stepForSample := chunk.TruncateTimestamp(sample.Timestamp, step) - for i := currentStep + step; i <= stepForSample; i += step { - aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ - Timestamp: i, - Value: 0, - }) - } - currentStep = stepForSample + + for i, sample := range c.Samples { + if first >= start { + break } - var v int64 - if typ == Bytes { - v = int64(sample.Bytes) - } else { - v = int64(sample.Count) + if first < start && sample.Timestamp >= start { + lo = i + first = sample.Timestamp } + } - aggregatedSamples[len(aggregatedSamples)-1].Value += v + for i := hi - 1; i >= 0; i-- { + if last < end { + break + } + + sample := c.Samples[i] + if last >= end && sample.Timestamp < end { + hi = i + 1 + last = sample.Timestamp + } + } + + aggregatedSamples := make([]logproto.Sample, len(c.Samples[lo:hi])) + for i, sample := range c.Samples[lo:hi] { + if sample.Timestamp >= start && sample.Timestamp < end { + var v float64 + if typ == Bytes { + v = sample.Bytes + } else { + v = sample.Count + } + aggregatedSamples[i] = logproto.Sample{ + Timestamp: sample.Timestamp.UnixNano(), + Value: v, + } + } } return aggregatedSamples, nil diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 4ae6a0575358..77848cc3957b 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -13,14 +13,14 @@ import ( "github.com/stretchr/testify/require" ) -func TestForRangeAndType(t *testing.T) { +func TestForTypeAndRange(t *testing.T) { testCases := []struct { name string c *Chunk metricType MetricType start model.Time end model.Time - expected []logproto.PatternSample + expected []logproto.Sample }{ { name: "Empty count", @@ -72,10 +72,10 @@ func TestForRangeAndType(t *testing.T) { metricType: Count, start: 0, end: 10, - expected: []logproto.PatternSample{ - {Timestamp: 2, Value: 2}, - {Timestamp: 4, Value: 4}, - {Timestamp: 6, Value: 6}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, + {Timestamp: 6 * 1e6, Value: 6}, }, }, { @@ -88,10 +88,10 @@ func TestForRangeAndType(t *testing.T) { metricType: Bytes, start: 0, end: 10, - expected: []logproto.PatternSample{ - {Timestamp: 2, Value: 2}, - {Timestamp: 4, Value: 4}, - {Timestamp: 6, Value: 6}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, + {Timestamp: 6 * 1e6, Value: 6}, }, }, { @@ -102,9 +102,12 @@ func TestForRangeAndType(t *testing.T) { {Timestamp: 6, Count: 6}, }}, metricType: Count, - start: 3, + start: 2, end: 5, - expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, + }, }, { name: "Partial Overlap -- bytes", @@ -114,9 +117,12 @@ func TestForRangeAndType(t *testing.T) { {Timestamp: 6, Bytes: 6}, }}, metricType: Bytes, - start: 3, + start: 2, end: 5, - expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, + }, }, { name: "Single Element in Range -- count", @@ -128,7 +134,7 @@ func TestForRangeAndType(t *testing.T) { metricType: Count, start: 4, end: 5, - expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + expected: []logproto.Sample{{Timestamp: 4 * 1e6, Value: 4}}, }, { name: "Single Element in Range -- bytes", @@ -140,7 +146,7 @@ func TestForRangeAndType(t *testing.T) { metricType: Bytes, start: 4, end: 5, - expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + expected: []logproto.Sample{{Timestamp: 4 * 1e6, Value: 4}}, }, { name: "Start Before First Element -- count", @@ -152,9 +158,9 @@ func TestForRangeAndType(t *testing.T) { metricType: Count, start: 0, end: 5, - expected: []logproto.PatternSample{ - {Timestamp: 2, Value: 2}, - {Timestamp: 4, Value: 4}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, }, }, { @@ -167,9 +173,9 @@ func TestForRangeAndType(t *testing.T) { metricType: Bytes, start: 0, end: 5, - expected: []logproto.PatternSample{ - {Timestamp: 2, Value: 2}, - {Timestamp: 4, Value: 4}, + expected: []logproto.Sample{ + {Timestamp: 2 * 1e6, Value: 2}, + {Timestamp: 4 * 1e6, Value: 4}, }, }, { @@ -182,8 +188,8 @@ func TestForRangeAndType(t *testing.T) { metricType: Count, start: 5, end: 10, - expected: []logproto.PatternSample{ - {Timestamp: 6, Value: 6}, + expected: []logproto.Sample{ + {Timestamp: 6 * 1e6, Value: 6}, }, }, { @@ -196,36 +202,40 @@ func TestForRangeAndType(t *testing.T) { metricType: Bytes, start: 5, end: 10, - expected: []logproto.PatternSample{ - {Timestamp: 6, Value: 6}, + expected: []logproto.Sample{ + {Timestamp: 6 * 1e6, Value: 6}, }, }, { - name: "Start before First and End Inclusive of First Element -- count", + name: "End Exclusive -- count", c: &Chunk{Samples: MetricSamples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, }}, metricType: Count, - start: 0, - end: 2, - expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + start: 4, + end: 6, + expected: []logproto.Sample{ + {Timestamp: 4 * 1e6, Value: 4}, + }, }, { - name: "Start before First and End Inclusive of First Element -- bytes", + name: "End Exclusive -- bytes", c: &Chunk{Samples: MetricSamples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, }}, metricType: Bytes, - start: 0, - end: 2, - expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + start: 4, + end: 6, + expected: []logproto.Sample{ + {Timestamp: 4 * 1e6, Value: 4}, + }, }, { - name: "Start and End before First Element -- count", + name: "Start before First and End Inclusive of First Element -- count", c: &Chunk{Samples: MetricSamples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, @@ -233,11 +243,11 @@ func TestForRangeAndType(t *testing.T) { }}, metricType: Count, start: 0, - end: 1, - expected: nil, + end: 3, + expected: []logproto.Sample{{Timestamp: 2 * 1e6, Value: 2}}, }, { - name: "Start and End before First Element -- bytes", + name: "Start before First and End Inclusive of First Element -- bytes", c: &Chunk{Samples: MetricSamples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, @@ -245,84 +255,38 @@ func TestForRangeAndType(t *testing.T) { }}, metricType: Bytes, start: 0, - end: 1, - expected: nil, - }, - { - name: "Higher resolution samples down-sampled to preceding step bucket -- count", - c: &Chunk{Samples: MetricSamples{ - {Timestamp: 1, Count: 2}, - {Timestamp: 2, Count: 4}, - {Timestamp: 3, Count: 6}, - {Timestamp: 4, Count: 8}, - {Timestamp: 5, Count: 10}, - {Timestamp: 6, Count: 12}, - }}, - metricType: Count, - start: 1, - end: 6, - expected: []logproto.PatternSample{ - {Timestamp: 0, Value: 2}, - {Timestamp: 2, Value: 10}, - {Timestamp: 4, Value: 18}, - {Timestamp: 6, Value: 12}, - }, + end: 3, + expected: []logproto.Sample{{Timestamp: 2 * 1e6, Value: 2}}, }, { - name: "Higher resolution samples down-sampled to preceding step bucket -- bytes", - c: &Chunk{Samples: MetricSamples{ - {Timestamp: 1, Bytes: 2}, - {Timestamp: 2, Bytes: 4}, - {Timestamp: 3, Bytes: 6}, - {Timestamp: 4, Bytes: 8}, - {Timestamp: 5, Bytes: 10}, - {Timestamp: 6, Bytes: 12}, - }}, - metricType: Bytes, - start: 1, - end: 6, - expected: []logproto.PatternSample{ - {Timestamp: 0, Value: 2}, - {Timestamp: 2, Value: 10}, - {Timestamp: 4, Value: 18}, - {Timestamp: 6, Value: 12}, - }, - }, - { - name: "Low resolution samples insert 0 values for empty steps -- count", + name: "Start and End before First Element -- count", c: &Chunk{Samples: MetricSamples{ - {Timestamp: 1, Count: 2}, - {Timestamp: 5, Count: 10}, + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, }}, metricType: Count, - start: 1, - end: 6, - expected: []logproto.PatternSample{ - {Timestamp: 0, Value: 2}, - {Timestamp: 2, Value: 0}, - {Timestamp: 4, Value: 10}, - }, + start: 0, + end: 1, + expected: nil, }, { - name: "Low resolution samples insert 0 values for empty steps -- bytes", + name: "Start and End before First Element -- bytes", c: &Chunk{Samples: MetricSamples{ - {Timestamp: 1, Bytes: 2}, - {Timestamp: 5, Bytes: 10}, + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, }}, metricType: Bytes, - start: 1, - end: 6, - expected: []logproto.PatternSample{ - {Timestamp: 0, Value: 2}, - {Timestamp: 2, Value: 0}, - {Timestamp: 4, Value: 10}, - }, + start: 0, + end: 1, + expected: nil, }, } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - result, err := tc.c.ForRangeAndType(tc.metricType, tc.start, tc.end, model.Time(2)) + result, err := tc.c.ForTypeAndRange(tc.metricType, tc.start, tc.end) require.NoError(t, err) if !reflect.DeepEqual(result, tc.expected) { t.Errorf("Expected %v, got %v", tc.expected, result) @@ -332,6 +296,7 @@ func TestForRangeAndType(t *testing.T) { } } +// TODO(twhitney): test the maximum steps logic func Test_Chunks_Iterator(t *testing.T) { ctx := context.Background() lbls := labels.Labels{ @@ -357,7 +322,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err := chunks.Iterator(ctx, Bytes, nil, 0, 10, 2) require.NoError(t, err) - res, err := iter.ReadAllWithLabels(it) + res, err := iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) @@ -366,7 +331,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err = chunks.Iterator(ctx, Count, nil, 0, 10, 2) require.NoError(t, err) - res, err = iter.ReadAllWithLabels(it) + res, err = iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) @@ -389,7 +354,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) require.NoError(t, err) - res, err := iter.ReadAllWithLabels(it) + res, err := iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) @@ -398,7 +363,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) require.NoError(t, err) - res, err = iter.ReadAllWithLabels(it) + res, err = iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) @@ -421,7 +386,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) require.NoError(t, err) - res, err := iter.ReadAllWithLabels(it) + res, err := iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) @@ -430,7 +395,7 @@ func Test_Chunks_Iterator(t *testing.T) { it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) require.NoError(t, err) - res, err = iter.ReadAllWithLabels(it) + res, err = iter.ReadAllSamples(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index e4c7368a959e..aeb3f11c5c3e 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -10,7 +10,6 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" "github.com/pkg/errors" "github.com/prometheus/common/model" @@ -118,11 +117,11 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( params := NewParams( e, - from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset), - through.Add(-rangExpr.Left.Offset), + from, + through, step, ) - return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), rangExpr, params, rangExpr.Left.Offset) + return NewSampleRangeAggEvaluator(loki_iter.NewPeekingSampleIterator(it), rangExpr, params, rangExpr.Left.Offset) }) } @@ -152,11 +151,11 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( params := NewParams( e, - from.Add(-e.Left.Interval).Add(-e.Left.Offset), - through.Add(-e.Left.Offset), - step, // expecting nanoseconds + from, + through, + step, ) - return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), e, params, e.Left.Offset) + return NewSampleRangeAggEvaluator(loki_iter.NewPeekingSampleIterator(it), e, params, e.Left.Offset) default: return nil, errors.Errorf("unexpected expr type (%T)", e) } @@ -165,7 +164,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( // Need to create our own StepEvaluator since we only support bytes and count over time, // and always sum to get those values. In order to accomplish this we need control over the // aggregation operation.. -func NewPatternSampleRangeAggEvaluator( +func NewSampleRangeAggEvaluator( it loki_iter.PeekingSampleIterator, expr *syntax.RangeAggregationExpr, q logql.Params, @@ -243,7 +242,7 @@ func newRangeVectorIterator( ), nil } -type SeriesToSampleIterator struct { +type seriesToSampleIterator struct { floats []promql.FPoint curTs int64 cur float64 @@ -252,47 +251,47 @@ type SeriesToSampleIterator struct { // TODO: could this me a matrix iterator that returned multiple samples with // different labels for the same timestamp? -func NewSeriesToSampleIterator(series *promql.Series) *SeriesToSampleIterator { - return &SeriesToSampleIterator{ +func NewSeriesToSampleIterator(series *promql.Series) *seriesToSampleIterator { + return &seriesToSampleIterator{ floats: series.Floats, lbls: series.Metric, } } -func (s *SeriesToSampleIterator) Next() bool { +func (s *seriesToSampleIterator) Next() bool { if len(s.floats) == 0 { return false } current, rest := s.floats[0], s.floats[1:] - s.curTs = current.T + s.curTs = current.T * 1e6 // convert to nanoseconds s.cur = current.F s.floats = rest return true } -func (s *SeriesToSampleIterator) Pattern() string { - return "" +func (s *seriesToSampleIterator) Labels() string { + return s.lbls.String() } -func (s *SeriesToSampleIterator) Labels() labels.Labels { - return s.lbls +func (s *seriesToSampleIterator) Sample() logproto.Sample { + return logproto.Sample{ + Timestamp: s.curTs, + Value: s.cur, + } } -func (s *SeriesToSampleIterator) At() logproto.PatternSample { - return logproto.PatternSample{ - Timestamp: model.Time(s.curTs), - Value: int64(s.cur), - } +func (s *seriesToSampleIterator) StreamHash() uint64 { + return s.lbls.Hash() } -func (s *SeriesToSampleIterator) Error() error { +func (s *seriesToSampleIterator) Error() error { return nil } -func (s *SeriesToSampleIterator) Close() error { +func (s *seriesToSampleIterator) Close() error { return nil } diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 62440a354a64..6d81ab7edfd6 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -12,7 +12,8 @@ import ( ) func Test_SampleEvaluator(t *testing.T) { - fiveMin := int64(300000) + fiveMinMs := int64(300000) + tenSecMs := int64(1e4) stream := labels.Labels{ labels.Label{ Name: "foo", @@ -24,7 +25,7 @@ func Test_SampleEvaluator(t *testing.T) { }, } - setup := func(chunks Chunks, now int64, query string) logql.StepEvaluator { + setup := func(chunks Chunks, from, through, step int64, query string) logql.StepEvaluator { factory := NewDefaultEvaluatorFactory(&chunks) expr, err := syntax.ParseSampleExpr(query) @@ -34,7 +35,8 @@ func Test_SampleEvaluator(t *testing.T) { context.Background(), factory, expr.(syntax.SampleExpr), - model.Time(now-fiveMin), model.Time(now), model.Time(fiveMin), + // add 10s to the end to include chunks @ now + model.Time(from), model.Time(through), model.Time(fiveMinMs), ) require.NoError(t, err) @@ -81,13 +83,13 @@ func Test_SampleEvaluator(t *testing.T) { }, } t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { - now := int64(1715964275000) - then := now - fiveMin // 1715963975000 -- 5m before n - beforeThen := then - fiveMin // 1715963675000 -- 5m before then + now := int64(600000) // 10 minutes after 0 + then := now - fiveMinMs // 300000, 5m before now + beforeThen := then - fiveMinMs // 0, 5m before then chks := chunks(now, then, beforeThen) t.Run("count", func(t *testing.T) { - evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + evaluator := setup(chks, beforeThen, now+1e4, fiveMinMs, `sum by (level)(count_over_time({foo="bar"}[5m]))`) resultTs := make([]int64, 3) resultVals := make([]float64, 3) @@ -95,9 +97,11 @@ func Test_SampleEvaluator(t *testing.T) { for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, then-fiveMin <= ts && ts <= now) + require.True(t, then-fiveMinMs <= ts && ts <= now) // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + vec := r.SampleVector() + require.Equal(t, 1, len(vec)) resultTs[i] = r.SampleVector()[0].T resultVals[i] = r.SampleVector()[0].F @@ -117,7 +121,7 @@ func Test_SampleEvaluator(t *testing.T) { }) t.Run("bytes", func(t *testing.T) { - evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + evaluator := setup(chks, beforeThen, now+tenSecMs, fiveMinMs, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) resultTs := make([]int64, 3) resultVals := make([]float64, 3) @@ -125,7 +129,7 @@ func Test_SampleEvaluator(t *testing.T) { for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, then-fiveMin <= ts && ts <= now) + require.True(t, then-fiveMinMs <= ts && ts <= now) // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. resultTs[i] = r.SampleVector()[0].T @@ -148,84 +152,116 @@ func Test_SampleEvaluator(t *testing.T) { }) t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { - now := int64(1715964275000) - then := now - 150000 // 1715964125000 -- 2.5m before now - beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + now := int64(150000 + 450000) // 600000, 10m after 0 + then := now - 150000 // 450000 -- 2.5m before now + beforeThen := then - 450000 // 0 -- 7.5m before then, 10m before now chks := chunks(now, then, beforeThen) t.Run("count", func(t *testing.T) { - evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) - - resultTs := make([]int64, 3) - resultVals := make([]float64, 3) - - start := (now - fiveMin - fiveMin) // from - step + evaluator := setup(chks, beforeThen, now+1e4, fiveMinMs, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 0, 2) + resultVals := make([]float64, 0, 2) + + start := (beforeThen - fiveMinMs) // from - step + + // Datapoints(ts -> val): + // 0 -> 1 + // 450000 -> 2 + // 600000 -> 3 + // + // the selection range logic is (start, end] so we expect the + // second window of 0 to 300000 to be empty + // + // 0 -> range -30000 to 0, ts: 0, val: 1 + // 1 -> range 0 to 300000, val: empty + // 2 -> range 30000 to 610000, val: 5 (includes last 2 datapoints) for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, start <= ts && ts <= now) - - // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. - resultTs[i] = r.SampleVector()[0].T - resultVals[i] = r.SampleVector()[0].F - - require.Equal(t, group, r.SampleVector()[0].Metric) + require.True(t, start <= ts && ts < now+tenSecMs) + + if i == 1 { + require.Equal(t, 0, len(r.SampleVector())) + } else { + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs = append(resultTs, r.SampleVector()[0].T) + resultVals = append(resultVals, r.SampleVector()[0].F) + require.Equal(t, group, r.SampleVector()[0].Metric) + } } ok, _, _ := evaluator.Next() require.False(t, ok) - require.Equal(t, now-600000, resultTs[0]) - require.Equal(t, now-fiveMin, resultTs[1]) - require.Equal(t, now, resultTs[2]) + // Because of the 5m step and 5m lookback, the first window will be + // -300000 to 0, the second 0 to 300000, and the third 300000 to 600000. + // We don't expect the 2nd to have any data, so below we check results of the 1st and 3rd. + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, beforeThen+fiveMinMs+fiveMinMs, resultTs[1]) require.Equal(t, float64(1), resultVals[0]) - require.Equal(t, float64(0), resultVals[1]) - require.Equal(t, float64(5), resultVals[2]) + require.Equal(t, float64(5), resultVals[1]) }) t.Run("bytes", func(t *testing.T) { - evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) - - resultTs := make([]int64, 3) - resultVals := make([]float64, 3) - - start := (now - fiveMin - fiveMin) // from - step + evaluator := setup(chks, beforeThen, now+1e4, fiveMinMs, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 0, 2) + resultVals := make([]float64, 0, 2) + + start := (beforeThen - fiveMinMs) // from - step + + // Datapoints(ts -> val): + // 0 -> 1 + // 450000 -> 3 + // 600000 -> 5 + // + // the selection range logic is (start, end] so we expect the + // second window of 0 to 300000 to be empty + // + // 0 -> range -30000 to 0, ts: 0, val: 1 + // 1 -> range 0 to 300000, val: empty + // 2 -> range 30000 to 610000, val: 8 (includes last 2 datapoints) for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) require.True(t, start <= ts && ts <= now) - // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. - resultTs[i] = r.SampleVector()[0].T - resultVals[i] = r.SampleVector()[0].F - - require.Equal(t, group, r.SampleVector()[0].Metric) + if i == 1 { + require.Equal(t, 0, len(r.SampleVector())) + } else { + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs = append(resultTs, r.SampleVector()[0].T) + resultVals = append(resultVals, r.SampleVector()[0].F) + require.Equal(t, group, r.SampleVector()[0].Metric) + } } ok, _, _ := evaluator.Next() require.False(t, ok) + // Because of the 5m step and 5m lookback, the first window will be + // -300000 to 0, the second 0 to 300000, and the third 300000 to 600000. + // We don't expect the 2nd to have any data, so below we check results of the 1st and 3rd. require.Equal(t, now-600000, resultTs[0]) - require.Equal(t, now-fiveMin, resultTs[1]) - require.Equal(t, now, resultTs[2]) + require.Equal(t, now, resultTs[1]) require.Equal(t, float64(1), resultVals[0]) - require.Equal(t, float64(0), resultVals[1]) - require.Equal(t, float64(8), resultVals[2]) + require.Equal(t, float64(8), resultVals[1]) }) }) }) t.Run("without grouping", func(t *testing.T) { t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { - now := int64(1715964275000) - then := now - fiveMin // 1715963975000 -- 5m before n - beforeThen := then - fiveMin // 1715963675000 -- 5m before then + now := int64(600000) // 10 minutes after 0 + then := now - fiveMinMs // 300000, 5m before now + beforeThen := then - fiveMinMs // 0, 5m before then chks := chunks(now, then, beforeThen) t.Run("count", func(t *testing.T) { - evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) + evaluator := setup(chks, beforeThen, now+tenSecMs, fiveMinMs, `count_over_time({foo="bar"}[5m])`) resultTs := make([]int64, 3) resultVals := make([]float64, 3) @@ -233,7 +269,7 @@ func Test_SampleEvaluator(t *testing.T) { for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, then-fiveMin <= ts && ts <= now) + require.True(t, then-fiveMinMs <= ts && ts <= now) // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. samples := r.SampleVector() @@ -256,15 +292,16 @@ func Test_SampleEvaluator(t *testing.T) { }) t.Run("bytes", func(t *testing.T) { - evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) + evaluator := setup(chks, beforeThen, now+1e4, fiveMinMs, `bytes_over_time({foo="bar"}[5m])`) resultTs := make([]int64, 3) resultVals := make([]float64, 3) + start := beforeThen - fiveMinMs // from - step for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, then-fiveMin <= ts && ts <= now) + require.True(t, start <= ts && ts < now+fiveMinMs) // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. samples := r.SampleVector() @@ -288,71 +325,101 @@ func Test_SampleEvaluator(t *testing.T) { }) t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { - now := int64(1715964275000) - then := now - 150000 // 1715964125000 -- 2.5m before now - beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + now := int64(150000 + 450000) // 600000 + then := now - 150000 // 450000 -- 2.5m before now + beforeThen := then - 450000 // 0 -- 7.5m before then, 10m before now chks := chunks(now, then, beforeThen) t.Run("count", func(t *testing.T) { - evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) - - resultTs := make([]int64, 3) - resultVals := make([]float64, 3) - - start := (now - fiveMin - fiveMin) // from - step + evaluator := setup(chks, beforeThen, now+tenSecMs, fiveMinMs, `count_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 0, 2) + resultVals := make([]float64, 0, 2) + + start := beforeThen - fiveMinMs // from - step + // Datapoints(ts -> val): + // 0 -> 1 + // 450000 -> 2 + // 600000 -> 3 + // + // the selection range logic is (start, end] so we expect the + // second window of 0 to 300000 to be empty + // + // 0 -> range -30000 to 0, ts: 0, val: 1 + // 1 -> range 0 to 300000, val: empty + // 2 -> range 30000 to 610000, val: 5 (includes last 2 datapoints) for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, start <= ts && ts <= now) - - // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. - resultTs[i] = r.SampleVector()[0].T - resultVals[i] = r.SampleVector()[0].F - - require.Equal(t, stream, r.SampleVector()[0].Metric) + require.True(t, start <= ts && ts < now+tenSecMs) + + if i == 1 { + require.Equal(t, 0, len(r.SampleVector())) + } else { + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs = append(resultTs, r.SampleVector()[0].T) + resultVals = append(resultVals, r.SampleVector()[0].F) + require.Equal(t, stream, r.SampleVector()[0].Metric) + } } ok, _, _ := evaluator.Next() require.False(t, ok) + // Because of the 5m step and 5m lookback, the first window will be + // -300000 to 0, the second 0 to 300000, and the third 300000 to 600000. + // We don't expect the 2nd to have any data, so below we check results of the 1st and 3rd. require.Equal(t, now-600000, resultTs[0]) - require.Equal(t, now-fiveMin, resultTs[1]) - require.Equal(t, now, resultTs[2]) + require.Equal(t, now, resultTs[1]) require.Equal(t, float64(1), resultVals[0]) - require.Equal(t, float64(0), resultVals[1]) - require.Equal(t, float64(5), resultVals[2]) + require.Equal(t, float64(5), resultVals[1]) }) t.Run("bytes", func(t *testing.T) { - evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) - - resultTs := make([]int64, 3) - resultVals := make([]float64, 3) - - start := (now - fiveMin - fiveMin) // from - step + evaluator := setup(chks, beforeThen, now+tenSecMs, fiveMinMs, `bytes_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 0, 2) + resultVals := make([]float64, 0, 2) + + start := beforeThen - fiveMinMs // from - step + // Datapoints(ts -> val): + // 0 -> 1 + // 450000 -> 3 + // 600000 -> 5 + // + // the selection range logic is (start, end] so we expect the + // second window of 0 to 300000 to be empty + // + // 0 -> range -30000 to 0, ts: 0, val: 1 + // 1 -> range 0 to 300000, val: empty + // 2 -> range 30000 to 610000, val: 8 (includes last 2 datapoints) for i := 0; i < 3; i++ { ok, ts, r := evaluator.Next() require.True(t, ok) - require.True(t, start <= ts && ts <= now) - - // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. - resultTs[i] = r.SampleVector()[0].T - resultVals[i] = r.SampleVector()[0].F - - require.Equal(t, stream, r.SampleVector()[0].Metric) + require.True(t, start <= ts && ts < now+tenSecMs) + + if i == 1 { + require.Equal(t, 0, len(r.SampleVector())) + } else { + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs = append(resultTs, r.SampleVector()[0].T) + resultVals = append(resultVals, r.SampleVector()[0].F) + require.Equal(t, stream, r.SampleVector()[0].Metric) + } } ok, _, _ := evaluator.Next() require.False(t, ok) + // Because of the 5m step and 5m lookback, the first window will be + // -300000 to 0, the second 0 to 300000, and the third 300000 to 600000. + // We don't expect the 2nd to have any data, so below we check results of the 1st and 3rd. require.Equal(t, now-600000, resultTs[0]) - require.Equal(t, now-fiveMin, resultTs[1]) - require.Equal(t, now, resultTs[2]) + require.Equal(t, now, resultTs[1]) require.Equal(t, float64(1), resultVals[0]) - require.Equal(t, float64(0), resultVals[1]) - require.Equal(t, float64(8), resultVals[2]) + require.Equal(t, float64(8), resultVals[1]) }) }) }) diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index de3538d7b05b..52f539a5f7b0 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -2,7 +2,6 @@ package pattern import ( "context" - "errors" "math" "sync" "time" @@ -12,9 +11,11 @@ import ( "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/pattern/drain" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/pattern/metric" + loki_iter "github.com/grafana/loki/v3/pkg/iter" + pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" @@ -70,14 +71,14 @@ func (s *stream) Push( s.mtx.Lock() defer s.mtx.Unlock() - bytes := uint64(0) - count := uint64(len(entries)) + bytes := float64(0) + count := float64(len(entries)) for _, entry := range entries { if entry.Timestamp.UnixNano() < s.lastTs { continue } - bytes += uint64(len(entry.Line)) + bytes += float64(len(entry.Line)) s.lastTs = entry.Timestamp.UnixNano() s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) @@ -89,13 +90,13 @@ func (s *stream) Push( return nil } -func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (iter.Iterator, error) { +func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (pattern_iter.Iterator, error) { // todo we should improve locking. s.mtx.Lock() defer s.mtx.Unlock() clusters := s.patterns.Clusters() - iters := make([]iter.Iterator, 0, len(clusters)) + iters := make([]pattern_iter.Iterator, 0, len(clusters)) for _, cluster := range clusters { if cluster.String() == "" { @@ -103,14 +104,14 @@ func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (it } iters = append(iters, cluster.Iterator(from, through, step)) } - return iter.NewMerge(iters...), nil + return pattern_iter.NewMerge(iters...), nil } func (s *stream) SampleIterator( ctx context.Context, expr syntax.SampleExpr, from, through, step model.Time, -) (iter.Iterator, error) { +) (loki_iter.SampleIterator, error) { s.mtx.Lock() defer s.mtx.Unlock() @@ -135,7 +136,7 @@ func (s *stream) SampleIterator( // this is only 1 series since we're already on a stream // this this limit needs to also be enforced higher up maxSeries := 1000 - series, err := s.JoinSampleVector( + matrix, err := s.JoinSampleVectors( next, ts, r, @@ -146,18 +147,17 @@ func (s *stream) SampleIterator( return nil, err } - return metric.NewSeriesToSampleIterator(series), nil + return loki_iter.NewMultiSeriesIterator(matrix), nil } -//TODO: should this join multiple series into a matrix, so we don't have the weird hack? -func (s *stream) JoinSampleVector( +func (s *stream) JoinSampleVectors( next bool, ts int64, r logql.StepResult, stepEvaluator logql.StepEvaluator, maxSeries int, from, through, step model.Time, -) (*promql.Series, error) { +) ([]logproto.Series, error) { stepCount := int(math.Ceil(float64(through.Sub(from).Nanoseconds()) / float64(step.UnixNano()))) if stepCount <= 0 { stepCount = 1 @@ -173,24 +173,31 @@ func (s *stream) JoinSampleVector( return nil, logqlmodel.NewSeriesLimitError(maxSeries) } - var seriesHash string - series := map[string]*promql.Series{} - for next { + series := map[uint64]*logproto.Series{} + + // step evaluator logic is slightly different than the normal contract in Loki + // when evaluating a selection range, it's counts datapoints within (start, end] + // so an additional condition of ts < through is needed to make sure this loop + // doesn't go beyond the through time. the contract for Loki queries is [start, end), + // and the samples to evaluate are selected based on [start, end) so the last result + // is likely incorrect anyway + for next && ts < int64(through) { vec = r.SampleVector() for _, p := range vec { - seriesHash = p.Metric.String() - s, ok := series[seriesHash] + hash := p.Metric.Hash() + s, ok := series[hash] if !ok { - s = &promql.Series{ - Metric: p.Metric, - Floats: make([]promql.FPoint, 0, stepCount), + s = &logproto.Series{ + Labels: p.Metric.String(), + Samples: make([]logproto.Sample, 0, stepCount), + StreamHash: hash, } - series[p.Metric.String()] = s + series[hash] = s } - s.Floats = append(s.Floats, promql.FPoint{ - T: ts, - F: p.F, + s.Samples = append(s.Samples, logproto.Sample{ + Timestamp: ts * 1e6, // convert milliseconds to nanoseconds + Value: p.F, }) } @@ -200,12 +207,12 @@ func (s *stream) JoinSampleVector( } } - if len(series) > 1 { - // TODO: is this actually a problem? Should this just become a Matrix - return nil, errors.New("multiple series found in a single stream") + matrix := make([]logproto.Series, 0, len(series)) + for _, s := range series { + matrix = append(matrix, *s) } - return series[seriesHash], stepEvaluator.Error() + return matrix, stepEvaluator.Error() } func (s *stream) prune(olderThan time.Duration) bool { diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index 7f6f98d15b21..65da8dc497a5 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -36,7 +36,7 @@ func TestAddStream(t *testing.T) { require.NoError(t, err) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAllWithPatterns(it) + res, err := iter.ReadAll(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(2), res.Series[0].Samples[0].Value) @@ -75,7 +75,7 @@ func TestPruneStream(t *testing.T) { require.Equal(t, false, stream.prune(time.Hour)) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAllWithPatterns(it) + res, err := iter.ReadAll(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index b05d9411faaf..a6618ef1931e 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -52,9 +52,7 @@ const ( tailerWaitEntryThrottle = time.Second / 2 ) -var ( - nowFunc = func() time.Time { return time.Now() } -) +var nowFunc = func() time.Time { return time.Now() } type interval struct { start, end time.Time @@ -107,6 +105,7 @@ type Querier interface { DetectedFields(ctx context.Context, req *logproto.DetectedFieldsRequest) (*logproto.DetectedFieldsResponse, error) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) DetectedLabels(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.DetectedLabelsResponse, error) + SelectMetricSamples(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QueryPatternsResponse, error) } type Limits querier_limits.Limits @@ -1036,6 +1035,7 @@ func countLabelsAndCardinality(storeLabelsMap map[string][]string, ingesterLabel type PatterQuerier interface { Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) + Samples(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QuerySamplesResponse, error) } func (q *SingleTenantQuerier) WithPatternQuerier(pq PatterQuerier) { @@ -1054,6 +1054,18 @@ func (q *SingleTenantQuerier) Patterns(ctx context.Context, req *logproto.QueryP return res, err } +func (q *SingleTenantQuerier) SelectMetricSamples(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QuerySamplesResponse, error) { + if q.patternQuerier == nil { + return nil, httpgrpc.Errorf(http.StatusNotFound, "") + } + res, err := q.patternQuerier.Samples(ctx, req) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + + return res, err +} + // containsAllIDTypes filters out all UUID, GUID and numeric types. Returns false if even one value is not of the type func containsAllIDTypes(values []string) bool { for _, v := range values { diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 51e4a6a9d8dd..957b38fb6315 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -626,6 +626,21 @@ func (q *querierMock) DetectedLabels(ctx context.Context, req *logproto.Detected return resp.(*logproto.DetectedLabelsResponse), err } +func (q *querierMock) SelectMetricSamples( + ctx context.Context, + req *logproto.QuerySamplesRequest, +) (*logproto.QuerySamplesResponse, error) { + args := q.MethodCalled("SelectMetricSamples", ctx, req) + + resp := args.Get(0) + err := args.Error(1) + if resp == nil { + return nil, err + } + + return resp.(*logproto.QuerySamplesResponse), err +} + type engineMock struct { util.ExtendedMock } diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 7bab6f6c5d05..4863df7cfcfa 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -1364,6 +1364,10 @@ func encodeResponseJSONTo(version loghttp.Version, res queryrangebase.Response, if err := marshal.WriteDetectedLabelsResponseJSON(response.Response, w); err != nil { return err } + case *QuerySamplesResponse: + if err := marshal.WriteQuerySamplesResponseJSON(response.Response, w); err != nil { + return err + } default: return httpgrpc.Errorf(http.StatusInternalServerError, fmt.Sprintf("invalid response format, got (%T)", res)) } diff --git a/pkg/querier/queryrange/extensions.go b/pkg/querier/queryrange/extensions.go index ec5fa25ae308..2aa9842e7548 100644 --- a/pkg/querier/queryrange/extensions.go +++ b/pkg/querier/queryrange/extensions.go @@ -262,6 +262,15 @@ func (m *QueryPatternsResponse) GetHeaders() []*queryrangebase.PrometheusRespons return nil } +func (m *QueryPatternsResponse) SetHeader(name, value string) { + m.Headers = setHeader(m.Headers, name, value) +} + +func (m *QueryPatternsResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { + m.Headers = h + return m +} + // GetHeaders returns the HTTP headers in the response. func (m *DetectedLabelsResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader { if m != nil { @@ -270,20 +279,30 @@ func (m *DetectedLabelsResponse) GetHeaders() []*queryrangebase.PrometheusRespon return nil } -func (m *QueryPatternsResponse) SetHeader(name, value string) { +func (m *DetectedLabelsResponse) SetHeader(name, value string) { m.Headers = setHeader(m.Headers, name, value) } -func (m *QueryPatternsResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { +func (m *DetectedLabelsResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { m.Headers = h return m } -func (m *DetectedLabelsResponse) SetHeader(name, value string) { +// GetHeaders returns the HTTP headers in the response. +func (m *QuerySamplesResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader { + if m != nil { + return convertPrometheusResponseHeadersToPointers(m.Headers) + } + return nil +} + +func (m *QuerySamplesResponse) SetHeader(name, value string) { m.Headers = setHeader(m.Headers, name, value) } -func (m *DetectedLabelsResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { +func (m *QuerySamplesResponse) WithHeaders( + h []queryrangebase.PrometheusResponseHeader, +) queryrangebase.Response { m.Headers = h return m } diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index ae2dddee539a..5db2c2ce0df6 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -1040,6 +1040,43 @@ func (m *DetectedLabelsResponse) XXX_DiscardUnknown() { var xxx_messageInfo_DetectedLabelsResponse proto.InternalMessageInfo +type QuerySamplesResponse struct { + Response *github_com_grafana_loki_v3_pkg_logproto.QuerySamplesResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/v3/pkg/logproto.QuerySamplesResponse" 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:"-"` +} + +func (m *QuerySamplesResponse) Reset() { *m = QuerySamplesResponse{} } +func (*QuerySamplesResponse) ProtoMessage() {} +func (*QuerySamplesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_51b9d53b40d11902, []int{17} +} +func (m *QuerySamplesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QuerySamplesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_QuerySamplesResponse.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 *QuerySamplesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QuerySamplesResponse.Merge(m, src) +} +func (m *QuerySamplesResponse) XXX_Size() int { + return m.Size() +} +func (m *QuerySamplesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_QuerySamplesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_QuerySamplesResponse proto.InternalMessageInfo + type QueryResponse struct { Status *rpc.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` // Types that are valid to be assigned to Response: @@ -1055,13 +1092,14 @@ type QueryResponse struct { // *QueryResponse_DetectedFields // *QueryResponse_PatternsResponse // *QueryResponse_DetectedLabels + // *QueryResponse_SamplesResponse 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) @@ -1133,6 +1171,9 @@ type QueryResponse_PatternsResponse struct { type QueryResponse_DetectedLabels struct { DetectedLabels *DetectedLabelsResponse `protobuf:"bytes,13,opt,name=detectedLabels,proto3,oneof"` } +type QueryResponse_SamplesResponse struct { + SamplesResponse *QuerySamplesResponse `protobuf:"bytes,14,opt,name=samplesResponse,proto3,oneof"` +} func (*QueryResponse_Series) isQueryResponse_Response() {} func (*QueryResponse_Labels) isQueryResponse_Response() {} @@ -1146,6 +1187,7 @@ func (*QueryResponse_ShardsResponse) isQueryResponse_Response() {} func (*QueryResponse_DetectedFields) isQueryResponse_Response() {} func (*QueryResponse_PatternsResponse) isQueryResponse_Response() {} func (*QueryResponse_DetectedLabels) isQueryResponse_Response() {} +func (*QueryResponse_SamplesResponse) isQueryResponse_Response() {} func (m *QueryResponse) GetResponse() isQueryResponse_Response { if m != nil { @@ -1245,6 +1287,13 @@ func (m *QueryResponse) GetDetectedLabels() *DetectedLabelsResponse { return nil } +func (m *QueryResponse) GetSamplesResponse() *QuerySamplesResponse { + if x, ok := m.GetResponse().(*QueryResponse_SamplesResponse); ok { + return x.SamplesResponse + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*QueryResponse) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -1260,6 +1309,7 @@ func (*QueryResponse) XXX_OneofWrappers() []interface{} { (*QueryResponse_DetectedFields)(nil), (*QueryResponse_PatternsResponse)(nil), (*QueryResponse_DetectedLabels)(nil), + (*QueryResponse_SamplesResponse)(nil), } } @@ -1275,6 +1325,7 @@ type QueryRequest struct { // *QueryRequest_DetectedFields // *QueryRequest_PatternsRequest // *QueryRequest_DetectedLabels + // *QueryRequest_SamplesRequest Request isQueryRequest_Request `protobuf_oneof:"request"` Metadata map[string]string `protobuf:"bytes,7,rep,name=metadata,proto3" json:"metadata" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } @@ -1282,7 +1333,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) @@ -1348,6 +1399,9 @@ type QueryRequest_PatternsRequest struct { type QueryRequest_DetectedLabels struct { DetectedLabels *logproto.DetectedLabelsRequest `protobuf:"bytes,11,opt,name=detectedLabels,proto3,oneof"` } +type QueryRequest_SamplesRequest struct { + SamplesRequest *logproto.QuerySamplesRequest `protobuf:"bytes,12,opt,name=samplesRequest,proto3,oneof"` +} func (*QueryRequest_Series) isQueryRequest_Request() {} func (*QueryRequest_Labels) isQueryRequest_Request() {} @@ -1359,6 +1413,7 @@ func (*QueryRequest_ShardsRequest) isQueryRequest_Request() {} func (*QueryRequest_DetectedFields) isQueryRequest_Request() {} func (*QueryRequest_PatternsRequest) isQueryRequest_Request() {} func (*QueryRequest_DetectedLabels) isQueryRequest_Request() {} +func (*QueryRequest_SamplesRequest) isQueryRequest_Request() {} func (m *QueryRequest) GetRequest() isQueryRequest_Request { if m != nil { @@ -1437,6 +1492,13 @@ func (m *QueryRequest) GetDetectedLabels() *logproto.DetectedLabelsRequest { return nil } +func (m *QueryRequest) GetSamplesRequest() *logproto.QuerySamplesRequest { + if x, ok := m.GetRequest().(*QueryRequest_SamplesRequest); ok { + return x.SamplesRequest + } + return nil +} + func (m *QueryRequest) GetMetadata() map[string]string { if m != nil { return m.Metadata @@ -1457,6 +1519,7 @@ func (*QueryRequest) XXX_OneofWrappers() []interface{} { (*QueryRequest_DetectedFields)(nil), (*QueryRequest_PatternsRequest)(nil), (*QueryRequest_DetectedLabels)(nil), + (*QueryRequest_SamplesRequest)(nil), } } @@ -1478,6 +1541,7 @@ func init() { proto.RegisterType((*DetectedFieldsResponse)(nil), "queryrange.DetectedFieldsResponse") proto.RegisterType((*QueryPatternsResponse)(nil), "queryrange.QueryPatternsResponse") proto.RegisterType((*DetectedLabelsResponse)(nil), "queryrange.DetectedLabelsResponse") + proto.RegisterType((*QuerySamplesResponse)(nil), "queryrange.QuerySamplesResponse") proto.RegisterType((*QueryResponse)(nil), "queryrange.QueryResponse") proto.RegisterType((*QueryRequest)(nil), "queryrange.QueryRequest") proto.RegisterMapType((map[string]string)(nil), "queryrange.QueryRequest.MetadataEntry") @@ -1488,128 +1552,133 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1935 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcb, 0x6f, 0x23, 0x49, - 0x19, 0x77, 0xfb, 0x19, 0x7f, 0x79, 0x4c, 0xa8, 0x09, 0xd9, 0x26, 0x3b, 0xeb, 0x36, 0x96, 0xd8, - 0x0d, 0x08, 0xec, 0x1d, 0x67, 0x77, 0xd8, 0x0d, 0xc3, 0x68, 0xa7, 0x27, 0x33, 0x64, 0x86, 0x59, - 0x76, 0xb6, 0x13, 0x71, 0xe0, 0x82, 0x2a, 0x76, 0xc5, 0x6e, 0x62, 0x77, 0xf7, 0x74, 0x97, 0x33, - 0x13, 0x09, 0xa1, 0xfd, 0x07, 0x56, 0xec, 0x5f, 0x81, 0xb8, 0x71, 0xe1, 0xc4, 0x89, 0xe3, 0xee, - 0x01, 0x69, 0x8e, 0x2b, 0x4b, 0x34, 0x8c, 0x07, 0x21, 0x94, 0xd3, 0x4a, 0x5c, 0x39, 0xa0, 0x7a, - 0x74, 0xbb, 0xca, 0x76, 0x18, 0x27, 0x20, 0xa4, 0xc0, 0x5e, 0x92, 0x7a, 0x7c, 0xbf, 0xea, 0xaa, - 0xdf, 0xf7, 0xfb, 0xbe, 0x7a, 0x18, 0xde, 0x08, 0x8e, 0x3a, 0x8d, 0xc7, 0x03, 0x12, 0xba, 0x24, - 0xe4, 0xff, 0x4f, 0x42, 0xec, 0x75, 0x88, 0x52, 0xac, 0x07, 0xa1, 0x4f, 0x7d, 0x04, 0xe3, 0x96, - 0x8d, 0x66, 0xc7, 0xa5, 0xdd, 0xc1, 0x41, 0xbd, 0xe5, 0xf7, 0x1b, 0x1d, 0xbf, 0xe3, 0x37, 0x3a, - 0xbe, 0xdf, 0xe9, 0x11, 0x1c, 0xb8, 0x91, 0x2c, 0x36, 0xc2, 0xa0, 0xd5, 0x88, 0x28, 0xa6, 0x83, - 0x48, 0xe0, 0x37, 0xd6, 0x98, 0x21, 0x2f, 0x72, 0x88, 0x6c, 0xb5, 0xa4, 0x39, 0xaf, 0x1d, 0x0c, - 0x0e, 0x1b, 0xd4, 0xed, 0x93, 0x88, 0xe2, 0x7e, 0x90, 0x18, 0xb0, 0xf9, 0xf5, 0xfc, 0x8e, 0x40, - 0xba, 0x5e, 0x9b, 0x3c, 0xed, 0x60, 0x4a, 0x9e, 0xe0, 0x13, 0x69, 0xf0, 0xaa, 0x66, 0x90, 0x14, - 0x64, 0xe7, 0x86, 0xd6, 0x19, 0x60, 0x4a, 0x49, 0xe8, 0xc9, 0xbe, 0xaf, 0x69, 0x7d, 0xd1, 0x11, - 0xa1, 0xad, 0xae, 0xec, 0xaa, 0xca, 0xae, 0xc7, 0xbd, 0xbe, 0xdf, 0x26, 0x3d, 0xbe, 0x90, 0x48, - 0xfc, 0x95, 0x16, 0x57, 0x99, 0x45, 0x30, 0x88, 0xba, 0xfc, 0x8f, 0x6c, 0xbc, 0xf3, 0x52, 0x2e, - 0x0f, 0x70, 0x44, 0x1a, 0x6d, 0x72, 0xe8, 0x7a, 0x2e, 0x75, 0x7d, 0x2f, 0x52, 0xcb, 0x72, 0x90, - 0x1b, 0xf3, 0x0d, 0x32, 0xe9, 0x9f, 0x8d, 0x37, 0x19, 0x2e, 0xa2, 0x7e, 0x88, 0x3b, 0xa4, 0xd1, - 0xea, 0x0e, 0xbc, 0xa3, 0x46, 0x0b, 0xb7, 0xba, 0xa4, 0x11, 0x92, 0x68, 0xd0, 0xa3, 0x91, 0xa8, - 0xd0, 0x93, 0x80, 0xc8, 0x2f, 0xd5, 0x3e, 0xcb, 0xc3, 0xe2, 0x43, 0xff, 0xc8, 0x75, 0xc8, 0xe3, - 0x01, 0x89, 0x28, 0x5a, 0x83, 0x02, 0x1f, 0xd5, 0x34, 0xaa, 0xc6, 0x66, 0xd9, 0x11, 0x15, 0xd6, - 0xda, 0x73, 0xfb, 0x2e, 0x35, 0xb3, 0x55, 0x63, 0x73, 0xd9, 0x11, 0x15, 0x84, 0x20, 0x1f, 0x51, - 0x12, 0x98, 0xb9, 0xaa, 0xb1, 0x99, 0x73, 0x78, 0x19, 0x6d, 0xc0, 0x82, 0xeb, 0x51, 0x12, 0x1e, - 0xe3, 0x9e, 0x59, 0xe6, 0xed, 0x69, 0x1d, 0xdd, 0x82, 0x52, 0x44, 0x71, 0x48, 0xf7, 0x23, 0x33, - 0x5f, 0x35, 0x36, 0x17, 0x9b, 0x1b, 0x75, 0xe1, 0xf9, 0x7a, 0xe2, 0xf9, 0xfa, 0x7e, 0xe2, 0x79, - 0x7b, 0xe1, 0xd3, 0xd8, 0xca, 0x7c, 0xf2, 0x27, 0xcb, 0x70, 0x12, 0x10, 0xda, 0x86, 0x02, 0xf1, - 0xda, 0xfb, 0x91, 0x59, 0x38, 0x07, 0x5a, 0x40, 0xd0, 0x75, 0x28, 0xb7, 0xdd, 0x90, 0xb4, 0x18, - 0xcb, 0x66, 0xb1, 0x6a, 0x6c, 0xae, 0x34, 0xaf, 0xd6, 0x53, 0xa1, 0xec, 0x24, 0x5d, 0xce, 0xd8, - 0x8a, 0x2d, 0x2f, 0xc0, 0xb4, 0x6b, 0x96, 0x38, 0x13, 0xbc, 0x8c, 0x6a, 0x50, 0x8c, 0xba, 0x38, - 0x6c, 0x47, 0xe6, 0x42, 0x35, 0xb7, 0x59, 0xb6, 0xe1, 0x34, 0xb6, 0x64, 0x8b, 0x23, 0xff, 0xa3, - 0x9f, 0x42, 0x3e, 0xe8, 0x61, 0xcf, 0x04, 0x3e, 0xcb, 0xd5, 0xba, 0xe2, 0xa5, 0x47, 0x3d, 0xec, - 0xd9, 0xef, 0x0e, 0x63, 0xeb, 0x6d, 0x35, 0x78, 0x42, 0x7c, 0x88, 0x3d, 0xdc, 0xe8, 0xf9, 0x47, - 0x6e, 0xe3, 0x78, 0xab, 0xa1, 0xfa, 0x9e, 0x0d, 0x54, 0xff, 0x90, 0x0d, 0xc0, 0xa0, 0x0e, 0x1f, - 0x18, 0x3d, 0x80, 0x45, 0xe6, 0x63, 0x72, 0x87, 0x39, 0x38, 0x32, 0x17, 0xf9, 0x77, 0x5e, 0x19, - 0xaf, 0x86, 0xb7, 0x3b, 0xe4, 0xf0, 0x07, 0xa1, 0x3f, 0x08, 0xec, 0x2b, 0xa7, 0xb1, 0xa5, 0xda, - 0x3b, 0x6a, 0x05, 0x3d, 0x80, 0x15, 0x26, 0x0a, 0xd7, 0xeb, 0x7c, 0x10, 0x70, 0x05, 0x9a, 0x4b, - 0x7c, 0xb8, 0x6b, 0x75, 0x55, 0x32, 0xf5, 0x3b, 0x9a, 0x8d, 0x9d, 0x67, 0xf4, 0x3a, 0x13, 0xc8, - 0xda, 0x28, 0x07, 0x88, 0x69, 0xe9, 0xbe, 0x17, 0x51, 0xec, 0xd1, 0x8b, 0x48, 0xea, 0x26, 0x14, - 0x59, 0xf0, 0xef, 0x47, 0x5c, 0x54, 0xf3, 0xfa, 0x58, 0x62, 0x74, 0x27, 0xe7, 0xcf, 0xe5, 0xe4, - 0xc2, 0x4c, 0x27, 0x17, 0x5f, 0xea, 0xe4, 0xd2, 0x7f, 0xc9, 0xc9, 0x0b, 0xff, 0x59, 0x27, 0x97, - 0x2f, 0xec, 0x64, 0x13, 0xf2, 0x6c, 0x96, 0x68, 0x15, 0x72, 0x21, 0x7e, 0xc2, 0x7d, 0xba, 0xe4, - 0xb0, 0x62, 0x6d, 0x94, 0x87, 0x25, 0x91, 0x4a, 0xa2, 0xc0, 0xf7, 0x22, 0xc2, 0x78, 0xdc, 0xe3, - 0xd9, 0x5f, 0x78, 0x5e, 0xf2, 0xc8, 0x5b, 0x1c, 0xd9, 0x83, 0xde, 0x83, 0xfc, 0x0e, 0xa6, 0x98, - 0xab, 0x60, 0xb1, 0xb9, 0xa6, 0xf2, 0xc8, 0xc6, 0x62, 0x7d, 0xf6, 0x3a, 0x9b, 0xc8, 0x69, 0x6c, - 0xad, 0xb4, 0x31, 0xc5, 0xdf, 0xf6, 0xfb, 0x2e, 0x25, 0xfd, 0x80, 0x9e, 0x38, 0x1c, 0x89, 0xde, - 0x86, 0xf2, 0xdd, 0x30, 0xf4, 0xc3, 0xfd, 0x93, 0x80, 0x70, 0xd5, 0x94, 0xed, 0x57, 0x4e, 0x63, - 0xeb, 0x2a, 0x49, 0x1a, 0x15, 0xc4, 0xd8, 0x12, 0x7d, 0x13, 0x0a, 0xbc, 0xc2, 0x75, 0x52, 0xb6, - 0xaf, 0x9e, 0xc6, 0xd6, 0x15, 0x0e, 0x51, 0xcc, 0x85, 0x85, 0x2e, 0xab, 0xc2, 0x5c, 0xb2, 0x4a, - 0xd5, 0x5d, 0x54, 0xd5, 0x6d, 0x42, 0xe9, 0x98, 0x84, 0x11, 0x1b, 0xa6, 0xc4, 0xdb, 0x93, 0x2a, - 0xba, 0x0d, 0xc0, 0x88, 0x71, 0x23, 0xea, 0xb6, 0x12, 0x67, 0x2f, 0xd7, 0xc5, 0x66, 0xe3, 0x70, - 0x1f, 0xd9, 0x48, 0xb2, 0xa0, 0x18, 0x3a, 0x4a, 0x19, 0xfd, 0xc6, 0x80, 0xd2, 0x2e, 0xc1, 0x6d, - 0x12, 0x32, 0xf7, 0xe6, 0x36, 0x17, 0x9b, 0xdf, 0xa8, 0xab, 0x3b, 0xcb, 0xa3, 0xd0, 0xef, 0x13, - 0xda, 0x25, 0x83, 0x28, 0x71, 0x90, 0xb0, 0xb6, 0xbd, 0x61, 0x6c, 0x91, 0x39, 0xa5, 0x3a, 0xd7, - 0x86, 0x76, 0xe6, 0xa7, 0x4e, 0x63, 0xcb, 0xf8, 0x8e, 0x93, 0xcc, 0x12, 0x35, 0x61, 0xe1, 0x09, - 0x0e, 0x3d, 0xd7, 0xeb, 0x44, 0x26, 0xf0, 0x48, 0x5b, 0x3f, 0x8d, 0x2d, 0x94, 0xb4, 0x29, 0x8e, - 0x48, 0xed, 0x6a, 0x7f, 0x34, 0xe0, 0x2b, 0x4c, 0x18, 0x7b, 0x6c, 0x3e, 0x91, 0x92, 0x62, 0xfa, - 0x98, 0xb6, 0xba, 0xa6, 0xc1, 0x86, 0x71, 0x44, 0x45, 0xdd, 0x6f, 0xb2, 0xff, 0xd6, 0x7e, 0x93, - 0x3b, 0xff, 0x7e, 0x93, 0xe4, 0x95, 0xfc, 0xcc, 0xbc, 0x52, 0x38, 0x2b, 0xaf, 0xd4, 0x7e, 0x29, - 0x73, 0x68, 0xb2, 0xbe, 0x73, 0x84, 0xd2, 0xbd, 0x34, 0x94, 0x72, 0x7c, 0xb6, 0xa9, 0x42, 0xc5, - 0x58, 0xf7, 0xdb, 0xc4, 0xa3, 0xee, 0xa1, 0x4b, 0xc2, 0x97, 0x04, 0x94, 0xa2, 0xd2, 0x9c, 0xae, - 0x52, 0x55, 0x62, 0xf9, 0x4b, 0x21, 0x31, 0x3d, 0xae, 0x0a, 0x17, 0x88, 0xab, 0xda, 0xdf, 0xb3, - 0xb0, 0xce, 0x3c, 0xf2, 0x10, 0x1f, 0x90, 0xde, 0x8f, 0x70, 0xff, 0x9c, 0x5e, 0x79, 0x5d, 0xf1, - 0x4a, 0xd9, 0x46, 0x5f, 0xb2, 0x3e, 0x1f, 0xeb, 0xbf, 0x32, 0x60, 0x21, 0xd9, 0x00, 0x50, 0x1d, - 0x40, 0xc0, 0x78, 0x8e, 0x17, 0x5c, 0xaf, 0x30, 0x70, 0x98, 0xb6, 0x3a, 0x8a, 0x05, 0xfa, 0x19, - 0x14, 0x45, 0x4d, 0xc6, 0x82, 0xb2, 0x6d, 0xee, 0xd1, 0x90, 0xe0, 0xfe, 0xed, 0x36, 0x0e, 0x28, - 0x09, 0xed, 0x77, 0xd9, 0x2c, 0x86, 0xb1, 0xf5, 0xc6, 0x59, 0x2c, 0x25, 0x27, 0x7c, 0x89, 0x63, - 0xfe, 0x15, 0xdf, 0x74, 0xe4, 0x17, 0x6a, 0x1f, 0x1b, 0xb0, 0xca, 0x26, 0xca, 0xa8, 0x49, 0x85, - 0xb1, 0x03, 0x0b, 0xa1, 0x2c, 0xf3, 0xe9, 0x2e, 0x36, 0x6b, 0x75, 0x9d, 0xd6, 0x19, 0x54, 0xf2, - 0x0d, 0xd7, 0x70, 0x52, 0x24, 0xda, 0xd2, 0x68, 0xcc, 0xce, 0xa2, 0x51, 0xec, 0xd1, 0x2a, 0x71, - 0xbf, 0xcf, 0x02, 0xba, 0xcf, 0x6e, 0x48, 0x4c, 0x7f, 0x63, 0xa9, 0x3e, 0x9d, 0x9a, 0xd1, 0xb5, - 0x31, 0x29, 0xd3, 0xf6, 0xf6, 0xad, 0x61, 0x6c, 0x6d, 0xbf, 0x44, 0x3b, 0xff, 0x02, 0xaf, 0xac, - 0x42, 0x95, 0x6f, 0xf6, 0x32, 0xc8, 0xb7, 0xf6, 0xdb, 0x2c, 0xac, 0xfc, 0xd8, 0xef, 0x0d, 0xfa, - 0x24, 0xa5, 0x2f, 0x98, 0xa2, 0xcf, 0x1c, 0xd3, 0xa7, 0xdb, 0xda, 0xdb, 0xc3, 0xd8, 0xba, 0x31, - 0x2f, 0x75, 0x3a, 0xf6, 0x52, 0xd3, 0xf6, 0xd7, 0x2c, 0xac, 0xed, 0xfb, 0xc1, 0x0f, 0xf7, 0xf8, - 0x2d, 0x5a, 0x49, 0x93, 0xdd, 0x29, 0xf2, 0xd6, 0xc6, 0xe4, 0x31, 0xc4, 0xfb, 0x98, 0x86, 0xee, - 0x53, 0xfb, 0xc6, 0x30, 0xb6, 0x9a, 0xf3, 0x12, 0x37, 0xc6, 0x5d, 0x66, 0xd2, 0xb4, 0x33, 0x50, - 0x6e, 0xce, 0x33, 0xd0, 0x3f, 0xb2, 0xb0, 0xfe, 0xe1, 0x00, 0x7b, 0xd4, 0xed, 0x11, 0x41, 0x76, - 0x4a, 0xf5, 0xcf, 0xa7, 0xa8, 0xae, 0x8c, 0xa9, 0xd6, 0x31, 0x92, 0xf4, 0xf7, 0x86, 0xb1, 0x75, - 0x73, 0x5e, 0xd2, 0x67, 0x8d, 0xf0, 0x7f, 0x47, 0xff, 0xef, 0xb2, 0xb0, 0xb2, 0x27, 0x4e, 0x6d, - 0xc9, 0xc2, 0x8f, 0x67, 0xd0, 0xae, 0x3e, 0x53, 0x05, 0x07, 0x75, 0x1d, 0x71, 0xbe, 0x24, 0xa1, - 0x63, 0x2f, 0x75, 0x92, 0xf8, 0x43, 0x16, 0xd6, 0x77, 0x08, 0x25, 0x2d, 0x4a, 0xda, 0xf7, 0x5c, - 0xd2, 0x53, 0x48, 0xfc, 0xc8, 0x98, 0x62, 0xb1, 0xaa, 0x5c, 0xb3, 0x66, 0x82, 0x6c, 0x7b, 0x18, - 0x5b, 0xb7, 0xe6, 0xe5, 0x71, 0xf6, 0x18, 0x97, 0x9a, 0xcf, 0xcf, 0xb2, 0xf0, 0x55, 0xf1, 0x74, - 0x20, 0xde, 0x35, 0xc7, 0x74, 0xfe, 0x62, 0x8a, 0x4d, 0x4b, 0x4d, 0x05, 0x33, 0x20, 0xf6, 0xed, - 0x61, 0x6c, 0x7d, 0x7f, 0xfe, 0x5c, 0x30, 0x63, 0x88, 0xff, 0x19, 0x6d, 0xf2, 0xd3, 0xfe, 0x79, - 0xb5, 0xa9, 0x83, 0x2e, 0xa6, 0x4d, 0x7d, 0x8c, 0x4b, 0xcd, 0xe7, 0x5f, 0x8a, 0xb0, 0xcc, 0x55, - 0x92, 0xd2, 0xf8, 0x2d, 0x90, 0xd7, 0x23, 0xc9, 0x21, 0x4a, 0xae, 0xd4, 0x61, 0xd0, 0xaa, 0xef, - 0xc9, 0x8b, 0x93, 0xb0, 0x40, 0xef, 0x40, 0x31, 0xe2, 0x17, 0x57, 0x79, 0xf2, 0xad, 0x4c, 0xbe, - 0x0d, 0xe9, 0x57, 0xe4, 0xdd, 0x8c, 0x23, 0xed, 0xd1, 0x4d, 0x28, 0xf6, 0x38, 0x8b, 0xf2, 0xe2, - 0x5e, 0x9b, 0x44, 0x4e, 0x5f, 0xe5, 0x18, 0x5a, 0x60, 0xd0, 0x0d, 0x28, 0xf0, 0x23, 0xb6, 0x7c, - 0xa3, 0xd6, 0x3e, 0x3b, 0x7d, 0xd0, 0xdd, 0xcd, 0x38, 0xc2, 0x1c, 0x35, 0x21, 0x1f, 0x84, 0x7e, - 0x5f, 0x5e, 0x77, 0xae, 0x4d, 0x7e, 0x53, 0xbd, 0x1f, 0xec, 0x66, 0x1c, 0x6e, 0x8b, 0xde, 0x82, - 0x52, 0xc4, 0x2f, 0x16, 0x11, 0x7f, 0x28, 0x62, 0xa7, 0xca, 0x09, 0x98, 0x02, 0x49, 0x4c, 0xd1, - 0x5b, 0x50, 0x3c, 0xe6, 0xc7, 0x46, 0xf9, 0xfa, 0xb8, 0xa1, 0x82, 0xf4, 0x03, 0x25, 0x5b, 0x97, - 0xb0, 0x45, 0xf7, 0x60, 0x89, 0xfa, 0xc1, 0x51, 0x72, 0x3a, 0x93, 0x8f, 0x4c, 0x55, 0x15, 0x3b, - 0xeb, 0xf4, 0xb6, 0x9b, 0x71, 0x34, 0x1c, 0x7a, 0x04, 0xab, 0x8f, 0xb5, 0x63, 0x00, 0x49, 0x9e, - 0x13, 0x35, 0x9e, 0x67, 0x1f, 0x50, 0x76, 0x33, 0xce, 0x14, 0x1a, 0xed, 0xc0, 0x4a, 0xa4, 0xed, - 0x70, 0xf2, 0xe9, 0x5c, 0x5b, 0x97, 0xbe, 0x07, 0xee, 0x66, 0x9c, 0x09, 0x0c, 0x7a, 0x08, 0x2b, - 0x6d, 0x2d, 0xbf, 0xcb, 0x87, 0x71, 0x6d, 0x56, 0xb3, 0x77, 0x00, 0x36, 0x9a, 0x8e, 0x45, 0x1f, - 0xc0, 0x6a, 0x30, 0x91, 0xdb, 0xe4, 0xcb, 0xf8, 0xd7, 0xf5, 0x55, 0xce, 0x48, 0x82, 0x6c, 0x91, - 0x93, 0x60, 0x75, 0x7a, 0x22, 0xc4, 0xcd, 0xe5, 0xb3, 0xa7, 0xa7, 0x27, 0x01, 0x75, 0x7a, 0xa2, - 0xc7, 0x86, 0x71, 0x3a, 0xaa, 0x7d, 0x5c, 0x84, 0x25, 0x19, 0x66, 0xe2, 0x35, 0xec, 0xbb, 0x69, - 0xe4, 0x88, 0x28, 0x7b, 0xed, 0xac, 0xc8, 0xe1, 0xe6, 0x4a, 0xe0, 0xbc, 0x99, 0x06, 0x8e, 0x08, - 0xb9, 0xf5, 0x71, 0x8a, 0xe3, 0xdf, 0x55, 0x10, 0x32, 0x58, 0xb6, 0x92, 0x60, 0x11, 0x91, 0xf6, - 0xea, 0xec, 0x3b, 0x65, 0x82, 0x92, 0x91, 0xb2, 0x0d, 0x25, 0x57, 0xfc, 0x44, 0x30, 0x2b, 0xc6, - 0xa6, 0x7f, 0x41, 0x60, 0xda, 0x97, 0x00, 0xb4, 0x35, 0x8e, 0x98, 0x82, 0x7c, 0x12, 0x9f, 0x8a, - 0x98, 0x14, 0x94, 0x04, 0xcc, 0xf5, 0x34, 0x60, 0x8a, 0x93, 0xcf, 0xe8, 0x49, 0xb8, 0xa4, 0x0b, - 0x93, 0xd1, 0x72, 0x17, 0x96, 0x13, 0x7d, 0xf1, 0x2e, 0x19, 0x2e, 0xaf, 0x9d, 0x75, 0xac, 0x4b, - 0xf0, 0x3a, 0x0a, 0xdd, 0x9f, 0x12, 0x65, 0x79, 0x72, 0x2b, 0x9e, 0x94, 0x64, 0x32, 0xd2, 0xa4, - 0x22, 0x1f, 0xc0, 0x95, 0xb1, 0xa8, 0xc4, 0x9c, 0x60, 0xfa, 0x84, 0xaf, 0xc9, 0x31, 0x19, 0x6a, - 0x12, 0xa8, 0x4e, 0x4b, 0x8a, 0x71, 0xf1, 0xac, 0x69, 0x25, 0x52, 0x9c, 0x9a, 0x96, 0xe8, 0x40, - 0xbb, 0xb0, 0xd0, 0x27, 0x14, 0xb7, 0x31, 0xc5, 0x66, 0x89, 0x6f, 0x4b, 0xaf, 0x4f, 0x05, 0x88, - 0x44, 0xd7, 0xdf, 0x97, 0x86, 0x77, 0x3d, 0x1a, 0x9e, 0xc8, 0xb7, 0x8b, 0x14, 0xbd, 0xf1, 0x3d, - 0x58, 0xd6, 0x0c, 0xd0, 0x2a, 0xe4, 0x8e, 0x48, 0xf2, 0xb3, 0x11, 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, 0xeb, 0x51, 0xc5, 0xf8, 0x74, 0x54, 0x31, 0x9e, 0x8d, 0x2a, 0xc6, 0x9f, - 0x47, 0x15, 0xe3, 0x6f, 0xa3, 0x4a, 0xe6, 0x8b, 0x51, 0xc5, 0xf8, 0xe4, 0x45, 0x25, 0xf3, 0xec, - 0x45, 0x25, 0xf3, 0xf9, 0x8b, 0x4a, 0xe6, 0x27, 0xd7, 0xcf, 0xbd, 0x43, 0x1e, 0x14, 0x39, 0x53, - 0x5b, 0xff, 0x0c, 0x00, 0x00, 0xff, 0xff, 0xd4, 0x35, 0x32, 0x52, 0x3f, 0x1f, 0x00, 0x00, + // 2004 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x6f, 0x1c, 0x49, + 0xf5, 0x9f, 0x9e, 0x9f, 0x9e, 0x67, 0x7b, 0xe2, 0x6f, 0xc5, 0x5f, 0x6f, 0xe3, 0xcd, 0x4e, 0x9b, + 0x91, 0xd8, 0x35, 0x08, 0x66, 0x36, 0xf6, 0x6e, 0xd8, 0x35, 0x21, 0xda, 0x74, 0x9c, 0xac, 0x13, + 0xb2, 0x6c, 0xb6, 0x6d, 0x71, 0xe0, 0x82, 0xca, 0x33, 0xe5, 0x99, 0xc6, 0x33, 0xdd, 0x9d, 0xee, + 0x1a, 0x27, 0x96, 0x10, 0xda, 0x7f, 0x00, 0xb1, 0x7f, 0x05, 0xe2, 0x86, 0x84, 0x38, 0x71, 0xe2, + 0x84, 0x76, 0x0f, 0x48, 0x39, 0xa1, 0x95, 0x25, 0x06, 0xe2, 0x48, 0x08, 0xf9, 0xb4, 0x12, 0x57, + 0x0e, 0xa8, 0x7e, 0x74, 0x4f, 0xd5, 0x74, 0x1b, 0xcf, 0x04, 0x84, 0x64, 0xe0, 0x32, 0xd3, 0x55, + 0xf5, 0x3e, 0xd5, 0x55, 0x9f, 0xf7, 0x79, 0xf5, 0xaa, 0xaa, 0xe1, 0x8d, 0xe0, 0xb0, 0xdb, 0x7a, + 0x3c, 0x24, 0xa1, 0x4b, 0x42, 0xfe, 0x7f, 0x1c, 0x62, 0xaf, 0x4b, 0x94, 0xc7, 0x66, 0x10, 0xfa, + 0xd4, 0x47, 0x30, 0xae, 0x59, 0xdd, 0xe8, 0xba, 0xb4, 0x37, 0xdc, 0x6f, 0xb6, 0xfd, 0x41, 0xab, + 0xeb, 0x77, 0xfd, 0x56, 0xd7, 0xf7, 0xbb, 0x7d, 0x82, 0x03, 0x37, 0x92, 0x8f, 0xad, 0x30, 0x68, + 0xb7, 0x22, 0x8a, 0xe9, 0x30, 0x12, 0xf8, 0xd5, 0x65, 0x66, 0xc8, 0x1f, 0x39, 0x44, 0xd6, 0x5a, + 0xd2, 0x9c, 0x97, 0xf6, 0x87, 0x07, 0x2d, 0xea, 0x0e, 0x48, 0x44, 0xf1, 0x20, 0x88, 0x0d, 0xd8, + 0xf8, 0xfa, 0x7e, 0x57, 0x20, 0x5d, 0xaf, 0x43, 0x9e, 0x76, 0x31, 0x25, 0x4f, 0xf0, 0xb1, 0x34, + 0x78, 0x55, 0x33, 0x88, 0x1f, 0x64, 0xe3, 0xaa, 0xd6, 0x18, 0x60, 0x4a, 0x49, 0xe8, 0xc9, 0xb6, + 0x2f, 0x69, 0x6d, 0xd1, 0x21, 0xa1, 0xed, 0x9e, 0x6c, 0x5a, 0x93, 0x4d, 0x8f, 0xfb, 0x03, 0xbf, + 0x43, 0xfa, 0x7c, 0x22, 0x91, 0xf8, 0x95, 0x16, 0x57, 0x99, 0x45, 0x30, 0x8c, 0x7a, 0xfc, 0x47, + 0x56, 0xde, 0xb9, 0x90, 0xcb, 0x7d, 0x1c, 0x91, 0x56, 0x87, 0x1c, 0xb8, 0x9e, 0x4b, 0x5d, 0xdf, + 0x8b, 0xd4, 0x67, 0xd9, 0xc9, 0x8d, 0xe9, 0x3a, 0x99, 0xf4, 0xcf, 0xea, 0x9b, 0x0c, 0x17, 0x51, + 0x3f, 0xc4, 0x5d, 0xd2, 0x6a, 0xf7, 0x86, 0xde, 0x61, 0xab, 0x8d, 0xdb, 0x3d, 0xd2, 0x0a, 0x49, + 0x34, 0xec, 0xd3, 0x48, 0x14, 0xe8, 0x71, 0x40, 0xe4, 0x9b, 0x1a, 0x9f, 0x15, 0x61, 0xfe, 0xa1, + 0x7f, 0xe8, 0x3a, 0xe4, 0xf1, 0x90, 0x44, 0x14, 0x2d, 0x43, 0x89, 0xf7, 0x6a, 0x1a, 0x6b, 0xc6, + 0x7a, 0xd5, 0x11, 0x05, 0x56, 0xdb, 0x77, 0x07, 0x2e, 0x35, 0xf3, 0x6b, 0xc6, 0xfa, 0xa2, 0x23, + 0x0a, 0x08, 0x41, 0x31, 0xa2, 0x24, 0x30, 0x0b, 0x6b, 0xc6, 0x7a, 0xc1, 0xe1, 0xcf, 0x68, 0x15, + 0xe6, 0x5c, 0x8f, 0x92, 0xf0, 0x08, 0xf7, 0xcd, 0x2a, 0xaf, 0x4f, 0xca, 0xe8, 0x16, 0x54, 0x22, + 0x8a, 0x43, 0xba, 0x17, 0x99, 0xc5, 0x35, 0x63, 0x7d, 0x7e, 0x63, 0xb5, 0x29, 0x3c, 0xdf, 0x8c, + 0x3d, 0xdf, 0xdc, 0x8b, 0x3d, 0x6f, 0xcf, 0x7d, 0x3a, 0xb2, 0x72, 0x9f, 0xfc, 0xd1, 0x32, 0x9c, + 0x18, 0x84, 0xb6, 0xa0, 0x44, 0xbc, 0xce, 0x5e, 0x64, 0x96, 0x66, 0x40, 0x0b, 0x08, 0xba, 0x0e, + 0xd5, 0x8e, 0x1b, 0x92, 0x36, 0x63, 0xd9, 0x2c, 0xaf, 0x19, 0xeb, 0xb5, 0x8d, 0xab, 0xcd, 0x44, + 0x28, 0xdb, 0x71, 0x93, 0x33, 0xb6, 0x62, 0xd3, 0x0b, 0x30, 0xed, 0x99, 0x15, 0xce, 0x04, 0x7f, + 0x46, 0x0d, 0x28, 0x47, 0x3d, 0x1c, 0x76, 0x22, 0x73, 0x6e, 0xad, 0xb0, 0x5e, 0xb5, 0xe1, 0x6c, + 0x64, 0xc9, 0x1a, 0x47, 0xfe, 0xa3, 0x1f, 0x40, 0x31, 0xe8, 0x63, 0xcf, 0x04, 0x3e, 0xca, 0xa5, + 0xa6, 0xe2, 0xa5, 0x47, 0x7d, 0xec, 0xd9, 0xef, 0x9e, 0x8c, 0xac, 0xb7, 0xd5, 0xe0, 0x09, 0xf1, + 0x01, 0xf6, 0x70, 0xab, 0xef, 0x1f, 0xba, 0xad, 0xa3, 0xcd, 0x96, 0xea, 0x7b, 0xd6, 0x51, 0xf3, + 0x23, 0xd6, 0x01, 0x83, 0x3a, 0xbc, 0x63, 0xf4, 0x00, 0xe6, 0x99, 0x8f, 0xc9, 0x1d, 0xe6, 0xe0, + 0xc8, 0x9c, 0xe7, 0xef, 0x79, 0x65, 0x3c, 0x1b, 0x5e, 0xef, 0x90, 0x83, 0xf7, 0x43, 0x7f, 0x18, + 0xd8, 0x57, 0xce, 0x46, 0x96, 0x6a, 0xef, 0xa8, 0x05, 0xf4, 0x00, 0x6a, 0x4c, 0x14, 0xae, 0xd7, + 0xfd, 0x30, 0xe0, 0x0a, 0x34, 0x17, 0x78, 0x77, 0xd7, 0x9a, 0xaa, 0x64, 0x9a, 0x77, 0x34, 0x1b, + 0xbb, 0xc8, 0xe8, 0x75, 0x26, 0x90, 0x8d, 0xd3, 0x02, 0x20, 0xa6, 0xa5, 0xfb, 0x5e, 0x44, 0xb1, + 0x47, 0x5f, 0x46, 0x52, 0x37, 0xa1, 0xcc, 0x82, 0x7f, 0x2f, 0xe2, 0xa2, 0x9a, 0xd6, 0xc7, 0x12, + 0xa3, 0x3b, 0xb9, 0x38, 0x93, 0x93, 0x4b, 0x99, 0x4e, 0x2e, 0x5f, 0xe8, 0xe4, 0xca, 0xbf, 0xc9, + 0xc9, 0x73, 0xff, 0x5a, 0x27, 0x57, 0x5f, 0xda, 0xc9, 0x26, 0x14, 0xd9, 0x28, 0xd1, 0x12, 0x14, + 0x42, 0xfc, 0x84, 0xfb, 0x74, 0xc1, 0x61, 0x8f, 0x8d, 0xd3, 0x22, 0x2c, 0x88, 0xa5, 0x24, 0x0a, + 0x7c, 0x2f, 0x22, 0x8c, 0xc7, 0x5d, 0xbe, 0xfa, 0x0b, 0xcf, 0x4b, 0x1e, 0x79, 0x8d, 0x23, 0x5b, + 0xd0, 0x7b, 0x50, 0xdc, 0xc6, 0x14, 0x73, 0x15, 0xcc, 0x6f, 0x2c, 0xab, 0x3c, 0xb2, 0xbe, 0x58, + 0x9b, 0xbd, 0xc2, 0x06, 0x72, 0x36, 0xb2, 0x6a, 0x1d, 0x4c, 0xf1, 0xd7, 0xfd, 0x81, 0x4b, 0xc9, + 0x20, 0xa0, 0xc7, 0x0e, 0x47, 0xa2, 0xb7, 0xa1, 0x7a, 0x37, 0x0c, 0xfd, 0x70, 0xef, 0x38, 0x20, + 0x5c, 0x35, 0x55, 0xfb, 0x95, 0xb3, 0x91, 0x75, 0x95, 0xc4, 0x95, 0x0a, 0x62, 0x6c, 0x89, 0xbe, + 0x0a, 0x25, 0x5e, 0xe0, 0x3a, 0xa9, 0xda, 0x57, 0xcf, 0x46, 0xd6, 0x15, 0x0e, 0x51, 0xcc, 0x85, + 0x85, 0x2e, 0xab, 0xd2, 0x54, 0xb2, 0x4a, 0xd4, 0x5d, 0x56, 0xd5, 0x6d, 0x42, 0xe5, 0x88, 0x84, + 0x11, 0xeb, 0xa6, 0xc2, 0xeb, 0xe3, 0x22, 0xba, 0x0d, 0xc0, 0x88, 0x71, 0x23, 0xea, 0xb6, 0x63, + 0x67, 0x2f, 0x36, 0x45, 0xb2, 0x71, 0xb8, 0x8f, 0x6c, 0x24, 0x59, 0x50, 0x0c, 0x1d, 0xe5, 0x19, + 0xfd, 0xc2, 0x80, 0xca, 0x0e, 0xc1, 0x1d, 0x12, 0x32, 0xf7, 0x16, 0xd6, 0xe7, 0x37, 0xbe, 0xd2, + 0x54, 0x33, 0xcb, 0xa3, 0xd0, 0x1f, 0x10, 0xda, 0x23, 0xc3, 0x28, 0x76, 0x90, 0xb0, 0xb6, 0xbd, + 0x93, 0x91, 0x45, 0xa6, 0x94, 0xea, 0x54, 0x09, 0xed, 0xdc, 0x57, 0x9d, 0x8d, 0x2c, 0xe3, 0x1b, + 0x4e, 0x3c, 0x4a, 0xb4, 0x01, 0x73, 0x4f, 0x70, 0xe8, 0xb9, 0x5e, 0x37, 0x32, 0x81, 0x47, 0xda, + 0xca, 0xd9, 0xc8, 0x42, 0x71, 0x9d, 0xe2, 0x88, 0xc4, 0xae, 0xf1, 0x07, 0x03, 0xfe, 0x8f, 0x09, + 0x63, 0x97, 0x8d, 0x27, 0x52, 0x96, 0x98, 0x01, 0xa6, 0xed, 0x9e, 0x69, 0xb0, 0x6e, 0x1c, 0x51, + 0x50, 0xf3, 0x4d, 0xfe, 0x9f, 0xca, 0x37, 0x85, 0xd9, 0xf3, 0x4d, 0xbc, 0xae, 0x14, 0x33, 0xd7, + 0x95, 0xd2, 0x79, 0xeb, 0x4a, 0xe3, 0xa7, 0x72, 0x0d, 0x8d, 0xe7, 0x37, 0x43, 0x28, 0xdd, 0x4b, + 0x42, 0xa9, 0xc0, 0x47, 0x9b, 0x28, 0x54, 0xf4, 0x75, 0xbf, 0x43, 0x3c, 0xea, 0x1e, 0xb8, 0x24, + 0xbc, 0x20, 0xa0, 0x14, 0x95, 0x16, 0x74, 0x95, 0xaa, 0x12, 0x2b, 0x5e, 0x0a, 0x89, 0xe9, 0x71, + 0x55, 0x7a, 0x89, 0xb8, 0x6a, 0xfc, 0x35, 0x0f, 0x2b, 0xcc, 0x23, 0x0f, 0xf1, 0x3e, 0xe9, 0x7f, + 0x17, 0x0f, 0x66, 0xf4, 0xca, 0xeb, 0x8a, 0x57, 0xaa, 0x36, 0xfa, 0x1f, 0xeb, 0xd3, 0xb1, 0xfe, + 0x33, 0x03, 0xe6, 0xe2, 0x04, 0x80, 0x9a, 0x00, 0x02, 0xc6, 0xd7, 0x78, 0xc1, 0x75, 0x8d, 0x81, + 0xc3, 0xa4, 0xd6, 0x51, 0x2c, 0xd0, 0x0f, 0xa1, 0x2c, 0x4a, 0x32, 0x16, 0x94, 0xb4, 0xb9, 0x4b, + 0x43, 0x82, 0x07, 0xb7, 0x3b, 0x38, 0xa0, 0x24, 0xb4, 0xdf, 0x65, 0xa3, 0x38, 0x19, 0x59, 0x6f, + 0x9c, 0xc7, 0x52, 0xbc, 0xc3, 0x97, 0x38, 0xe6, 0x5f, 0xf1, 0x4e, 0x47, 0xbe, 0xa1, 0xf1, 0x13, + 0x03, 0x96, 0xd8, 0x40, 0x19, 0x35, 0x89, 0x30, 0xb6, 0x61, 0x2e, 0x94, 0xcf, 0x7c, 0xb8, 0xf3, + 0x1b, 0x8d, 0xa6, 0x4e, 0x6b, 0x06, 0x95, 0x3c, 0xe1, 0x1a, 0x4e, 0x82, 0x44, 0x9b, 0x1a, 0x8d, + 0xf9, 0x2c, 0x1a, 0x45, 0x8e, 0x56, 0x89, 0xfb, 0x4d, 0x1e, 0xd0, 0x7d, 0x76, 0x42, 0x62, 0xfa, + 0x1b, 0x4b, 0xf5, 0x69, 0x6a, 0x44, 0xd7, 0xc6, 0xa4, 0xa4, 0xed, 0xed, 0x5b, 0x27, 0x23, 0x6b, + 0xeb, 0x02, 0xed, 0xfc, 0x03, 0xbc, 0x32, 0x0b, 0x55, 0xbe, 0xf9, 0xcb, 0x20, 0xdf, 0xc6, 0xaf, + 0xf2, 0x50, 0xfb, 0x9e, 0xdf, 0x1f, 0x0e, 0x48, 0x42, 0x5f, 0x90, 0xa2, 0xcf, 0x1c, 0xd3, 0xa7, + 0xdb, 0xda, 0x5b, 0x27, 0x23, 0xeb, 0xc6, 0xb4, 0xd4, 0xe9, 0xd8, 0x4b, 0x4d, 0xdb, 0x9f, 0xf3, + 0xb0, 0xbc, 0xe7, 0x07, 0xdf, 0xd9, 0xe5, 0xa7, 0x68, 0x65, 0x99, 0xec, 0xa5, 0xc8, 0x5b, 0x1e, + 0x93, 0xc7, 0x10, 0x1f, 0x60, 0x1a, 0xba, 0x4f, 0xed, 0x1b, 0x27, 0x23, 0x6b, 0x63, 0x5a, 0xe2, + 0xc6, 0xb8, 0xcb, 0x4c, 0x9a, 0xb6, 0x07, 0x2a, 0x4c, 0xb9, 0x07, 0xfa, 0x5b, 0x1e, 0x56, 0x3e, + 0x1a, 0x62, 0x8f, 0xba, 0x7d, 0x22, 0xc8, 0x4e, 0xa8, 0xfe, 0x51, 0x8a, 0xea, 0xfa, 0x98, 0x6a, + 0x1d, 0x23, 0x49, 0x7f, 0xef, 0x64, 0x64, 0xdd, 0x9c, 0x96, 0xf4, 0xac, 0x1e, 0xfe, 0xeb, 0xe8, + 0xff, 0x75, 0x1e, 0x6a, 0xbb, 0x62, 0xd7, 0x16, 0x4f, 0xfc, 0x28, 0x83, 0x76, 0xf5, 0x9a, 0x2a, + 0xd8, 0x6f, 0xea, 0x88, 0xd9, 0x16, 0x09, 0x1d, 0x7b, 0xa9, 0x17, 0x89, 0xdf, 0xe5, 0x61, 0x65, + 0x9b, 0x50, 0xd2, 0xa6, 0xa4, 0x73, 0xcf, 0x25, 0x7d, 0x85, 0xc4, 0x8f, 0x8d, 0x14, 0x8b, 0x6b, + 0xca, 0x31, 0x2b, 0x13, 0x64, 0xdb, 0x27, 0x23, 0xeb, 0xd6, 0xb4, 0x3c, 0x66, 0xf7, 0x71, 0xa9, + 0xf9, 0xfc, 0x2c, 0x0f, 0xff, 0x2f, 0xae, 0x0e, 0xc4, 0xbd, 0xe6, 0x98, 0xce, 0x1f, 0xa7, 0xd8, + 0xb4, 0xd4, 0xa5, 0x20, 0x03, 0x62, 0xdf, 0x3e, 0x19, 0x59, 0xdf, 0x9e, 0x7e, 0x2d, 0xc8, 0xe8, + 0xe2, 0x3f, 0x46, 0x9b, 0x7c, 0xb7, 0x3f, 0xab, 0x36, 0x75, 0xd0, 0xcb, 0x69, 0x53, 0xef, 0xe3, + 0x52, 0xf3, 0xf9, 0xdb, 0x3c, 0x2c, 0x73, 0x95, 0xec, 0xe2, 0x41, 0xd0, 0x57, 0x36, 0x04, 0x17, + 0x64, 0xa9, 0x34, 0x62, 0xd6, 0x2c, 0x95, 0xee, 0xe1, 0x52, 0x13, 0xf9, 0xcb, 0x0a, 0x2c, 0xf2, + 0x49, 0x25, 0x0c, 0x7e, 0x0d, 0xe4, 0x39, 0x53, 0xf2, 0x87, 0xe2, 0xbb, 0x89, 0x30, 0x68, 0x37, + 0x77, 0xe5, 0x09, 0x54, 0x58, 0xa0, 0x77, 0xa0, 0x1c, 0xf1, 0x1b, 0x00, 0x79, 0x84, 0xa8, 0x4f, + 0x5e, 0xb2, 0xe9, 0x77, 0x0d, 0x3b, 0x39, 0x47, 0xda, 0xa3, 0x9b, 0x50, 0xee, 0x73, 0x39, 0xca, + 0x1b, 0x90, 0xc6, 0x24, 0x32, 0x7d, 0x26, 0x66, 0x68, 0x81, 0x41, 0x37, 0xa0, 0xc4, 0xcf, 0x2a, + 0xf2, 0xb2, 0x5f, 0x7b, 0x6d, 0xfa, 0xc4, 0xb0, 0x93, 0x73, 0x84, 0x39, 0xda, 0x80, 0x62, 0x10, + 0xfa, 0x03, 0x79, 0x6e, 0xbc, 0x36, 0xf9, 0x4e, 0xf5, 0xa0, 0xb5, 0x93, 0x73, 0xb8, 0x2d, 0x7a, + 0x0b, 0x2a, 0x11, 0x3f, 0xa1, 0x45, 0xfc, 0xc6, 0x8d, 0x6d, 0xcf, 0x27, 0x60, 0x0a, 0x24, 0x36, + 0x45, 0x6f, 0x41, 0xf9, 0x88, 0xef, 0xbf, 0xe5, 0x35, 0xee, 0xaa, 0x0a, 0xd2, 0x77, 0xe6, 0x6c, + 0x5e, 0xc2, 0x16, 0xdd, 0x83, 0x05, 0xea, 0x07, 0x87, 0xf1, 0x36, 0x57, 0xde, 0xd6, 0xad, 0xa9, + 0xd8, 0xac, 0x6d, 0xf0, 0x4e, 0xce, 0xd1, 0x70, 0xe8, 0x11, 0x2c, 0x3d, 0xd6, 0xf6, 0x53, 0x24, + 0xbe, 0x97, 0xd5, 0x78, 0xce, 0xde, 0xe9, 0xed, 0xe4, 0x9c, 0x14, 0x1a, 0x6d, 0x43, 0x2d, 0xd2, + 0xb6, 0x0a, 0xf2, 0x1b, 0x84, 0x36, 0x2f, 0x7d, 0x33, 0xb1, 0x93, 0x73, 0x26, 0x30, 0xe8, 0x21, + 0xd4, 0x3a, 0x5a, 0xa2, 0x94, 0x5f, 0x18, 0xb4, 0x51, 0x65, 0xa7, 0x52, 0xd6, 0x9b, 0x8e, 0x45, + 0x1f, 0xc2, 0x52, 0x30, 0x91, 0x24, 0xe4, 0x27, 0x86, 0x2f, 0xeb, 0xb3, 0xcc, 0xc8, 0x26, 0x6c, + 0x92, 0x93, 0x60, 0x75, 0x78, 0x62, 0xad, 0x34, 0x17, 0xcf, 0x1f, 0x9e, 0xbe, 0x9a, 0xaa, 0xc3, + 0x13, 0x2d, 0xe8, 0x21, 0x5c, 0x89, 0xf4, 0x95, 0xc2, 0xac, 0xa5, 0xfd, 0x99, 0xb5, 0xa2, 0xec, + 0xe4, 0x9c, 0x49, 0xa8, 0x0d, 0xe3, 0x85, 0xad, 0xf1, 0xfb, 0x32, 0x2c, 0xc8, 0xa0, 0x15, 0x97, + 0x94, 0xdf, 0x4c, 0xe2, 0x50, 0xc4, 0xec, 0x6b, 0xe7, 0xc5, 0x21, 0x37, 0x57, 0xc2, 0xf0, 0xcd, + 0x24, 0x0c, 0x45, 0x00, 0xaf, 0x8c, 0x17, 0x4b, 0x3e, 0x0b, 0x05, 0x21, 0x43, 0x6f, 0x33, 0x0e, + 0x3d, 0x11, 0xb7, 0xaf, 0x66, 0x1f, 0xf5, 0x63, 0x94, 0x8c, 0xbb, 0x2d, 0xa8, 0xb8, 0xe2, 0xcb, + 0x4d, 0x56, 0xc4, 0xa6, 0x3f, 0xec, 0xb0, 0x48, 0x92, 0x00, 0xb4, 0x39, 0x8e, 0xbf, 0x92, 0xfc, + 0x52, 0x91, 0x8a, 0xbf, 0x04, 0x14, 0x87, 0xdf, 0xf5, 0x24, 0xfc, 0xca, 0x93, 0x5f, 0x37, 0xe2, + 0xe0, 0x4b, 0x26, 0x26, 0x63, 0xef, 0x2e, 0x2c, 0xc6, 0x6a, 0xe5, 0x4d, 0x32, 0xf8, 0x5e, 0x3b, + 0x6f, 0xb7, 0x1d, 0xe3, 0x75, 0x14, 0xba, 0x9f, 0x92, 0x78, 0x75, 0x72, 0x87, 0x34, 0x29, 0xf0, + 0xb8, 0xa7, 0x49, 0x7d, 0x3f, 0x80, 0x2b, 0x63, 0x89, 0x8a, 0x31, 0x41, 0x66, 0x4a, 0x7b, 0xa4, + 0x5b, 0x31, 0xf9, 0x4c, 0x00, 0xd5, 0x61, 0x49, 0x69, 0xcf, 0x9f, 0x37, 0xac, 0x58, 0xd8, 0xa9, + 0x61, 0x49, 0x5d, 0xbf, 0x0f, 0xb5, 0x44, 0x9c, 0x62, 0x54, 0x0b, 0x92, 0xa9, 0x73, 0xd2, 0x64, + 0xd2, 0x91, 0x0e, 0x43, 0x3b, 0x30, 0x37, 0x20, 0x14, 0x77, 0x30, 0xc5, 0x66, 0x85, 0x67, 0xcb, + 0xd7, 0x53, 0x91, 0x21, 0x6d, 0x9b, 0x1f, 0x48, 0xc3, 0xbb, 0x1e, 0x0d, 0x8f, 0xe5, 0xdd, 0x54, + 0x82, 0x5e, 0xfd, 0x16, 0x2c, 0x6a, 0x06, 0x68, 0x09, 0x0a, 0x87, 0x24, 0xfe, 0x2c, 0xc8, 0x1e, + 0xd1, 0x32, 0x94, 0x8e, 0x70, 0x7f, 0x48, 0xb8, 0xd0, 0xab, 0x8e, 0x28, 0x6c, 0xe5, 0xdf, 0x31, + 0xec, 0x2a, 0x54, 0x42, 0xf1, 0x16, 0xbb, 0xfb, 0xec, 0x79, 0x3d, 0xf7, 0xf9, 0xf3, 0x7a, 0xee, + 0x8b, 0xe7, 0x75, 0xe3, 0xe3, 0xd3, 0xba, 0xf1, 0xf3, 0xd3, 0xba, 0xf1, 0xe9, 0x69, 0xdd, 0x78, + 0x76, 0x5a, 0x37, 0xfe, 0x74, 0x5a, 0x37, 0xfe, 0x72, 0x5a, 0xcf, 0x7d, 0x71, 0x5a, 0x37, 0x3e, + 0x79, 0x51, 0xcf, 0x3d, 0x7b, 0x51, 0xcf, 0x7d, 0xfe, 0xa2, 0x9e, 0xfb, 0xfe, 0xf5, 0x99, 0x13, + 0xf7, 0x7e, 0x99, 0xf3, 0xb4, 0xf9, 0xf7, 0x00, 0x00, 0x00, 0xff, 0xff, 0x42, 0x9e, 0x81, 0xb1, + 0x1f, 0x21, 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -2321,6 +2390,42 @@ func (this *DetectedLabelsResponse) Equal(that interface{}) bool { } return true } +func (this *QuerySamplesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QuerySamplesResponse) + if !ok { + that2, ok := that.(QuerySamplesResponse) + 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 + } + } + return true +} func (this *QueryResponse) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2642,6 +2747,30 @@ func (this *QueryResponse_DetectedLabels) Equal(that interface{}) bool { } return true } +func (this *QueryResponse_SamplesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryResponse_SamplesResponse) + if !ok { + that2, ok := that.(QueryResponse_SamplesResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.SamplesResponse.Equal(that1.SamplesResponse) { + return false + } + return true +} func (this *QueryRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2920,6 +3049,30 @@ func (this *QueryRequest_DetectedLabels) Equal(that interface{}) bool { } return true } +func (this *QueryRequest_SamplesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryRequest_SamplesRequest) + if !ok { + that2, ok := that.(QueryRequest_SamplesRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.SamplesRequest.Equal(that1.SamplesRequest) { + return false + } + return true +} func (this *LokiRequest) GoString() string { if this == nil { return "nil" @@ -3154,11 +3307,22 @@ func (this *DetectedLabelsResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *QuerySamplesResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&queryrange.QuerySamplesResponse{") + s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n") + s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} 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") @@ -3265,11 +3429,19 @@ func (this *QueryResponse_DetectedLabels) GoString() string { `DetectedLabels:` + fmt.Sprintf("%#v", this.DetectedLabels) + `}`}, ", ") return s } +func (this *QueryResponse_SamplesResponse) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&queryrange.QueryResponse_SamplesResponse{` + + `SamplesResponse:` + fmt.Sprintf("%#v", this.SamplesResponse) + `}`}, ", ") + return s +} func (this *QueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 15) + s := make([]string, 0, 16) s = append(s, "&queryrange.QueryRequest{") if this.Request != nil { s = append(s, "Request: "+fmt.Sprintf("%#v", this.Request)+",\n") @@ -3370,6 +3542,14 @@ func (this *QueryRequest_DetectedLabels) GoString() string { `DetectedLabels:` + fmt.Sprintf("%#v", this.DetectedLabels) + `}`}, ", ") return s } +func (this *QueryRequest_SamplesRequest) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&queryrange.QueryRequest_SamplesRequest{` + + `SamplesRequest:` + fmt.Sprintf("%#v", this.SamplesRequest) + `}`}, ", ") + return s +} func valueToGoStringQueryrange(v interface{}, typ string) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -4428,6 +4608,55 @@ func (m *DetectedLabelsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *QuerySamplesResponse) 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 *QuerySamplesResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QuerySamplesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + 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 *QueryResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4712,6 +4941,26 @@ func (m *QueryResponse_DetectedLabels) MarshalToSizedBuffer(dAtA []byte) (int, e } return len(dAtA) - i, nil } +func (m *QueryResponse_SamplesResponse) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *QueryResponse_SamplesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.SamplesResponse != nil { + { + size, err := m.SamplesResponse.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) @@ -4963,6 +5212,26 @@ func (m *QueryRequest_DetectedLabels) MarshalToSizedBuffer(dAtA []byte) (int, er } return len(dAtA) - i, nil } +func (m *QueryRequest_SamplesRequest) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *QueryRequest_SamplesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.SamplesRequest != nil { + { + size, err := m.SamplesRequest.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + return len(dAtA) - i, nil +} func encodeVarintQueryrange(dAtA []byte, offset int, v uint64) int { offset -= sovQueryrange(v) base := offset @@ -5410,6 +5679,25 @@ func (m *DetectedLabelsResponse) Size() (n int) { return n } +func (m *QuerySamplesResponse) 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)) + } + } + return n +} + func (m *QueryResponse) Size() (n int) { if m == nil { return 0 @@ -5570,6 +5858,18 @@ func (m *QueryResponse_DetectedLabels) Size() (n int) { } return n } +func (m *QueryResponse_SamplesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SamplesResponse != nil { + l = m.SamplesResponse.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} func (m *QueryRequest) Size() (n int) { if m == nil { return 0 @@ -5710,6 +6010,18 @@ func (m *QueryRequest_DetectedLabels) Size() (n int) { } return n } +func (m *QueryRequest_SamplesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SamplesRequest != nil { + l = m.SamplesRequest.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} func sovQueryrange(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 @@ -5944,6 +6256,17 @@ func (this *DetectedLabelsResponse) String() string { }, "") return s } +func (this *QuerySamplesResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QuerySamplesResponse{`, + `Response:` + fmt.Sprintf("%v", this.Response) + `,`, + `Headers:` + fmt.Sprintf("%v", this.Headers) + `,`, + `}`, + }, "") + return s +} func (this *QueryResponse) String() string { if this == nil { return "nil" @@ -6075,6 +6398,16 @@ func (this *QueryResponse_DetectedLabels) String() string { }, "") return s } +func (this *QueryResponse_SamplesResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryResponse_SamplesResponse{`, + `SamplesResponse:` + strings.Replace(fmt.Sprintf("%v", this.SamplesResponse), "QuerySamplesResponse", "QuerySamplesResponse", 1) + `,`, + `}`, + }, "") + return s +} func (this *QueryRequest) String() string { if this == nil { return "nil" @@ -6196,6 +6529,16 @@ func (this *QueryRequest_DetectedLabels) String() string { }, "") return s } +func (this *QueryRequest_SamplesRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryRequest_SamplesRequest{`, + `SamplesRequest:` + strings.Replace(fmt.Sprintf("%v", this.SamplesRequest), "QuerySamplesRequest", "logproto.QuerySamplesRequest", 1) + `,`, + `}`, + }, "") + return s +} func valueToStringQueryrange(v interface{}) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -9262,6 +9605,129 @@ func (m *DetectedLabelsResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *QuerySamplesResponse) 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: QuerySamplesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QuerySamplesResponse: 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.QuerySamplesResponse{} + } + 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 + 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 *QueryResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -9747,6 +10213,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 SamplesResponse", 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 := &QuerySamplesResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Response = &QueryResponse_SamplesResponse{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) @@ -10277,6 +10778,41 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error { } m.Request = &QueryRequest_DetectedLabels{v} iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SamplesRequest", 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 := &logproto.QuerySamplesRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Request = &QueryRequest_SamplesRequest{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) diff --git a/pkg/querier/queryrange/queryrange.proto b/pkg/querier/queryrange/queryrange.proto index 2169b2da3154..bc4083bc9dc5 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -210,6 +210,13 @@ message DetectedLabelsResponse { (gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" ]; } +message QuerySamplesResponse { + logproto.QuerySamplesResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/logproto.QuerySamplesResponse"]; + repeated definitions.PrometheusResponseHeader Headers = 2 [ + (gogoproto.jsontag) = "-", + (gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" + ]; +} message QueryResponse { google.rpc.Status status = 1; @@ -226,6 +233,7 @@ message QueryResponse { DetectedFieldsResponse detectedFields = 11; QueryPatternsResponse patternsResponse = 12; DetectedLabelsResponse detectedLabels = 13; + QuerySamplesResponse samplesResponse = 14; } } @@ -241,6 +249,7 @@ message QueryRequest { logproto.DetectedFieldsRequest detectedFields = 9; logproto.QueryPatternsRequest patternsRequest = 10; logproto.DetectedLabelsRequest detectedLabels = 11; + logproto.QuerySamplesRequest samplesRequest = 12; } map metadata = 7 [(gogoproto.nullable) = false]; } diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 609c7ede41d0..3c8ffbe9d0e3 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -9,6 +9,7 @@ import ( "github.com/gorilla/websocket" jsoniter "github.com/json-iterator/go" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/grafana/loki/v3/pkg/loghttp" @@ -46,6 +47,8 @@ func WriteResponseJSON(r *http.Request, v any, w http.ResponseWriter) error { return WriteVolumeResponseJSON(result, w) case *logproto.QueryPatternsResponse: return WriteQueryPatternsResponseJSON(result, w) + case *logproto.QuerySamplesResponse: + return WriteQuerySamplesResponseJSON(result, w) } return fmt.Errorf("unknown response type %T", v) } @@ -202,14 +205,8 @@ func WriteQueryPatternsResponseJSON(r *logproto.QueryPatternsResponse, w io.Writ if len(r.Series) > 0 { for i, series := range r.Series { s.WriteObjectStart() - if pattern := series.GetPattern(); pattern != "" { - s.WriteObjectField("pattern") - s.WriteStringWithHTMLEscaped(pattern) - } - if labels := series.GetLabels(); labels != "" { - s.WriteObjectField("labels") - s.WriteStringWithHTMLEscaped(labels) - } + s.WriteObjectField("pattern") + s.WriteStringWithHTMLEscaped(series.Pattern) s.WriteMore() s.WriteObjectField("samples") s.WriteArrayStart() @@ -245,3 +242,46 @@ func WriteDetectedLabelsResponseJSON(r *logproto.DetectedLabelsResponse, w io.Wr s.WriteRaw("\n") return s.Flush() } + +// WriteQuerySamplesResponseJSON marshals a logproto.QuerySamplesResponse to JSON and then +// writes it to the provided io.Writer. +func WriteQuerySamplesResponseJSON(r *logproto.QuerySamplesResponse, w io.Writer) error { + s := jsoniter.ConfigFastest.BorrowStream(w) + defer jsoniter.ConfigFastest.ReturnStream(s) + + matrix, err := logprotoSeriesToPromQLMatrix(r.Series) + if err != nil { + return fmt.Errorf("could not convert logproto series to promql matrix: %w", err) + } + + // TODO(twhitney): add stats + err = EncodeResult(matrix, nil, stats.Result{}, s, nil) + if err != nil { + return fmt.Errorf("could not write JSON response: %w", err) + } + + s.WriteRaw("\n") + s.Flush() + return nil +} + +func logprotoSeriesToPromQLMatrix(series []logproto.Series) (promql.Matrix, error) { + promMatrix := make(promql.Matrix, len(series)) + for i, s := range series { + lbls, err := parser.ParseMetric(s.Labels) + if err != nil { + return nil, err + } + + promSeries := promql.Series{ + Metric: lbls, + Floats: make([]promql.FPoint, len(s.Samples)), + } + for i, sample := range s.Samples { + promSeries.Floats[i] = promql.FPoint{T: sample.Timestamp / 1e6, F: sample.Value} + } + promMatrix[i] = promSeries + } + + return promMatrix, nil +} diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index 4688388744c9..b17ead9e1d2f 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -1077,11 +1077,13 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, + { + Pattern: "foo <*> bar", + Samples: []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, + }, }, - ), }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]}]}`, @@ -1089,17 +1091,20 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, + { + Pattern: "foo <*> bar", + Samples: []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, + }, }, - ), - logproto.NewPatternSeriesWithPattern("foo <*> buzz", - []*logproto.PatternSample{ + { + Pattern: "foo <*> buzz", + Samples: []*logproto.PatternSample{ {Timestamp: model.TimeFromUnix(3), Value: 1}, {Timestamp: model.TimeFromUnix(3), Value: 2}, }, - ), + }, }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]},{"pattern":"foo <*> buzz","samples":[[3,1],[3,2]]}]}`, @@ -1107,64 +1112,148 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithPattern("foo <*> bar", - []*logproto.PatternSample{}, - ), - logproto.NewPatternSeriesWithPattern("foo <*> buzz", - []*logproto.PatternSample{}, - ), + { + Pattern: "foo <*> bar", + Samples: []*logproto.PatternSample{}, + }, + { + Pattern: "foo <*> buzz", + Samples: []*logproto.PatternSample{}, + }, }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[]},{"pattern":"foo <*> buzz","samples":[]}]}`, }, + } { + tc := tc + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + var b bytes.Buffer + err := WriteQueryPatternsResponseJSON(tc.input, &b) + require.NoError(t, err) + got := b.String() + require.JSONEqf(t, tc.expected, got, "Patterns Test %d failed", i) + }) + } +} + +//TODO: use real times for clarity +func Test_WriteQuerySamplesResponseJSON(t *testing.T) { + expected := func(result string) string { + return fmt.Sprintf(`{ + "status": "success", + "data": { + "result": %s, + "resultType": "matrix", + "stats" : %s + } + }`, result, emptyStats) + } + + for i, tc := range []struct { + input *logproto.QuerySamplesResponse + expected string + }{ + { + &logproto.QuerySamplesResponse{}, + expected("[]"), + }, { - &logproto.QueryPatternsResponse{ - Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, + &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 1, Value: 1}, + {Timestamp: 2, Value: 2}, + }, }, - ), }, }, - `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]}]}`, + expected(`[ + { + "metric": { + "foo": "bar" + }, + "values": [ + [0.001, "1"], + [0.002, "2"] + ] + } + ]`), }, { - &logproto.QueryPatternsResponse{ - Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, + &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 1, Value: 1}, + {Timestamp: 2, Value: 2}, + }, }, - ), - logproto.NewPatternSeriesWithLabels(`{foo="buzz"}`, - []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(3), Value: 1}, - {Timestamp: model.TimeFromUnix(3), Value: 2}, + { + Labels: `{foo="buzz"}`, + Samples: []logproto.Sample{ + {Timestamp: 3, Value: 1}, + {Timestamp: 3, Value: 2}, }, - ), + }, }, }, - `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]},{"labels":"{foo=\"buzz\"}","samples":[[3,1],[3,2]]}]}`, + expected(`[ + { + "metric": { + "foo": "bar" + }, + "values": [ + [0.001, "1"], + [0.002, "2"] + ] + }, + { + "metric": { + "foo": "buzz" + }, + "values": [ + [0.003, "1"], + [0.003, "2"] + ] + } + ]`), }, { - &logproto.QueryPatternsResponse{ - Series: []*logproto.PatternSeries{ - logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, - []*logproto.PatternSample{}, - ), - logproto.NewPatternSeriesWithPattern(`{foo="buzz"}`, - []*logproto.PatternSample{}, - ), + &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{}, + }, + { + Labels: `{foo="buzz"}`, + Samples: []logproto.Sample{}, + }, }, }, - `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[]},{"pattern":"{foo=\"buzz\"}","samples":[]}]}`, + expected(`[ + { + "metric": { + "foo": "bar" + }, + "values": [] + }, + { + "metric": { + "foo": "buzz" + }, + "values": [] + } + ]`), }, } { tc := tc t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { var b bytes.Buffer - err := WriteQueryPatternsResponseJSON(tc.input, &b) + err := WriteQuerySamplesResponseJSON(tc.input, &b) require.NoError(t, err) got := b.String() require.JSONEqf(t, tc.expected, got, "Patterns Test %d failed", i) From eb84303d1ab10c9a40b4519c5666407e8a60d667 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 13:07:49 -0600 Subject: [PATCH 07/38] chore: a bit of cleanup --- pkg/pattern/ingester_querier.go | 2 +- pkg/pattern/ingester_test.go | 6 ---- pkg/pattern/metric/evaluator.go | 53 -------------------------------- pkg/util/marshal/marshal_test.go | 1 - 4 files changed, 1 insertion(+), 61 deletions(-) diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 60eacb657d5a..4aed20fe8fc2 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -103,7 +103,7 @@ func (q *IngesterQuerier) Samples( return nil, err } - // TODO: what should batch size be here? + // TODO(twhitney): what should batch size be here? resp, err := pattern_iter.ReadMetricsBatch(loki_iter.NewSortSampleIterator(iterators), math.MaxInt32) if err != nil { return nil, err diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index 3eb162114091..c1295de8e9d3 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -144,12 +144,6 @@ func TestInstancePushQuerySamples(t *testing.T) { expr, err = syntax.ParseSampleExpr(`count_over_time({test="test"}[80s])`) require.NoError(t, err) - // it, err := inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ - // Query: expr.String(), - // Start: time.Unix(0, 0), - // End: time.Unix(int64(20*30), 0), - // Step: 10000, - // }) it, err = inst.QuerySample(context.Background(), expr, &logproto.QuerySamplesRequest{ Query: expr.String(), Start: time.Unix(0, 0), diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index aeb3f11c5c3e..96939ffc5187 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -242,59 +242,6 @@ func newRangeVectorIterator( ), nil } -type seriesToSampleIterator struct { - floats []promql.FPoint - curTs int64 - cur float64 - lbls labels.Labels -} - -// TODO: could this me a matrix iterator that returned multiple samples with -// different labels for the same timestamp? -func NewSeriesToSampleIterator(series *promql.Series) *seriesToSampleIterator { - return &seriesToSampleIterator{ - floats: series.Floats, - lbls: series.Metric, - } -} - -func (s *seriesToSampleIterator) Next() bool { - if len(s.floats) == 0 { - return false - } - - current, rest := s.floats[0], s.floats[1:] - - s.curTs = current.T * 1e6 // convert to nanoseconds - s.cur = current.F - - s.floats = rest - return true -} - -func (s *seriesToSampleIterator) Labels() string { - return s.lbls.String() -} - -func (s *seriesToSampleIterator) Sample() logproto.Sample { - return logproto.Sample{ - Timestamp: s.curTs, - Value: s.cur, - } -} - -func (s *seriesToSampleIterator) StreamHash() uint64 { - return s.lbls.Hash() -} - -func (s *seriesToSampleIterator) Error() error { - return nil -} - -func (s *seriesToSampleIterator) Close() error { - return nil -} - type paramCompat struct { expr syntax.SampleExpr from model.Time diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index b17ead9e1d2f..e72bce625de3 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -1136,7 +1136,6 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { } } -//TODO: use real times for clarity func Test_WriteQuerySamplesResponseJSON(t *testing.T) { expected := func(result string) string { return fmt.Sprintf(`{ From 33ead601778904d76a063b061cf2fb5a17b9867b Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 16:03:04 -0600 Subject: [PATCH 08/38] feat: hook up samples endpoint --- pkg/loghttp/patterns.go | 1 + pkg/loghttp/samples.go | 35 ++++++++ pkg/loghttp/samples_test.go | 122 ++++++++++++++++++++++++++++ pkg/logproto/compat.go | 33 ++++++++ pkg/logproto/extensions.go | 42 ++++++++++ pkg/logproto/extensions_test.go | 56 +++++++++++++ pkg/loki/modules.go | 2 + pkg/querier/handler.go | 8 ++ pkg/querier/http.go | 13 +++ pkg/querier/querier.go | 2 +- pkg/querier/queryrange/codec.go | 44 ++++++++++ pkg/querier/queryrange/marshal.go | 2 + pkg/querier/queryrange/roundtrip.go | 6 +- 13 files changed, 364 insertions(+), 2 deletions(-) create mode 100644 pkg/loghttp/samples.go create mode 100644 pkg/loghttp/samples_test.go diff --git a/pkg/loghttp/patterns.go b/pkg/loghttp/patterns.go index 8b855b2c951a..1a7b73d748a4 100644 --- a/pkg/loghttp/patterns.go +++ b/pkg/loghttp/patterns.go @@ -33,3 +33,4 @@ func ParsePatternsQuery(r *http.Request) (*logproto.QueryPatternsRequest, error) return req, nil } + diff --git a/pkg/loghttp/samples.go b/pkg/loghttp/samples.go new file mode 100644 index 000000000000..9266dc228501 --- /dev/null +++ b/pkg/loghttp/samples.go @@ -0,0 +1,35 @@ +package loghttp + +import ( + "net/http" + + "github.com/grafana/loki/v3/pkg/logproto" +) + +func ParseSamplesQuery(r *http.Request) (*logproto.QuerySamplesRequest, error) { + req := &logproto.QuerySamplesRequest{} + + req.Query = query(r) + start, end, err := bounds(r) + if err != nil { + return nil, err + } + req.Start = start + req.End = end + + calculatedStep, err := step(r, start, end) + if err != nil { + return nil, err + } + if calculatedStep <= 0 { + return nil, errZeroOrNegativeStep + } + // For safety, limit the number of returned points per timeseries. + // This is sufficient for 60s resolution for a week or 1h resolution for a year. + if (req.End.Sub(req.Start) / calculatedStep) > 11000 { + return nil, errStepTooSmall + } + req.Step = calculatedStep.Milliseconds() + + return req, nil +} diff --git a/pkg/loghttp/samples_test.go b/pkg/loghttp/samples_test.go new file mode 100644 index 000000000000..971b009045b1 --- /dev/null +++ b/pkg/loghttp/samples_test.go @@ -0,0 +1,122 @@ +package loghttp + +import ( + "net/http" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logproto" +) + +func TestParseSamplesQuery(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + path string + want *logproto.QuerySamplesRequest + wantErr bool + }{ + { + name: "should correctly parse valid params", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=5s", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(3600, 0), + Step: (5 * time.Second).Milliseconds(), + }, + }, + { + name: "should default empty step param to sensible step for the range", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(3600, 0), + Step: (14 * time.Second).Milliseconds(), + }, + }, + { + name: "should default start to zero for empty start param", + path: "/loki/api/v1/patterns?query={}&end=3600000000000", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(0, 0), + End: time.Unix(3600, 0), + Step: (14 * time.Second).Milliseconds(), + }, + }, + { + name: "should accept step with no units as seconds", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=10", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(3600, 0), + Step: (10 * time.Second).Milliseconds(), + }, + }, + { + name: "should accept step as string duration in seconds", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=15s", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(3600, 0), + Step: (15 * time.Second).Milliseconds(), + }, + }, + { + name: "should correctly parse long duration for step", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=10h", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(3600, 0), + Step: (10 * time.Hour).Milliseconds(), + }, + }, + { + name: "should reject negative step value", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=-5s", + want: nil, + wantErr: true, + }, + { + name: "should reject very small step for big range", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=3600000000000&step=50ms", + want: nil, + wantErr: true, + }, + { + name: "should accept very small step for small range", + path: "/loki/api/v1/patterns?query={}&start=100000000000&end=110000000000&step=50ms", + want: &logproto.QuerySamplesRequest{ + Query: "{}", + Start: time.Unix(100, 0), + End: time.Unix(110, 0), + Step: (50 * time.Millisecond).Milliseconds(), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + req, err := http.NewRequest(http.MethodGet, tt.path, nil) + require.NoError(t, err) + err = req.ParseForm() + require.NoError(t, err) + + got, err := ParseSamplesQuery(req) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + assert.Equalf(t, tt.want, got, "Incorrect response from input path: %s", tt.path) + }) + } +} diff --git a/pkg/logproto/compat.go b/pkg/logproto/compat.go index a11467584b58..93007df88d1f 100644 --- a/pkg/logproto/compat.go +++ b/pkg/logproto/compat.go @@ -82,6 +82,10 @@ func FromMetricsToLabelAdapters(metric model.Metric) []LabelAdapter { return result } +func FromMetricsToLabels(metric model.Metric) labels.Labels { + return FromLabelAdaptersToLabels(FromMetricsToLabelAdapters(metric)) +} + type byLabel []LabelAdapter func (s byLabel) Len() int { return len(s) } @@ -534,3 +538,32 @@ func (m *QueryPatternsRequest) LogToSpan(sp opentracing.Span) { } sp.LogFields(fields...) } + +func (m *QuerySamplesRequest) GetCachingOptions() (res definitions.CachingOptions) { return } + +func (m *QuerySamplesRequest) WithStartEnd(start, end time.Time) definitions.Request { + clone := *m + clone.Start = start + clone.End = end + return &clone +} + +func (m *QuerySamplesRequest) WithQuery(query string) definitions.Request { + clone := *m + clone.Query = query + return &clone +} + +func (m *QuerySamplesRequest) WithStartEndForCache(start, end time.Time) resultscache.Request { + return m.WithStartEnd(start, end).(resultscache.Request) +} + +func (m *QuerySamplesRequest) LogToSpan(sp opentracing.Span) { + fields := []otlog.Field{ + otlog.String("query", m.GetQuery()), + otlog.String("start", m.Start.String()), + otlog.String("end", m.End.String()), + otlog.String("step", time.Duration(m.Step).String()), + } + sp.LogFields(fields...) +} diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index 9ca73b7ecbb4..284ac792ebb9 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -1,10 +1,12 @@ package logproto import ( + "encoding/json" "sort" "strings" "sync/atomic" //lint:ignore faillint we can't use go.uber.org/atomic with a protobuf struct without wrapping it. + "github.com/buger/jsonparser" "github.com/cespare/xxhash/v2" "github.com/dustin/go-humanize" jsoniter "github.com/json-iterator/go" @@ -192,3 +194,43 @@ func (m *ShardsResponse) Merge(other *ShardsResponse) { func NewPatternSeries(pattern string, samples []*PatternSample) *PatternSeries { return &PatternSeries{Pattern: pattern, Samples: samples} } + +// UnmarshalJSON implements the json.Unmarshaler interface. +// QuerySamplesResponse json representation is different from the proto +func (r *QuerySamplesResponse) UnmarshalJSON(data []byte) error { + return jsonparser.ObjectEach( + data, + func(key, value []byte, dataType jsonparser.ValueType, offset int) error { + if string(key) == "data" { + var m []model.SampleStream + if err := json.Unmarshal(value, &m); err != nil { + return err + } + series := make([]Series, len(m)) + + for i, s := range m { + lbls := FromMetricsToLabels(s.Metric) + + newSeries := Series{ + Labels: s.Metric.String(), + StreamHash: lbls.Hash(), + Samples: make([]Sample, len(s.Values)), + } + + for j, samplePair := range s.Values { + newSeries.Samples[j] = Sample{ + Timestamp: samplePair.Timestamp.UnixNano(), + Value: float64(samplePair.Value), + } + } + + series[i] = newSeries + } + + r.Series = series + } + + return nil + }, + ) +} diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go index 95afc117bb9b..12ab2d940fdf 100644 --- a/pkg/logproto/extensions_test.go +++ b/pkg/logproto/extensions_test.go @@ -3,6 +3,7 @@ package logproto import ( "testing" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" ) @@ -74,3 +75,58 @@ func TestQueryPatternsResponse_UnmarshalJSON(t *testing.T) { require.Nil(t, err) require.Equal(t, expectedSeries, r.Series) } + +func TestQuerySamplesResponse_UnmarshalJSON(t *testing.T) { + mockData := []byte(`{ + "status": "success", + "data": [{ + "metric": { + "foo": "bar" + }, + "values": [ + [0.001, "1"], + [0.002, "2"] + ] + }, + { + "metric": { + "foo": "baz", + "bar": "qux" + }, + "values": [ + [0.003, "3"], + [0.004, "4"] + ] + }] + }`) + + lbls1, err := syntax.ParseLabels(`{foo="bar"}`) + require.NoError(t, err) + lbls2, err := syntax.ParseLabels(`{bar="qux", foo="baz"}`) + require.NoError(t, err) + + expectedSamples := []Series{ + { + Labels: lbls1.String(), + Samples: []Sample{ + {Timestamp: 1e6, Value: 1}, // 1ms after epoch in ns + {Timestamp: 2e6, Value: 2}, // 2ms after epoch in ns + }, + StreamHash: lbls1.Hash(), + }, + { + Labels: lbls2.String(), + Samples: []Sample{ + {Timestamp: 3e6, Value: 3}, // 3ms after epoch in ns + {Timestamp: 4e6, Value: 4}, // 4ms after epoch in ns + }, + StreamHash: lbls2.Hash(), + }, + } + + r := &QuerySamplesResponse{} + err = r.UnmarshalJSON(mockData) + + require.Nil(t, err) + require.Equal(t, expectedSamples, r.Series) +} diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index e73369aca2d7..27df97e8f90b 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -521,6 +521,7 @@ func (t *Loki) initQuerier() (services.Service, error) { router.Path("/loki/api/v1/index/volume").Methods("GET", "POST").Handler(volumeHTTPMiddleware.Wrap(httpHandler)) router.Path("/loki/api/v1/index/volume_range").Methods("GET", "POST").Handler(volumeRangeHTTPMiddleware.Wrap(httpHandler)) router.Path("/loki/api/v1/patterns").Methods("GET", "POST").Handler(httpHandler) + router.Path("/loki/api/v1/explore/query_range").Methods("GET", "POST").Handler(httpHandler) router.Path("/api/prom/query").Methods("GET", "POST").Handler( middleware.Merge( @@ -1105,6 +1106,7 @@ func (t *Loki) initQueryFrontend() (_ services.Service, err error) { t.Server.HTTP.Path("/loki/api/v1/label/{name}/values").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/series").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/patterns").Methods("GET", "POST").Handler(frontendHandler) + t.Server.HTTP.Path("/loki/api/v1/explore/query_range").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/detected_labels").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/detected_fields").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/index/stats").Methods("GET", "POST").Handler(frontendHandler) diff --git a/pkg/querier/handler.go b/pkg/querier/handler.go index bb01a0e3754b..3fa77e1f5fb7 100644 --- a/pkg/querier/handler.go +++ b/pkg/querier/handler.go @@ -131,6 +131,14 @@ func (h *Handler) Do(ctx context.Context, req queryrangebase.Request) (queryrang } return &queryrange.DetectedLabelsResponse{Response: result}, nil + case *logproto.QuerySamplesRequest: + result, err := h.api.SamplesHandler(ctx, concrete) + if err != nil { + return nil, err + } + return &queryrange.QuerySamplesResponse{ + Response: result, + }, nil default: return nil, fmt.Errorf("unsupported query type %T", req) } diff --git a/pkg/querier/http.go b/pkg/querier/http.go index d4e629153203..021cbbfe7ea4 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -406,6 +406,19 @@ func (q *QuerierAPI) PatternsHandler(ctx context.Context, req *logproto.QueryPat return resp, nil } +func (q *QuerierAPI) SamplesHandler(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QuerySamplesResponse, error) { + resp, err := q.querier.SelectMetricSamples(ctx, req) + if err != nil { + return nil, err + } + if resp == nil { // Some stores don't implement this + return &logproto.QuerySamplesResponse{ + Series: []logproto.Series{}, + }, nil + } + return resp, nil +} + func (q *QuerierAPI) validateMaxEntriesLimits(ctx context.Context, expr syntax.Expr, limit uint32) error { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index a6618ef1931e..3c0180e7d133 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -105,7 +105,7 @@ type Querier interface { DetectedFields(ctx context.Context, req *logproto.DetectedFieldsRequest) (*logproto.DetectedFieldsResponse, error) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) DetectedLabels(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.DetectedLabelsResponse, error) - SelectMetricSamples(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QueryPatternsResponse, error) + SelectMetricSamples(ctx context.Context, req *logproto.QuerySamplesRequest) (*logproto.QuerySamplesResponse, error) } type Limits querier_limits.Limits diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 4863df7cfcfa..d39f678aa2f1 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -459,6 +459,12 @@ func (Codec) DecodeRequest(_ context.Context, r *http.Request, _ []string) (quer DetectedLabelsRequest: *req, path: r.URL.Path, }, nil + case SamplesQueryOp: + req, err := loghttp.ParseSamplesQuery(r) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + return req, nil default: return nil, httpgrpc.Errorf(http.StatusNotFound, fmt.Sprintf("unknown request path: %s", r.URL.Path)) } @@ -644,6 +650,12 @@ func (Codec) DecodeHTTPGrpcRequest(ctx context.Context, r *httpgrpc.HTTPRequest) DetectedLabelsRequest: *req, path: httpReq.URL.Path, }, ctx, err + case SamplesQueryOp: + req, err := loghttp.ParseSamplesQuery(httpReq) + if err != nil { + return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + return req, ctx, nil default: return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, fmt.Sprintf("unknown request path in HTTP gRPC decode: %s", r.Url)) } @@ -975,6 +987,27 @@ func (c Codec) EncodeRequest(ctx context.Context, r queryrangebase.Request) (*ht Header: header, } + return req.WithContext(ctx), nil + case *logproto.QuerySamplesRequest: + params := url.Values{ + "query": []string{request.GetQuery()}, + "start": []string{fmt.Sprintf("%d", request.Start.UnixNano())}, + "end": []string{fmt.Sprintf("%d", request.End.UnixNano())}, + "step": []string{fmt.Sprintf("%d", request.GetStep())}, + } + + u := &url.URL{ + Path: "/loki/api/v1/explore/query_range", + RawQuery: params.Encode(), + } + req := &http.Request{ + Method: "GET", + RequestURI: u.String(), // This is what the httpgrpc code looks at. + URL: u, + Body: http.NoBody, + Header: header, + } + return req.WithContext(ctx), nil default: return nil, httpgrpc.Errorf(http.StatusInternalServerError, fmt.Sprintf("invalid request format, got (%T)", r)) @@ -1007,6 +1040,8 @@ func (c Codec) Path(r queryrangebase.Request) string { return "/loki/api/v1/patterns" case *DetectedLabelsRequest: return "/loki/api/v1/detected_labels" + case *logproto.QuerySamplesRequest: + return "/loki/api/v1/explore/query_range" } return "other" @@ -1134,6 +1169,15 @@ func decodeResponseJSONFrom(buf []byte, req queryrangebase.Request, headers http Response: &resp, Headers: httpResponseHeadersToPromResponseHeaders(headers), }, nil + case *logproto.QuerySamplesRequest: + var resp logproto.QuerySamplesResponse + if err := json.Unmarshal(buf, &resp); err != nil { + return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) + } + return &QuerySamplesResponse{ + Response: &resp, + Headers: httpResponseHeadersToPromResponseHeaders(headers), + }, nil default: var resp loghttp.QueryResponse if err := resp.UnmarshalJSON(buf); err != nil { diff --git a/pkg/querier/queryrange/marshal.go b/pkg/querier/queryrange/marshal.go index b3920e00a666..7f81563ebdfb 100644 --- a/pkg/querier/queryrange/marshal.go +++ b/pkg/querier/queryrange/marshal.go @@ -403,6 +403,8 @@ func (Codec) QueryRequestWrap(ctx context.Context, r queryrangebase.Request) (*Q result.Request = &QueryRequest_DetectedLabels{DetectedLabels: &req.DetectedLabelsRequest} case *DetectedFieldsRequest: result.Request = &QueryRequest_DetectedFields{DetectedFields: &req.DetectedFieldsRequest} + case *logproto.QuerySamplesRequest: + result.Request = &QueryRequest_SamplesRequest{SamplesRequest: req} default: return nil, fmt.Errorf("unsupported request type while wrapping, got (%T)", r) } diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 35806abb38d4..f80132116645 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -294,7 +294,8 @@ func NewDetectedLabelsTripperware(cfg Config, opts logql.EngineOpts, logger log. NewLimitsMiddleware(l), NewQuerySizeLimiterMiddleware(schema.Configs, opts, logger, l, statsHandler), base.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics), - SplitByIntervalMiddleware(schema.Configs, limits, merger, splitter, metrics.SplitByMetrics)} + SplitByIntervalMiddleware(schema.Configs, limits, merger, splitter, metrics.SplitByMetrics), + } // The sharding middleware takes care of enforcing this limit for both shardable and non-shardable queries. // If we are not using sharding, we enforce the limit by adding this middleware after time splitting. @@ -516,6 +517,7 @@ const ( DetectedFieldsOp = "detected_fields" PatternsQueryOp = "patterns" DetectedLabelsOp = "detected_labels" + SamplesQueryOp = "samples" ) func getOperation(path string) string { @@ -542,6 +544,8 @@ func getOperation(path string) string { return PatternsQueryOp case path == "/loki/api/v1/detected_labels": return DetectedLabelsOp + case path == "/loki/api/v1/explore/query_range": + return SamplesQueryOp default: return "" } From 87f7282caf83b04cd4fd32fc1e308eb5665fb92d Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 16:34:48 -0600 Subject: [PATCH 09/38] chore: clean up linting --- pkg/loghttp/patterns.go | 1 - pkg/logproto/compat.go | 2 +- pkg/logproto/pattern.pb.go | 76 ++++++++++++++-------------- pkg/logproto/pattern.proto | 3 +- pkg/pattern/drain/chunk_test.go | 2 +- pkg/pattern/ingester_querier.go | 6 +-- pkg/pattern/ingester_querier_test.go | 40 +++++++-------- pkg/pattern/ingester_test.go | 5 +- pkg/pattern/instance.go | 4 +- pkg/pattern/instance_test.go | 7 +-- pkg/pattern/iter/batch.go | 2 +- pkg/pattern/metric/chunk.go | 24 ++++----- pkg/pattern/metric/chunk_test.go | 54 ++++++++++---------- pkg/pattern/metric/evaluator.go | 39 +++++++------- pkg/pattern/metric/evaluator_test.go | 4 +- pkg/pattern/tee.go | 4 +- 16 files changed, 135 insertions(+), 138 deletions(-) diff --git a/pkg/loghttp/patterns.go b/pkg/loghttp/patterns.go index 1a7b73d748a4..8b855b2c951a 100644 --- a/pkg/loghttp/patterns.go +++ b/pkg/loghttp/patterns.go @@ -33,4 +33,3 @@ func ParsePatternsQuery(r *http.Request) (*logproto.QueryPatternsRequest, error) return req, nil } - diff --git a/pkg/logproto/compat.go b/pkg/logproto/compat.go index 93007df88d1f..0ebbe1c7c024 100644 --- a/pkg/logproto/compat.go +++ b/pkg/logproto/compat.go @@ -83,7 +83,7 @@ func FromMetricsToLabelAdapters(metric model.Metric) []LabelAdapter { } func FromMetricsToLabels(metric model.Metric) labels.Labels { - return FromLabelAdaptersToLabels(FromMetricsToLabelAdapters(metric)) + return FromLabelAdaptersToLabels(FromMetricsToLabelAdapters(metric)) } type byLabel []LabelAdapter diff --git a/pkg/logproto/pattern.pb.go b/pkg/logproto/pattern.pb.go index 9fb0cd5d73e2..eb3cfce9e03e 100644 --- a/pkg/logproto/pattern.pb.go +++ b/pkg/logproto/pattern.pb.go @@ -373,44 +373,44 @@ func init() { proto.RegisterFile("pkg/logproto/pattern.proto", fileDescriptor_aa var fileDescriptor_aaf4192acc66a4ea = []byte{ // 597 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x54, 0xbd, 0x6e, 0xd4, 0x40, - 0x10, 0xf6, 0xc6, 0x97, 0xbf, 0x8d, 0xf8, 0xd1, 0x26, 0x01, 0xcb, 0x88, 0xb5, 0xe5, 0x86, 0x93, - 0x90, 0xbc, 0x70, 0x91, 0x00, 0x51, 0x5e, 0x05, 0x12, 0x48, 0x87, 0xa1, 0x42, 0x50, 0xf8, 0x92, - 0x8d, 0xef, 0x14, 0xdb, 0xeb, 0xf3, 0xae, 0x83, 0xe8, 0x78, 0x84, 0xb4, 0xbc, 0x01, 0x3d, 0x3c, - 0xc4, 0x95, 0x57, 0x46, 0x29, 0x0e, 0xce, 0xd7, 0x50, 0xe6, 0x11, 0x90, 0x77, 0xd7, 0xf7, 0x27, - 0xae, 0xa0, 0xa4, 0x59, 0xef, 0xcc, 0xf7, 0xcd, 0x78, 0x66, 0xbe, 0xb1, 0xa1, 0x9d, 0x9d, 0x45, - 0x24, 0x66, 0x51, 0x96, 0x33, 0xc1, 0x48, 0x16, 0x0a, 0x41, 0xf3, 0xd4, 0x97, 0x16, 0xda, 0xa9, - 0xfd, 0xf6, 0x41, 0xc4, 0x22, 0xa6, 0x28, 0xd5, 0x4d, 0xe1, 0xb6, 0x13, 0x31, 0x16, 0xc5, 0x94, - 0x48, 0xab, 0x5b, 0x9c, 0x12, 0xd1, 0x4f, 0x28, 0x17, 0x61, 0x92, 0x69, 0xc2, 0xbd, 0xa5, 0xe4, - 0xf5, 0x45, 0x83, 0xfb, 0x15, 0x98, 0x15, 0xbc, 0x27, 0x0f, 0xed, 0x74, 0x75, 0xc4, 0x20, 0x4e, - 0xd8, 0x09, 0x8d, 0x09, 0x17, 0xa1, 0xe0, 0xea, 0x54, 0x0c, 0xef, 0x07, 0x80, 0x07, 0x6f, 0x0a, - 0x9a, 0x7f, 0xee, 0xa8, 0x5a, 0x79, 0x40, 0x07, 0x05, 0xe5, 0x02, 0x1d, 0xc0, 0xcd, 0x41, 0xe5, - 0xb7, 0x80, 0x0b, 0x9a, 0xbb, 0x81, 0x32, 0xd0, 0x73, 0xb8, 0xc9, 0x45, 0x98, 0x0b, 0x6b, 0xc3, - 0x05, 0xcd, 0xbd, 0x96, 0xed, 0xab, 0x9a, 0xfd, 0xba, 0x66, 0xff, 0x5d, 0x5d, 0x73, 0x7b, 0x67, - 0x38, 0x76, 0x8c, 0x8b, 0x9f, 0x0e, 0x08, 0x54, 0x08, 0x7a, 0x02, 0x4d, 0x9a, 0x9e, 0x58, 0xe6, - 0x3f, 0x44, 0x56, 0x01, 0x08, 0xc1, 0x06, 0x17, 0x34, 0xb3, 0x1a, 0x2e, 0x68, 0x9a, 0x81, 0xbc, - 0x7b, 0x2f, 0xe0, 0xe1, 0x4a, 0xd5, 0x3c, 0x63, 0x29, 0xa7, 0x88, 0xc0, 0x2d, 0x4e, 0xf3, 0x3e, - 0xe5, 0x16, 0x70, 0xcd, 0xe6, 0x5e, 0xeb, 0xae, 0x3f, 0x9b, 0x93, 0xe6, 0xbe, 0x95, 0x70, 0xa0, - 0x69, 0xde, 0x07, 0x78, 0x63, 0x09, 0x40, 0x16, 0xdc, 0xd6, 0xba, 0xe9, 0xd6, 0x6b, 0x13, 0x3d, - 0x86, 0xdb, 0x3c, 0x4c, 0xb2, 0x98, 0x72, 0x6b, 0x63, 0x5d, 0x72, 0x89, 0x07, 0x35, 0xcf, 0x13, - 0xf3, 0xec, 0xd2, 0x83, 0x5e, 0xc3, 0xdd, 0x99, 0xac, 0x32, 0xbf, 0xd9, 0x26, 0x55, 0xbb, 0x57, - 0x63, 0xe7, 0x41, 0xd4, 0x17, 0xbd, 0xa2, 0xeb, 0x1f, 0xb3, 0xa4, 0xda, 0x81, 0x84, 0x8a, 0x1e, - 0x2d, 0x38, 0x39, 0x66, 0x49, 0xc2, 0x52, 0x22, 0x05, 0x94, 0x43, 0x0a, 0xe6, 0x19, 0x2a, 0x95, - 0xce, 0xc3, 0xb8, 0xa0, 0x52, 0x0f, 0x33, 0x50, 0x86, 0xf7, 0x1d, 0xc0, 0x7d, 0x39, 0x1e, 0xf5, - 0xd2, 0xff, 0x44, 0xd3, 0xaf, 0xf5, 0x2a, 0xce, 0xaa, 0xd6, 0x9a, 0x3e, 0x5b, 0xd1, 0xf4, 0xf6, - 0x7c, 0xec, 0x4a, 0xb3, 0xf6, 0xcd, 0xe1, 0xd8, 0x01, 0x57, 0x63, 0x67, 0x6b, 0x59, 0x5c, 0xf4, - 0x50, 0xb6, 0x26, 0xb8, 0x6e, 0xed, 0x96, 0xaf, 0x56, 0xff, 0x65, 0x1a, 0x51, 0x2e, 0x68, 0xde, - 0x6e, 0x54, 0x55, 0x05, 0x8a, 0x83, 0x6c, 0xb8, 0xf3, 0x29, 0xcc, 0xd3, 0x7e, 0x1a, 0x71, 0xcb, - 0x74, 0xcd, 0xe6, 0x6e, 0x30, 0xb3, 0x5b, 0x25, 0x80, 0xdb, 0x5a, 0x48, 0xf4, 0x14, 0x36, 0x3a, - 0x05, 0xef, 0xa1, 0xc3, 0x05, 0xf5, 0x0b, 0xde, 0xd3, 0x43, 0xb6, 0xef, 0xac, 0xba, 0x55, 0x17, - 0x9e, 0x81, 0x5e, 0xc1, 0x4d, 0xd9, 0x1f, 0xc2, 0x73, 0xca, 0xdf, 0xbe, 0x3d, 0xdb, 0x59, 0x8b, - 0xd7, 0xb9, 0x1e, 0x01, 0xd4, 0x81, 0x7b, 0x0b, 0xd3, 0x42, 0xf7, 0x57, 0x62, 0x96, 0xa5, 0xb7, - 0xf1, 0x3a, 0x78, 0x9e, 0xb1, 0xfd, 0x71, 0x34, 0xc1, 0xc6, 0xe5, 0x04, 0x1b, 0xd7, 0x13, 0x0c, - 0xbe, 0x94, 0x18, 0x7c, 0x2b, 0x31, 0x18, 0x96, 0x18, 0x8c, 0x4a, 0x0c, 0x7e, 0x95, 0x18, 0xfc, - 0x2e, 0xb1, 0x71, 0x5d, 0x62, 0x70, 0x31, 0xc5, 0xc6, 0x68, 0x8a, 0x8d, 0xcb, 0x29, 0x36, 0xde, - 0x2f, 0xae, 0x6d, 0x94, 0x87, 0xa7, 0x61, 0x1a, 0x92, 0x98, 0x9d, 0xf5, 0xc9, 0xf9, 0x11, 0x59, - 0xfc, 0x61, 0x75, 0xb7, 0xe4, 0xe3, 0xe8, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x3e, 0x65, 0x57, - 0x8c, 0x24, 0x05, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x54, 0xbb, 0x8e, 0xd3, 0x40, + 0x14, 0xf5, 0xac, 0xb3, 0xaf, 0x59, 0xf1, 0xd0, 0xec, 0x2e, 0x58, 0x46, 0x8c, 0x2d, 0x37, 0x44, + 0x42, 0xf2, 0x40, 0x56, 0x02, 0x44, 0x99, 0x0a, 0x24, 0x90, 0x82, 0xa1, 0x42, 0x50, 0x38, 0xbb, + 0xb3, 0x4e, 0xb4, 0xb6, 0xc7, 0xf1, 0x8c, 0x17, 0xd1, 0xf1, 0x09, 0xdb, 0xf2, 0x07, 0xf4, 0xf0, + 0x11, 0x29, 0x53, 0xae, 0xb6, 0x08, 0xc4, 0x69, 0x28, 0xf7, 0x13, 0x90, 0x67, 0xc6, 0x79, 0x89, + 0x14, 0x94, 0x34, 0xce, 0xdc, 0x7b, 0xce, 0xbd, 0x39, 0xf7, 0x9e, 0xb1, 0xa1, 0x9d, 0x9d, 0x45, + 0x24, 0x66, 0x51, 0x96, 0x33, 0xc1, 0x48, 0x16, 0x0a, 0x41, 0xf3, 0xd4, 0x97, 0x11, 0xda, 0xa9, + 0xf3, 0xf6, 0x41, 0xc4, 0x22, 0xa6, 0x28, 0xd5, 0x49, 0xe1, 0xb6, 0x13, 0x31, 0x16, 0xc5, 0x94, + 0xc8, 0xa8, 0x5b, 0x9c, 0x12, 0xd1, 0x4f, 0x28, 0x17, 0x61, 0x92, 0x69, 0xc2, 0xbd, 0xa5, 0xe6, + 0xf5, 0x41, 0x83, 0xae, 0x06, 0x07, 0x71, 0xc2, 0x4e, 0x68, 0x4c, 0xb8, 0x08, 0x05, 0x57, 0x4f, + 0xcd, 0xd8, 0xaf, 0x18, 0x59, 0xc1, 0x7b, 0xf2, 0xa1, 0x92, 0xde, 0x0f, 0x00, 0x0f, 0xde, 0x14, + 0x34, 0xff, 0xdc, 0x51, 0x5a, 0x79, 0x40, 0x07, 0x05, 0xe5, 0x02, 0x1d, 0xc0, 0xcd, 0x41, 0x95, + 0xb7, 0x80, 0x0b, 0x9a, 0xbb, 0x81, 0x0a, 0xd0, 0x73, 0xb8, 0xc9, 0x45, 0x98, 0x0b, 0x6b, 0xc3, + 0x05, 0xcd, 0xbd, 0x96, 0xed, 0x2b, 0xcd, 0x7e, 0xad, 0xd9, 0x7f, 0x57, 0x6b, 0x6e, 0xef, 0x0c, + 0xc7, 0x8e, 0x71, 0xf1, 0xd3, 0x01, 0x81, 0x2a, 0x41, 0x4f, 0xa0, 0x49, 0xd3, 0x13, 0xcb, 0xfc, + 0x87, 0xca, 0xaa, 0x00, 0x21, 0xd8, 0xe0, 0x82, 0x66, 0x56, 0xc3, 0x05, 0x4d, 0x33, 0x90, 0x67, + 0xef, 0x05, 0x3c, 0x5c, 0x51, 0xcd, 0x33, 0x96, 0x72, 0x8a, 0x08, 0xdc, 0xe2, 0x34, 0xef, 0x53, + 0x6e, 0x01, 0xd7, 0x6c, 0xee, 0xb5, 0xee, 0xfa, 0xb3, 0x3d, 0x69, 0xee, 0x5b, 0x09, 0x07, 0x9a, + 0xe6, 0x7d, 0x80, 0x37, 0x96, 0x00, 0x64, 0xc1, 0x6d, 0xed, 0x9b, 0x1e, 0xbd, 0x0e, 0xd1, 0x63, + 0xb8, 0xcd, 0xc3, 0x24, 0x8b, 0x29, 0xb7, 0x36, 0xd6, 0x35, 0x97, 0x78, 0x50, 0xf3, 0x3c, 0x31, + 0xef, 0x2e, 0x33, 0xe8, 0x35, 0xdc, 0x9d, 0xd9, 0x2a, 0xfb, 0x9b, 0x6d, 0x52, 0x8d, 0x7b, 0x35, + 0x76, 0x1e, 0x44, 0x7d, 0xd1, 0x2b, 0xba, 0xfe, 0x31, 0x4b, 0xaa, 0x3b, 0x90, 0x50, 0xd1, 0xa3, + 0x05, 0x27, 0xc7, 0x2c, 0x49, 0x58, 0x4a, 0xa4, 0xab, 0x72, 0x49, 0xc1, 0xbc, 0x43, 0xe5, 0xd2, + 0x79, 0x18, 0x17, 0x54, 0xfa, 0x61, 0x06, 0x2a, 0xf0, 0xbe, 0x03, 0xb8, 0x2f, 0xd7, 0xa3, 0xfe, + 0xf4, 0x3f, 0xf1, 0xf4, 0x6b, 0x7d, 0x15, 0x67, 0xaa, 0xb5, 0xa7, 0xcf, 0x56, 0x3c, 0xbd, 0x3d, + 0x5f, 0xbb, 0xf2, 0xac, 0x7d, 0x73, 0x38, 0x76, 0xc0, 0xd5, 0xd8, 0xd9, 0x5a, 0x36, 0x17, 0x3d, + 0x94, 0xa3, 0x09, 0xae, 0x47, 0xbb, 0xe5, 0xab, 0xf7, 0xe1, 0x65, 0x1a, 0x51, 0x2e, 0x68, 0xde, + 0x6e, 0x54, 0xaa, 0x02, 0xc5, 0x41, 0x36, 0xdc, 0xf9, 0x14, 0xe6, 0x69, 0x3f, 0x8d, 0xb8, 0x65, + 0xba, 0x66, 0x73, 0x37, 0x98, 0xc5, 0xad, 0x12, 0xc0, 0x6d, 0x6d, 0x24, 0x7a, 0x0a, 0x1b, 0x9d, + 0x82, 0xf7, 0xd0, 0xe1, 0x82, 0xfb, 0x05, 0xef, 0xe9, 0x25, 0xdb, 0x77, 0x56, 0xd3, 0x6a, 0x0a, + 0xcf, 0x40, 0xaf, 0xe0, 0xa6, 0x9c, 0x0f, 0xe1, 0x39, 0xe5, 0x6f, 0xef, 0x9e, 0xed, 0xac, 0xc5, + 0xeb, 0x5e, 0x8f, 0x00, 0xea, 0xc0, 0xbd, 0x85, 0x6d, 0xa1, 0xfb, 0x2b, 0x35, 0xcb, 0xd6, 0xdb, + 0x78, 0x1d, 0x3c, 0xef, 0xd8, 0xfe, 0x38, 0x9a, 0x60, 0xe3, 0x72, 0x82, 0x8d, 0xeb, 0x09, 0x06, + 0x5f, 0x4a, 0x0c, 0xbe, 0x95, 0x18, 0x0c, 0x4b, 0x0c, 0x46, 0x25, 0x06, 0xbf, 0x4a, 0x0c, 0x7e, + 0x97, 0xd8, 0xb8, 0x2e, 0x31, 0xb8, 0x98, 0x62, 0x63, 0x34, 0xc5, 0xc6, 0xe5, 0x14, 0x1b, 0xef, + 0x17, 0xaf, 0x6d, 0x94, 0x87, 0xa7, 0x61, 0x1a, 0x92, 0x98, 0x9d, 0xf5, 0xc9, 0xf9, 0x11, 0x59, + 0xfc, 0x60, 0x75, 0xb7, 0xe4, 0xcf, 0xd1, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xbc, 0xa2, 0x72, + 0x12, 0x24, 0x05, 0x00, 0x00, } func (this *QueryPatternsRequest) Equal(that interface{}) bool { diff --git a/pkg/logproto/pattern.proto b/pkg/logproto/pattern.proto index 419cfa8194a1..c3e4caaffc5c 100644 --- a/pkg/logproto/pattern.proto +++ b/pkg/logproto/pattern.proto @@ -5,8 +5,8 @@ package logproto; import "gogoproto/gogo.proto"; import "google/protobuf/timestamp.proto"; import "pkg/logproto/logproto.proto"; -import "pkg/push/push.proto"; import "pkg/logqlmodel/stats/stats.proto"; +import "pkg/push/push.proto"; option go_package = "github.com/grafana/loki/v3/pkg/logproto"; @@ -59,7 +59,6 @@ message QuerySamplesRequest { int64 step = 4; } - message QuerySamplesResponse { repeated Series series = 1 [ (gogoproto.customtype) = "Series", diff --git a/pkg/pattern/drain/chunk_test.go b/pkg/pattern/drain/chunk_test.go index 0442ebd5442d..7f3cb4066b2c 100644 --- a/pkg/pattern/drain/chunk_test.go +++ b/pkg/pattern/drain/chunk_test.go @@ -22,7 +22,7 @@ func TestAdd(t *testing.T) { cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + chunk.TimeResolution + 1) require.Equal(t, 2, len(cks)) require.Equal(t, 1, len(cks[1].Samples)) - cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) - TimeResolution) + cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) - chunk.TimeResolution) require.Equal(t, 2, len(cks)) require.Equalf(t, 1, len(cks[1].Samples), "Older samples should not be added if they arrive out of order") } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 4aed20fe8fc2..b021ce4d645c 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -81,11 +81,11 @@ func (q *IngesterQuerier) Samples( } var selector syntax.LogSelectorExpr - switch expr.(type) { + switch e := expr.(type) { case *syntax.VectorAggregationExpr: - selector, err = expr.(*syntax.VectorAggregationExpr).Selector() + selector, err = e.Selector() case *syntax.RangeAggregationExpr: - selector, err = expr.(*syntax.RangeAggregationExpr).Selector() + selector, err = e.Selector() default: return nil, ErrParseQuery } diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index 579734644b49..5f8c04ccff67 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -144,11 +144,11 @@ func (f *fakeRingClient) Pool() *ring_client.Pool { panic("not implemented") } -func (f *fakeRingClient) StartAsync(ctx context.Context) error { +func (f *fakeRingClient) StartAsync(_ context.Context) error { panic("not implemented") } -func (f *fakeRingClient) AwaitRunning(ctx context.Context) error { +func (f *fakeRingClient) AwaitRunning(_ context.Context) error { panic("not implemented") } @@ -156,7 +156,7 @@ func (f *fakeRingClient) StopAsync() { panic("not implemented") } -func (f *fakeRingClient) AwaitTerminated(ctx context.Context) error { +func (f *fakeRingClient) AwaitTerminated(_ context.Context) error { panic("not implemented") } @@ -168,7 +168,7 @@ func (f *fakeRingClient) State() services.State { panic("not implemented") } -func (f *fakeRingClient) AddListener(listener services.Listener) { +func (f *fakeRingClient) AddListener(_ services.Listener) { panic("not implemented") } @@ -179,20 +179,20 @@ func (f *fakeRingClient) Ring() ring.ReadRing { type fakeRing struct{} func (f *fakeRing) Get( - key uint32, - op ring.Operation, - bufDescs []ring.InstanceDesc, - bufHosts []string, - bufZones []string, + _ uint32, + _ ring.Operation, + _ []ring.InstanceDesc, + _ []string, + _ []string, ) (ring.ReplicationSet, error) { panic("not implemented") } -func (f *fakeRing) GetAllHealthy(op ring.Operation) (ring.ReplicationSet, error) { +func (f *fakeRing) GetAllHealthy(_ ring.Operation) (ring.ReplicationSet, error) { panic("not implemented") } -func (f *fakeRing) GetReplicationSetForOperation(op ring.Operation) (ring.ReplicationSet, error) { +func (f *fakeRing) GetReplicationSetForOperation(_ ring.Operation) (ring.ReplicationSet, error) { return ring.ReplicationSet{}, nil } @@ -204,31 +204,31 @@ func (f *fakeRing) InstancesCount() int { panic("not implemented") } -func (f *fakeRing) ShuffleShard(identifier string, size int) ring.ReadRing { +func (f *fakeRing) ShuffleShard(_ string, _ int) ring.ReadRing { panic("not implemented") } -func (f *fakeRing) GetInstanceState(instanceID string) (ring.InstanceState, error) { +func (f *fakeRing) GetInstanceState(_ string) (ring.InstanceState, error) { panic("not implemented") } func (f *fakeRing) ShuffleShardWithLookback( - identifier string, - size int, - lookbackPeriod time.Duration, - now time.Time, + _ string, + _ int, + _ time.Duration, + _ time.Time, ) ring.ReadRing { panic("not implemented") } -func (f *fakeRing) HasInstance(instanceID string) bool { +func (f *fakeRing) HasInstance(_ string) bool { panic("not implemented") } -func (f *fakeRing) CleanupShuffleShardCache(identifier string) { +func (f *fakeRing) CleanupShuffleShardCache(_ string) { panic("not implemented") } -func (f *fakeRing) GetTokenRangesForInstance(instanceID string) (ring.TokenRanges, error) { +func (f *fakeRing) GetTokenRangesForInstance(_ string) (ring.TokenRanges, error) { panic("not implemented") } diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index c1295de8e9d3..35fb37577e80 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -44,6 +44,7 @@ func TestInstancePushQuery(t *testing.T) { }, }, }) + require.NoError(t, err) err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ @@ -58,6 +59,8 @@ func TestInstancePushQuery(t *testing.T) { }, }, }) + require.NoError(t, err) + for i := 0; i <= 30; i++ { err = inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ @@ -74,7 +77,7 @@ func TestInstancePushQuery(t *testing.T) { }) require.NoError(t, err) } - require.NoError(t, err) + it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ Query: "{test=\"test\"}", Start: time.Unix(0, 0), diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index a9309d88d819..63c716967840 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -64,7 +64,7 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels, func() (*stream, error) { // add stream - return i.createStream(ctx, reqStream, i.aggregationCfg.Enabled) + return i.createStream(ctx, reqStream) }, nil) if err != nil { appendErr.Add(err) @@ -182,7 +182,7 @@ outer: return nil } -func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream, aggregateMetrics bool) (*stream, error) { +func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream) (*stream, error) { labels, err := syntax.ParseLabels(pushReqStream.Labels) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 67befa7ab5a8..3790c07ea1f0 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -6,13 +6,14 @@ import ( "time" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/push" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/push" ) func TestInstance_QuerySample(t *testing.T) { @@ -41,8 +42,7 @@ func TestInstance_QuerySample(t *testing.T) { lastTsMilli := (then + oneMin + thirtySeconds) // 0 + 60000 + 30000 = 90000 - // TODO(twhitney): Add a few more pushes to this or another test - instance.Push(ctx, &logproto.PushRequest{ + err = instance.Push(ctx, &logproto.PushRequest{ Streams: []push.Stream{ { Labels: labels.String(), @@ -68,6 +68,7 @@ func TestInstance_QuerySample(t *testing.T) { }, }, }) + require.NoError(t, err) // 5 min query range // 1 min step diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index d40004b667de..fee835d16208 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -49,7 +49,7 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int) (*logproto.QuerySam Samples: []logproto.Sample{}, StreamHash: hash, } - series[hash] = s + series[hash] = s } s.Samples = append(s.Samples, it.Sample()) diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 0198a91a6164..a2f4cf520aa9 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -1,7 +1,6 @@ package metric import ( - "context" "fmt" "slices" "sort" @@ -16,10 +15,10 @@ import ( "github.com/grafana/loki/v3/pkg/iter" ) -type MetricType int +type Type int const ( - Bytes MetricType = iota + Bytes Type = iota Count Unsupported ) @@ -52,8 +51,7 @@ func (c *Chunks) Observe(bytes, count float64, ts model.Time) { } func (c *Chunks) Iterator( - ctx context.Context, - typ MetricType, + typ Type, grouping *syntax.Grouping, from, through, step model.Time, ) (iter.SampleIterator, error) { @@ -102,24 +100,24 @@ func (c *Chunks) Iterator( return iter.NewSeriesIterator(series), nil } -type MetricSample struct { +type Sample struct { Timestamp model.Time Bytes float64 Count float64 } -func newSample(bytes, count float64, ts model.Time) MetricSample { - return MetricSample{ +func newSample(bytes, count float64, ts model.Time) Sample { + return Sample{ Timestamp: ts, Bytes: bytes, Count: count, } } -type MetricSamples []MetricSample +type Samples []Sample type Chunk struct { - Samples MetricSamples + Samples Samples mint, maxt int64 } @@ -127,7 +125,7 @@ func (c *Chunk) Bounds() (fromT, toT time.Time) { return time.Unix(0, c.mint), time.Unix(0, c.maxt) } -func (c *Chunk) AddSample(s MetricSample) { +func (c *Chunk) AddSample(s Sample) { c.Samples = append(c.Samples, s) ts := int64(s.Timestamp) @@ -142,7 +140,7 @@ func (c *Chunk) AddSample(s MetricSample) { func newChunk(bytes, count float64, ts model.Time) Chunk { maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 - v := Chunk{Samples: make(MetricSamples, 1, maxSize)} + v := Chunk{Samples: make(Samples, 1, maxSize)} v.Samples[0] = newSample(bytes, count, ts) return v } @@ -160,7 +158,7 @@ func (c *Chunk) spaceFor(ts model.Time) bool { // the step evaluator. start and end are in milliseconds since epoch. // step is a duration in milliseconds. func (c *Chunk) ForTypeAndRange( - typ MetricType, + typ Type, start, end model.Time, ) ([]logproto.Sample, error) { if typ == Unsupported { diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 77848cc3957b..7c99e85c2a71 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -1,7 +1,6 @@ package metric import ( - "context" "reflect" "testing" @@ -17,7 +16,7 @@ func TestForTypeAndRange(t *testing.T) { testCases := []struct { name string c *Chunk - metricType MetricType + metricType Type start model.Time end model.Time expected []logproto.Sample @@ -40,7 +39,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "No Overlap -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -52,7 +51,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "No Overlap -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -64,7 +63,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Complete Overlap -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -80,7 +79,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Complete Overlap -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -96,7 +95,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Partial Overlap -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -111,7 +110,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Partial Overlap -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -126,7 +125,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Single Element in Range -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -138,7 +137,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Single Element in Range -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -150,7 +149,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start Before First Element -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -165,7 +164,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start Before First Element -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -180,7 +179,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "End After Last Element -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -194,7 +193,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "End After Last Element -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -208,7 +207,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "End Exclusive -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -222,7 +221,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "End Exclusive -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -236,7 +235,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start before First and End Inclusive of First Element -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -248,7 +247,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start before First and End Inclusive of First Element -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -260,7 +259,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start and End before First Element -- count", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Count: 2}, {Timestamp: 4, Count: 4}, {Timestamp: 6, Count: 6}, @@ -272,7 +271,7 @@ func TestForTypeAndRange(t *testing.T) { }, { name: "Start and End before First Element -- bytes", - c: &Chunk{Samples: MetricSamples{ + c: &Chunk{Samples: Samples{ {Timestamp: 2, Bytes: 2}, {Timestamp: 4, Bytes: 4}, {Timestamp: 6, Bytes: 6}, @@ -298,7 +297,6 @@ func TestForTypeAndRange(t *testing.T) { // TODO(twhitney): test the maximum steps logic func Test_Chunks_Iterator(t *testing.T) { - ctx := context.Background() lbls := labels.Labels{ labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "container", Value: "jar"}, @@ -306,7 +304,7 @@ func Test_Chunks_Iterator(t *testing.T) { chunks := Chunks{ chunks: []Chunk{ { - Samples: []MetricSample{ + Samples: []Sample{ {Timestamp: 2, Bytes: 2, Count: 1}, {Timestamp: 4, Bytes: 4, Count: 3}, {Timestamp: 6, Bytes: 6, Count: 5}, @@ -319,7 +317,7 @@ func Test_Chunks_Iterator(t *testing.T) { } t.Run("without grouping", func(t *testing.T) { - it, err := chunks.Iterator(ctx, Bytes, nil, 0, 10, 2) + it, err := chunks.Iterator(Bytes, nil, 0, 10, 2) require.NoError(t, err) res, err := iter.ReadAllSamples(it) @@ -328,7 +326,7 @@ func Test_Chunks_Iterator(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, lbls.String(), res.Series[0].GetLabels()) - it, err = chunks.Iterator(ctx, Count, nil, 0, 10, 2) + it, err = chunks.Iterator(Count, nil, 0, 10, 2) require.NoError(t, err) res, err = iter.ReadAllSamples(it) @@ -351,7 +349,7 @@ func Test_Chunks_Iterator(t *testing.T) { }, } - it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) + it, err := chunks.Iterator(Bytes, grouping, 0, 10, 2) require.NoError(t, err) res, err := iter.ReadAllSamples(it) @@ -360,7 +358,7 @@ func Test_Chunks_Iterator(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) - it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) + it, err = chunks.Iterator(Count, grouping, 0, 10, 2) require.NoError(t, err) res, err = iter.ReadAllSamples(it) @@ -383,7 +381,7 @@ func Test_Chunks_Iterator(t *testing.T) { }, } - it, err := chunks.Iterator(ctx, Bytes, grouping, 0, 10, 2) + it, err := chunks.Iterator(Bytes, grouping, 0, 10, 2) require.NoError(t, err) res, err := iter.ReadAllSamples(it) @@ -392,7 +390,7 @@ func Test_Chunks_Iterator(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) - it, err = chunks.Iterator(ctx, Count, grouping, 0, 10, 2) + it, err = chunks.Iterator(Count, grouping, 0, 10, 2) require.NoError(t, err) res, err = iter.ReadAllSamples(it) diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index 96939ffc5187..800776f8a934 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -18,8 +18,8 @@ import ( ) // TODO(twhitney): duplication with code in NewStepEvaluator -func extractMetricType(expr syntax.SampleExpr) (MetricType, error) { - var typ MetricType +func extractMetricType(expr syntax.SampleExpr) (Type, error) { + var typ Type switch e := expr.(type) { case *syntax.VectorAggregationExpr: if rangeExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { @@ -110,7 +110,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( ) (logql.StepEvaluator, error) { fromWithRangeAndOffset := from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset) throughWithOffset := through.Add(-rangExpr.Left.Offset) - it, err := ev.chunks.Iterator(ctx, metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) + it, err := ev.chunks.Iterator(metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) if err != nil { return nil, err } @@ -144,7 +144,7 @@ func (ev *DefaultEvaluatorFactory) NewStepEvaluator( case *syntax.RangeAggregationExpr: fromWithRangeAndOffset := from.Add(-e.Left.Interval).Add(-e.Left.Offset) throughWithOffset := through.Add(-e.Left.Offset) - it, err := ev.chunks.Iterator(ctx, metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) + it, err := ev.chunks.Iterator(metricType, e.Grouping, fromWithRangeAndOffset, throughWithOffset, step) if err != nil { return nil, err } @@ -171,7 +171,7 @@ func NewSampleRangeAggEvaluator( o time.Duration, ) (logql.StepEvaluator, error) { iter, err := newRangeVectorIterator( - it, expr, + it, expr.Left.Interval.Nanoseconds(), q.Step().Nanoseconds(), q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), @@ -185,7 +185,6 @@ func NewSampleRangeAggEvaluator( func newRangeVectorIterator( it loki_iter.PeekingSampleIterator, - expr *syntax.RangeAggregationExpr, selRange, step, start, end, offset int64, ) (logql.RangeVectorIterator, error) { // forces at least one step. @@ -242,7 +241,7 @@ func newRangeVectorIterator( ), nil } -type paramCompat struct { +type ParamCompat struct { expr syntax.SampleExpr from model.Time through model.Time @@ -252,8 +251,8 @@ type paramCompat struct { func NewParams( expr syntax.SampleExpr, from, through, step model.Time, -) *paramCompat { - return ¶mCompat{ +) *ParamCompat { + return &ParamCompat{ expr: expr, from: from, through: through, @@ -261,46 +260,46 @@ func NewParams( } } -func (p *paramCompat) QueryString() string { +func (p *ParamCompat) QueryString() string { return p.expr.String() } -func (p *paramCompat) Start() time.Time { +func (p *ParamCompat) Start() time.Time { return p.from.Time() } -func (p *paramCompat) End() time.Time { +func (p *ParamCompat) End() time.Time { return p.through.Time() } -func (p *paramCompat) Step() time.Duration { +func (p *ParamCompat) Step() time.Duration { return time.Duration(p.step.UnixNano()) } -func (p *paramCompat) Interval() time.Duration { +func (p *ParamCompat) Interval() time.Duration { return time.Duration(0) } -func (p *paramCompat) Limit() uint32 { +func (p *ParamCompat) Limit() uint32 { return 0 } -func (p *paramCompat) Direction() logproto.Direction { +func (p *ParamCompat) Direction() logproto.Direction { return logproto.BACKWARD } -func (p *paramCompat) Shards() []string { +func (p *ParamCompat) Shards() []string { return []string{} } -func (p *paramCompat) GetExpression() syntax.Expr { +func (p *ParamCompat) GetExpression() syntax.Expr { return p.expr } -func (p *paramCompat) GetStoreChunks() *logproto.ChunkRefGroup { +func (p *ParamCompat) GetStoreChunks() *logproto.ChunkRefGroup { return nil } -func (p *paramCompat) CachingOptions() (res resultscache.CachingOptions) { +func (p *ParamCompat) CachingOptions() (res resultscache.CachingOptions) { return } diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 6d81ab7edfd6..2596bb730724 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -34,7 +34,7 @@ func Test_SampleEvaluator(t *testing.T) { evaluator, err := factory.NewStepEvaluator( context.Background(), factory, - expr.(syntax.SampleExpr), + expr, // add 10s to the end to include chunks @ now model.Time(from), model.Time(through), model.Time(fiveMinMs), ) @@ -50,7 +50,7 @@ func Test_SampleEvaluator(t *testing.T) { return Chunks{ chunks: []Chunk{ { - Samples: []MetricSample{ + Samples: []Sample{ { Timestamp: beforeThenTime, Bytes: 1, diff --git a/pkg/pattern/tee.go b/pkg/pattern/tee.go index ed90d0dd478c..04d2bf7595fc 100644 --- a/pkg/pattern/tee.go +++ b/pkg/pattern/tee.go @@ -38,8 +38,8 @@ func NewTee( Name: "pattern_ingester_appends_total", Help: "The total number of batch appends sent to pattern ingesters.", }, []string{"ingester", "status"}), - cfg: cfg, - ringClient: ringClient, + cfg: cfg, + ringClient: ringClient, } return t, nil From 29febb7d8777a45fc6711122242d455a83264f45 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 16:42:12 -0600 Subject: [PATCH 10/38] chore: make format --- pkg/logproto/extensions_test.go | 3 ++- pkg/pattern/ingester_querier_test.go | 1 + pkg/pattern/instance_test.go | 7 ++++--- pkg/pattern/metric/chunk.go | 5 +++-- pkg/pattern/metric/chunk_test.go | 7 ++++--- pkg/pattern/metric/evaluator.go | 9 +++++---- pkg/pattern/metric/evaluator_test.go | 5 +++-- 7 files changed, 22 insertions(+), 15 deletions(-) diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go index 12ab2d940fdf..35823d3ca554 100644 --- a/pkg/logproto/extensions_test.go +++ b/pkg/logproto/extensions_test.go @@ -3,9 +3,10 @@ package logproto import ( "testing" - "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logql/syntax" ) func TestShard_SpaceFor(t *testing.T) { diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index 5f8c04ccff67..a40a1ba5c2dd 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -14,6 +14,7 @@ import ( "github.com/grafana/dskit/ring" ring_client "github.com/grafana/dskit/ring/client" "github.com/grafana/dskit/services" + "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/pattern/metric" ) diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 3790c07ea1f0..974e17403ce5 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -6,13 +6,14 @@ import ( "time" "github.com/go-kit/log" - "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/metric" + "github.com/grafana/loki/pkg/push" ) diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index a2f4cf520aa9..c48a3c78924b 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -6,11 +6,12 @@ import ( "sort" "time" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/chunk" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/iter" ) diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 7c99e85c2a71..0affcd197e14 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -4,12 +4,13 @@ import ( "reflect" "testing" - "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/iter" ) func TestForTypeAndRange(t *testing.T) { diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go index 800776f8a934..eb8bd4ac2d35 100644 --- a/pkg/pattern/metric/evaluator.go +++ b/pkg/pattern/metric/evaluator.go @@ -6,15 +6,16 @@ import ( "sort" "time" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" + loki_iter "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/promql" ) // TODO(twhitney): duplication with code in NewStepEvaluator diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 2596bb730724..1f9683d214b3 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -4,11 +4,12 @@ import ( "context" "testing" - "github.com/grafana/loki/v3/pkg/logql" - "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" ) func Test_SampleEvaluator(t *testing.T) { From cbf9fc08ee488a932cafca277995bf881ec30ee2 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 31 May 2024 16:46:39 -0600 Subject: [PATCH 11/38] docs: update docs --- docs/sources/shared/configuration.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index aa045ce1f977..9894cfa2ead6 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -316,6 +316,13 @@ pattern_ingester: # CLI flag: -pattern-ingester.flush-check-period [flush_check_period: | default = 30s] + # Configures the metric aggregation and storage behavior of the pattern + # ingester. + metric_aggregation: + # Whether the pattern ingester metric aggregation is enabled. + # CLI flag: -pattern-ingester.metric-aggregation.enabled + [enabled: | default = false] + # The index_gateway block configures the Loki index gateway server, responsible # for serving index queries without the need to constantly interact with the # object store. From 6ed195e4a3159fc0a9e85ef38c7fe2929ab39454 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 3 Jun 2024 09:25:37 -0600 Subject: [PATCH 12/38] fix: nanosecond values in test with non-decimal seconds value --- pkg/util/marshal/marshal.go | 4 +++- pkg/util/marshal/marshal_test.go | 24 ++++++++++++------------ 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 3c8ffbe9d0e3..5d146083a4bd 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -9,6 +9,7 @@ import ( "github.com/gorilla/websocket" jsoniter "github.com/json-iterator/go" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" @@ -278,7 +279,8 @@ func logprotoSeriesToPromQLMatrix(series []logproto.Series) (promql.Matrix, erro Floats: make([]promql.FPoint, len(s.Samples)), } for i, sample := range s.Samples { - promSeries.Floats[i] = promql.FPoint{T: sample.Timestamp / 1e6, F: sample.Value} + t := model.TimeFromUnixNano(sample.Timestamp) + promSeries.Floats[i] = promql.FPoint{T: int64(t), F: sample.Value} } promMatrix[i] = promSeries } diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index e72bce625de3..b8c90c5a61d4 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -1162,8 +1162,8 @@ func Test_WriteQuerySamplesResponseJSON(t *testing.T) { { Labels: `{foo="bar"}`, Samples: []logproto.Sample{ - {Timestamp: 1, Value: 1}, - {Timestamp: 2, Value: 2}, + {Timestamp: 1e9, Value: 1}, + {Timestamp: 2e9, Value: 2}, }, }, }, @@ -1174,8 +1174,8 @@ func Test_WriteQuerySamplesResponseJSON(t *testing.T) { "foo": "bar" }, "values": [ - [0.001, "1"], - [0.002, "2"] + [1, "1"], + [2, "2"] ] } ]`), @@ -1186,15 +1186,15 @@ func Test_WriteQuerySamplesResponseJSON(t *testing.T) { { Labels: `{foo="bar"}`, Samples: []logproto.Sample{ - {Timestamp: 1, Value: 1}, - {Timestamp: 2, Value: 2}, + {Timestamp: 1e9, Value: 1}, + {Timestamp: 2e9, Value: 2}, }, }, { Labels: `{foo="buzz"}`, Samples: []logproto.Sample{ - {Timestamp: 3, Value: 1}, - {Timestamp: 3, Value: 2}, + {Timestamp: 3e9, Value: 1}, + {Timestamp: 3e9, Value: 2}, }, }, }, @@ -1205,8 +1205,8 @@ func Test_WriteQuerySamplesResponseJSON(t *testing.T) { "foo": "bar" }, "values": [ - [0.001, "1"], - [0.002, "2"] + [1, "1"], + [2, "2"] ] }, { @@ -1214,8 +1214,8 @@ func Test_WriteQuerySamplesResponseJSON(t *testing.T) { "foo": "buzz" }, "values": [ - [0.003, "1"], - [0.003, "2"] + [3, "1"], + [3, "2"] ] } ]`), From 1822b880f456e8a01b1675f0e5ce1b0ae3fe8b91 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 3 Jun 2024 11:42:52 -0600 Subject: [PATCH 13/38] fix: formatting --- pkg/util/marshal/marshal.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 5d146083a4bd..6886650c822b 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -279,7 +279,7 @@ func logprotoSeriesToPromQLMatrix(series []logproto.Series) (promql.Matrix, erro Floats: make([]promql.FPoint, len(s.Samples)), } for i, sample := range s.Samples { - t := model.TimeFromUnixNano(sample.Timestamp) + t := model.TimeFromUnixNano(sample.Timestamp) promSeries.Floats[i] = promql.FPoint{T: int64(t), F: sample.Value} } promMatrix[i] = promSeries From 3a018805e034adacba879769ae06975e6a6a03bf Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 4 Jun 2024 15:48:53 -0600 Subject: [PATCH 14/38] fix: move /explore/query_range case up --- pkg/querier/queryrange/marshal.go | 6 ++++++ pkg/querier/queryrange/roundtrip.go | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/pkg/querier/queryrange/marshal.go b/pkg/querier/queryrange/marshal.go index 7f81563ebdfb..367163694590 100644 --- a/pkg/querier/queryrange/marshal.go +++ b/pkg/querier/queryrange/marshal.go @@ -233,6 +233,8 @@ func QueryResponseUnwrap(res *QueryResponse) (queryrangebase.Response, error) { return concrete.DetectedLabels, nil case *QueryResponse_DetectedFields: return concrete.DetectedFields, nil + case *QueryResponse_SamplesResponse: + return concrete.SamplesResponse, nil default: return nil, fmt.Errorf("unsupported QueryResponse response type, got (%T)", res.Response) } @@ -274,6 +276,8 @@ func QueryResponseWrap(res queryrangebase.Response) (*QueryResponse, error) { p.Response = &QueryResponse_DetectedLabels{response} case *DetectedFieldsResponse: p.Response = &QueryResponse_DetectedFields{response} + case *QuerySamplesResponse: + p.Response = &QueryResponse_SamplesResponse{response} default: return nil, fmt.Errorf("invalid response format, got (%T)", res) } @@ -372,6 +376,8 @@ func (Codec) QueryRequestUnwrap(ctx context.Context, req *QueryRequest) (queryra return &DetectedFieldsRequest{ DetectedFieldsRequest: *concrete.DetectedFields, }, ctx, nil + case *QueryRequest_SamplesRequest: + return concrete.SamplesRequest, ctx, nil default: return nil, ctx, fmt.Errorf("unsupported request type while unwrapping, got (%T)", req.Request) } diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 89fd198ee2e5..5c9f3dd3b3c9 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -522,6 +522,8 @@ const ( func getOperation(path string) string { switch { + case path == "/loki/api/v1/explore/query_range": + return SamplesQueryOp case strings.HasSuffix(path, "/query_range") || strings.HasSuffix(path, "/prom/query"): return QueryRangeOp case strings.HasSuffix(path, "/series"): @@ -544,8 +546,6 @@ func getOperation(path string) string { return PatternsQueryOp case path == "/loki/api/v1/detected_labels": return DetectedLabelsOp - case path == "/loki/api/v1/explore/query_range": - return SamplesQueryOp default: return "" } From 30c5b908922642bea602f35191765915e2116d28 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Wed, 5 Jun 2024 13:51:59 -0600 Subject: [PATCH 15/38] chore: add metrics and debug logging --- cmd/loki/loki-local-config.yaml | 2 ++ pkg/pattern/ingester.go | 27 +++++++++++++------- pkg/pattern/ingester_querier.go | 19 ++++---------- pkg/pattern/instance.go | 6 +++-- pkg/pattern/metric/chunk.go | 43 +++++++++++++++++++++++++++----- pkg/pattern/metric/chunk_test.go | 26 +++++++++++++++++++ pkg/pattern/metric/config.go | 2 ++ pkg/pattern/metric/metrics.go | 28 +++++++++++++++++++++ pkg/pattern/stream.go | 30 ++++++++++++++++------ 9 files changed, 145 insertions(+), 38 deletions(-) create mode 100644 pkg/pattern/metric/metrics.go diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 913b9a4c1873..e2c54d545279 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -3,6 +3,7 @@ auth_enabled: false server: http_listen_port: 3100 grpc_listen_port: 9096 + log_level: debug common: instance_addr: 127.0.0.1 @@ -37,6 +38,7 @@ pattern_ingester: enabled: true metric_aggregation: enabled: true + log_push_observations: true ruler: alertmanager_url: http://localhost:9093 diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 755ad668af6a..831c073fbe2f 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -83,7 +83,8 @@ type Ingester struct { loopDone sync.WaitGroup loopQuit chan struct{} - metrics *ingesterMetrics + metrics *ingesterMetrics + chunkMetrics *metric.ChunkMetrics } func New( @@ -93,16 +94,18 @@ func New( logger log.Logger, ) (*Ingester, error) { metrics := newIngesterMetrics(registerer, metricsNamespace) + chunkMetrics := metric.NewChunkMetrics(registerer, metricsNamespace) registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) i := &Ingester{ - cfg: cfg, - logger: log.With(logger, "component", "pattern-ingester"), - registerer: registerer, - metrics: metrics, - instances: make(map[string]*instance), - flushQueues: make([]*util.PriorityQueue, cfg.ConcurrentFlushes), - loopQuit: make(chan struct{}), + cfg: cfg, + logger: log.With(logger, "component", "pattern-ingester"), + registerer: registerer, + metrics: metrics, + chunkMetrics: chunkMetrics, + instances: make(map[string]*instance), + flushQueues: make([]*util.PriorityQueue, cfg.ConcurrentFlushes), + loopQuit: make(chan struct{}), } i.Service = services.NewBasicService(i.starting, i.running, i.stopping) var err error @@ -347,7 +350,13 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { / inst, ok = i.instances[instanceID] if !ok { var err error - inst, err = newInstance(instanceID, i.logger, i.metrics, i.cfg.MetricAggregation) + inst, err = newInstance( + instanceID, + i.logger, + i.metrics, + i.chunkMetrics, + i.cfg.MetricAggregation, + ) if err != nil { return nil, err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index b021ce4d645c..a68707abf07c 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -7,6 +7,7 @@ import ( "net/http" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/prometheus/client_golang/prometheus" @@ -111,20 +112,6 @@ func (q *IngesterQuerier) Samples( return resp, nil } -func (q *IngesterQuerier) queryPattern(ctx context.Context, req *logproto.QueryPatternsRequest) ([]pattern_iter.Iterator, error) { - resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { - return client.Query(ctx, req) - }) - if err != nil { - return nil, err - } - iterators := make([]pattern_iter.Iterator, len(resps)) - for i := range resps { - iterators[i] = pattern_iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) - } - return iterators, nil -} - func (q *IngesterQuerier) querySample(ctx context.Context, req *logproto.QuerySamplesRequest) ([]loki_iter.SampleIterator, error) { resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { return client.QuerySample(ctx, req) @@ -132,6 +119,10 @@ func (q *IngesterQuerier) querySample(ctx context.Context, req *logproto.QuerySa if err != nil { return nil, err } + level.Debug(q.logger).Log("msg", "queried patterns ingesters for metric samples", + "query", req.Query, + "num_responses", len(resps)) + iterators := make([]loki_iter.SampleIterator, len(resps)) for i := range resps { iterators[i] = pattern_iter.NewQuerySamplesClientIterator(resps[i].response.(logproto.Pattern_QuerySampleClient)) diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 63c716967840..a67e1506e9ff 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -34,10 +34,11 @@ type instance struct { index *index.BitPrefixInvertedIndex logger log.Logger metrics *ingesterMetrics + chunkMetrics *metric.ChunkMetrics aggregationCfg metric.AggregationConfig } -func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics, aggCfg metric.AggregationConfig) (*instance, error) { +func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics, chunkMetrics *metric.ChunkMetrics, aggCfg metric.AggregationConfig) (*instance, error) { index, err := index.NewBitPrefixWithShards(indexShards) if err != nil { return nil, err @@ -49,6 +50,7 @@ func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics, streams: newStreamsMap(), index: index, metrics: metrics, + chunkMetrics: chunkMetrics, aggregationCfg: aggCfg, } i.mapper = ingester.NewFPMapper(i.getLabelsFromFingerprint) @@ -189,7 +191,7 @@ func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream } fp := i.getHashForLabels(labels) sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp) - s, err := newStream(fp, sortedLabels, i.metrics, i.aggregationCfg.Enabled) + s, err := newStream(fp, sortedLabels, i.metrics, i.chunkMetrics, i.aggregationCfg, i.logger) if err != nil { return nil, fmt.Errorf("failed to create stream: %w", err) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index c48a3c78924b..487cf5ba49be 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -6,6 +6,7 @@ import ( "sort" "time" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -24,27 +25,48 @@ const ( Unsupported ) +type metrics struct { + chunks prometheus.Gauge + samples prometheus.Counter +} + type Chunks struct { - chunks []Chunk - labels labels.Labels + chunks []Chunk + labels labels.Labels + service string + metrics metrics } -func NewChunks(labels labels.Labels) *Chunks { +func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics) *Chunks { + service := labels.Get("service_name") + if service == "" { + service = "unknown_service" + } + return &Chunks{ - chunks: make([]Chunk, 0), - labels: labels, + chunks: make([]Chunk, 0), + labels: labels, + service: service, + metrics: metrics{ + chunks: chunkMetrics.chunks.WithLabelValues(service), + samples: chunkMetrics.samples.WithLabelValues(service), + }, } } func (c *Chunks) Observe(bytes, count float64, ts model.Time) { + c.metrics.samples.Inc() + if len(c.chunks) == 0 { c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + c.metrics.chunks.Set(float64(len(c.chunks))) return } last := &(c.chunks)[len(c.chunks)-1] if !last.spaceFor(ts) { c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + c.metrics.chunks.Set(float64(len(c.chunks))) return } @@ -70,8 +92,17 @@ func (c *Chunks) Iterator( } } - // could have up to through-from/step steps for each chunk maximumSteps := int64(((through-from)/step)+1) * int64(len(c.chunks)) + // prevent a panic if maximumSteps is negative + if maximumSteps < 0 { + series := logproto.Series{ + Labels: lbls.String(), + Samples: []logproto.Sample{}, + StreamHash: lbls.Hash(), + } + return iter.NewSeriesIterator(series), nil + } + samples := make([]logproto.Sample, 0, maximumSteps) for _, chunk := range c.chunks { ss, err := chunk.ForTypeAndRange(typ, from, through) diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 0affcd197e14..b89e4942dc30 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -400,4 +400,30 @@ func Test_Chunks_Iterator(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, expectedLabels.String(), res.Series[0].GetLabels()) }) + + t.Run("handle slice capacity out of range", func(t *testing.T) { + chunks := Chunks{ + chunks: []Chunk{ + { + Samples: []Sample{}, + }, + }, + labels: lbls, + } + it, err := chunks.Iterator(Bytes, nil, 5e4, 0, 1e4) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + + require.Equal(t, 0, len(res.Series)) + + it, err = chunks.Iterator(Count, nil, 5e4, 0, 1e4) + require.NoError(t, err) + + res, err = iter.ReadAllSamples(it) + require.NoError(t, err) + + require.Equal(t, 0, len(res.Series)) + }) } diff --git a/pkg/pattern/metric/config.go b/pkg/pattern/metric/config.go index 8bdf85cdcb64..5c5fa4a54a29 100644 --- a/pkg/pattern/metric/config.go +++ b/pkg/pattern/metric/config.go @@ -4,6 +4,7 @@ import "flag" type AggregationConfig struct { Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` + LogPushObservations bool `yaml:"log_push_observations,omitempty" doc:"description=Wheter to log push observations."` } // RegisterFlags registers pattern ingester related flags. @@ -13,4 +14,5 @@ func (cfg *AggregationConfig) RegisterFlags(fs *flag.FlagSet) { func (cfg *AggregationConfig) RegisterFlagsWithPrefix(fs *flag.FlagSet, prefix string) { fs.BoolVar(&cfg.Enabled, prefix+"metric-aggregation.enabled", false, "Flag to enable or disable metric aggregation.") + fs.BoolVar(&cfg.LogPushObservations, prefix+"metric-aggregation.log-push-observations", false, "Flag to enable or disable logging of push observations.") } diff --git a/pkg/pattern/metric/metrics.go b/pkg/pattern/metric/metrics.go new file mode 100644 index 000000000000..1b49daec796b --- /dev/null +++ b/pkg/pattern/metric/metrics.go @@ -0,0 +1,28 @@ +package metric + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type ChunkMetrics struct { + chunks *prometheus.GaugeVec + samples *prometheus.CounterVec +} + +func NewChunkMetrics(r prometheus.Registerer, metricsNamespace string) *ChunkMetrics { + return &ChunkMetrics{ + chunks: promauto.With(r).NewGaugeVec(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: "pattern_ingester", + Name: "metric_chunks", + Help: "The total number of chunks in memory.", + }, []string{"service_name"}), + samples: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: "pattern_ingester", + Name: "metric_samples", + Help: "The total number of samples in memory.", + }, []string{"service_name"}), + } +} diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 52f539a5f7b0..0910609c2892 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -6,6 +6,8 @@ import ( "sync" "time" + "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" @@ -29,19 +31,23 @@ type stream struct { patterns *drain.Drain mtx sync.Mutex - aggregateMetrics bool - metrics *metric.Chunks + cfg metric.AggregationConfig + metrics *metric.Chunks evaluator metric.SampleEvaluatorFactory lastTs int64 + + logger log.Logger } func newStream( fp model.Fingerprint, labels labels.Labels, metrics *ingesterMetrics, - aggregateMetrics bool, + chunkMetrics *metric.ChunkMetrics, + cfg metric.AggregationConfig, + logger log.Logger, ) (*stream, error) { stream := &stream{ fp: fp, @@ -52,11 +58,12 @@ func newStream( PatternsEvictedTotal: metrics.patternsDiscardedTotal, PatternsDetectedTotal: metrics.patternsDetectedTotal, }), - aggregateMetrics: aggregateMetrics, + cfg: cfg, + logger: logger, } - if aggregateMetrics { - chunks := metric.NewChunks(labels) + if cfg.Enabled { + chunks := metric.NewChunks(labels, chunkMetrics) stream.metrics = chunks stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) } @@ -84,7 +91,16 @@ func (s *stream) Push( s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) } - if s.aggregateMetrics && s.metrics != nil { + if s.cfg.Enabled && s.metrics != nil { + if s.cfg.LogPushObservations { + level.Debug(s.logger). + Log("msg", "observing pushed log entries", + "stream", s.labelsString, + "bytes", bytes, + "count", count, + "sample_ts_ns", s.lastTs, + ) + } s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) } return nil From d94349c765d4389ef6c1a9da3690ef7e7dae2b13 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Wed, 5 Jun 2024 14:45:30 -0600 Subject: [PATCH 16/38] chore: add more debug logging to chunk iteration --- pkg/pattern/metric/chunk.go | 31 ++++++++++++++++++++++++++++++- pkg/pattern/stream.go | 2 +- 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 487cf5ba49be..388b1421addb 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -6,6 +6,8 @@ import ( "sort" "time" + "github.com/go-kit/kit/log/level" + "github.com/go-kit/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -35,9 +37,10 @@ type Chunks struct { labels labels.Labels service string metrics metrics + logger log.Logger } -func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics) *Chunks { +func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics, logger log.Logger) *Chunks { service := labels.Get("service_name") if service == "" { service = "unknown_service" @@ -51,6 +54,7 @@ func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics) *Chunks { chunks: chunkMetrics.chunks.WithLabelValues(service), samples: chunkMetrics.samples.WithLabelValues(service), }, + logger: logger, } } @@ -95,6 +99,15 @@ func (c *Chunks) Iterator( maximumSteps := int64(((through-from)/step)+1) * int64(len(c.chunks)) // prevent a panic if maximumSteps is negative if maximumSteps < 0 { + level.Warn(c.logger).Log( + "msg", "returning an empty series because of a negative maximumSteps", + "labels", lbls.String(), + "from", from, + "through", through, + "step", step, + "maximumSteps", maximumSteps, + "num_chunks", len(c.chunks), + ) series := logproto.Series{ Labels: lbls.String(), Samples: []logproto.Sample{}, @@ -128,6 +141,22 @@ func (c *Chunks) Iterator( return 0 }) + numSamples := 0 + for _, chunk := range c.chunks { + numSamples += len(chunk.Samples) + } + + level.Debug(c.logger).Log( + "msg", "found matching samples", + "samples", samples, + "labels", lbls.String(), + "from", from, + "through", through, + "step", step, + "num_chunks", len(c.chunks), + "num_samples", numSamples, + ) + series := logproto.Series{Labels: lbls.String(), Samples: samples, StreamHash: lbls.Hash()} return iter.NewSeriesIterator(series), nil } diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 0910609c2892..3d2adec4f8a9 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -63,7 +63,7 @@ func newStream( } if cfg.Enabled { - chunks := metric.NewChunks(labels, chunkMetrics) + chunks := metric.NewChunks(labels, chunkMetrics, logger) stream.metrics = chunks stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) } From d3f760be1932b9c38dbb3ce8d1addbd1ea47b4a6 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Wed, 5 Jun 2024 15:11:56 -0600 Subject: [PATCH 17/38] fix: use pointers for chunks and samples, add chunk locking --- pkg/pattern/metric/chunk.go | 34 ++++++++++++++++++++++------------ pkg/pattern/stream.go | 10 +++++----- 2 files changed, 27 insertions(+), 17 deletions(-) diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 388b1421addb..8125df34121b 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -4,6 +4,7 @@ import ( "fmt" "slices" "sort" + "sync" "time" "github.com/go-kit/kit/log/level" @@ -33,11 +34,12 @@ type metrics struct { } type Chunks struct { - chunks []Chunk + chunks []*Chunk labels labels.Labels service string metrics metrics logger log.Logger + lock sync.RWMutex } func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics, logger log.Logger) *Chunks { @@ -47,7 +49,7 @@ func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics, logger log.Logg } return &Chunks{ - chunks: make([]Chunk, 0), + chunks: []*Chunk{}, labels: labels, service: service, metrics: metrics{ @@ -59,6 +61,9 @@ func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics, logger log.Logg } func (c *Chunks) Observe(bytes, count float64, ts model.Time) { + c.lock.Lock() + defer c.lock.Unlock() + c.metrics.samples.Inc() if len(c.chunks) == 0 { @@ -67,7 +72,7 @@ func (c *Chunks) Observe(bytes, count float64, ts model.Time) { return } - last := &(c.chunks)[len(c.chunks)-1] + last := c.chunks[len(c.chunks)-1] if !last.spaceFor(ts) { c.chunks = append(c.chunks, newChunk(bytes, count, ts)) c.metrics.chunks.Set(float64(len(c.chunks))) @@ -86,6 +91,9 @@ func (c *Chunks) Iterator( return nil, fmt.Errorf("unsupported metric type") } + c.lock.RLock() + defer c.lock.RUnlock() + lbls := c.labels if grouping != nil { sort.Strings(grouping.Groups) @@ -148,7 +156,8 @@ func (c *Chunks) Iterator( level.Debug(c.logger).Log( "msg", "found matching samples", - "samples", samples, + "samples", fmt.Sprintf("%v", samples), + "num_samples", len(samples), "labels", lbls.String(), "from", from, "through", through, @@ -167,15 +176,15 @@ type Sample struct { Count float64 } -func newSample(bytes, count float64, ts model.Time) Sample { - return Sample{ +func newSample(bytes, count float64, ts model.Time) *Sample { + return &Sample{ Timestamp: ts, Bytes: bytes, Count: count, } } -type Samples []Sample +type Samples []*Sample type Chunk struct { Samples Samples @@ -186,7 +195,7 @@ func (c *Chunk) Bounds() (fromT, toT time.Time) { return time.Unix(0, c.mint), time.Unix(0, c.maxt) } -func (c *Chunk) AddSample(s Sample) { +func (c *Chunk) AddSample(s *Sample) { c.Samples = append(c.Samples, s) ts := int64(s.Timestamp) @@ -199,10 +208,11 @@ func (c *Chunk) AddSample(s Sample) { } } -func newChunk(bytes, count float64, ts model.Time) Chunk { - maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 - v := Chunk{Samples: make(Samples, 1, maxSize)} - v.Samples[0] = newSample(bytes, count, ts) +func newChunk(bytes, count float64, ts model.Time) *Chunk { + // TODO(twhitney): maybe bring this back when we introduce downsampling + // maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 + v := &Chunk{Samples: Samples{}} + v.Samples = append(v.Samples, newSample(bytes, count, ts)) return v } diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 3d2adec4f8a9..a6ed4935dccd 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -31,8 +31,8 @@ type stream struct { patterns *drain.Drain mtx sync.Mutex - cfg metric.AggregationConfig - metrics *metric.Chunks + cfg metric.AggregationConfig + chunks *metric.Chunks evaluator metric.SampleEvaluatorFactory @@ -64,7 +64,7 @@ func newStream( if cfg.Enabled { chunks := metric.NewChunks(labels, chunkMetrics, logger) - stream.metrics = chunks + stream.chunks = chunks stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) } @@ -91,7 +91,7 @@ func (s *stream) Push( s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) } - if s.cfg.Enabled && s.metrics != nil { + if s.cfg.Enabled && s.chunks != nil { if s.cfg.LogPushObservations { level.Debug(s.logger). Log("msg", "observing pushed log entries", @@ -101,7 +101,7 @@ func (s *stream) Push( "sample_ts_ns", s.lastTs, ) } - s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) + s.chunks.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) } return nil } From a2c601e5537b2e408c16a1f8cf59edbdb857a1bc Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 11 Jun 2024 09:12:28 -0400 Subject: [PATCH 18/38] feat: add sum merge sample iterator --- pkg/iter/sample_iterator.go | 30 +- pkg/pattern/ingester.go | 5 +- pkg/pattern/ingester_querier.go | 4 +- pkg/pattern/ingester_test.go | 12 +- pkg/pattern/instance.go | 2 +- pkg/pattern/instance_test.go | 369 ++++++++++++++---- pkg/pattern/iter/batch.go | 9 +- .../iter/{merge.go => merge_pattern.go} | 0 .../{merge_test.go => merge_pattern_test.go} | 0 pkg/pattern/iter/merge_sample.go | 204 ++++++++++ pkg/pattern/iter/merge_sample_test.go | 252 ++++++++++++ pkg/pattern/iter/query_client.go | 8 +- pkg/pattern/metric/chunk.go | 18 +- pkg/pattern/metric/evaluator_test.go | 6 +- pkg/pattern/stream.go | 14 +- pkg/pattern/stream_test.go | 20 +- 16 files changed, 847 insertions(+), 106 deletions(-) rename pkg/pattern/iter/{merge.go => merge_pattern.go} (100%) rename pkg/pattern/iter/{merge_test.go => merge_pattern_test.go} (100%) create mode 100644 pkg/pattern/iter/merge_sample.go create mode 100644 pkg/pattern/iter/merge_sample_test.go diff --git a/pkg/iter/sample_iterator.go b/pkg/iter/sample_iterator.go index 12947bbdf893..be55678ce628 100644 --- a/pkg/iter/sample_iterator.go +++ b/pkg/iter/sample_iterator.go @@ -119,25 +119,31 @@ func (it *peekingSampleIterator) Error() error { return it.iter.Error() } -type sampleIteratorHeap struct { +type SampleIteratorHeap struct { its []SampleIterator } -func (h sampleIteratorHeap) Len() int { return len(h.its) } -func (h sampleIteratorHeap) Swap(i, j int) { h.its[i], h.its[j] = h.its[j], h.its[i] } -func (h sampleIteratorHeap) Peek() SampleIterator { return h.its[0] } -func (h *sampleIteratorHeap) Push(x interface{}) { +func NewSampleIteratorHeap(its []SampleIterator) SampleIteratorHeap { + return SampleIteratorHeap{ + its: its, + } +} + +func (h SampleIteratorHeap) Len() int { return len(h.its) } +func (h SampleIteratorHeap) Swap(i, j int) { h.its[i], h.its[j] = h.its[j], h.its[i] } +func (h SampleIteratorHeap) Peek() SampleIterator { return h.its[0] } +func (h *SampleIteratorHeap) Push(x interface{}) { h.its = append(h.its, x.(SampleIterator)) } -func (h *sampleIteratorHeap) Pop() interface{} { +func (h *SampleIteratorHeap) Pop() interface{} { n := len(h.its) x := h.its[n-1] h.its = h.its[0 : n-1] return x } -func (h sampleIteratorHeap) Less(i, j int) bool { +func (h SampleIteratorHeap) Less(i, j int) bool { s1, s2 := h.its[i].Sample(), h.its[j].Sample() if s1.Timestamp == s2.Timestamp { if h.its[i].StreamHash() == 0 { @@ -150,7 +156,7 @@ func (h sampleIteratorHeap) Less(i, j int) bool { // mergeSampleIterator iterates over a heap of iterators by merging samples. type mergeSampleIterator struct { - heap *sampleIteratorHeap + heap *SampleIteratorHeap is []SampleIterator prefetched bool stats *stats.Context @@ -170,7 +176,7 @@ type mergeSampleIterator struct { // This means using this iterator with a single iterator will result in the same result as the input iterator. // If you don't need to deduplicate sample, use `NewSortSampleIterator` instead. func NewMergeSampleIterator(ctx context.Context, is []SampleIterator) SampleIterator { - h := sampleIteratorHeap{ + h := SampleIteratorHeap{ its: make([]SampleIterator, 0, len(is)), } return &mergeSampleIterator{ @@ -350,7 +356,7 @@ func (i *mergeSampleIterator) Close() error { // sortSampleIterator iterates over a heap of iterators by sorting samples. type sortSampleIterator struct { - heap *sampleIteratorHeap + heap *SampleIteratorHeap is []SampleIterator prefetched bool @@ -369,7 +375,7 @@ func NewSortSampleIterator(is []SampleIterator) SampleIterator { if len(is) == 1 { return is[0] } - h := sampleIteratorHeap{ + h := SampleIteratorHeap{ its: make([]SampleIterator, 0, len(is)), } return &sortSampleIterator{ @@ -378,7 +384,7 @@ func NewSortSampleIterator(is []SampleIterator) SampleIterator { } } -// init initialize the underlaying heap +// init initialize the underlying heap func (i *sortSampleIterator) init() { if i.prefetched { return diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 831c073fbe2f..75cd1763432f 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -300,7 +300,7 @@ func (i *Ingester) QuerySample( // } defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) - return sendMetricSamples(ctx, iterator, stream) + return sendMetricSamples(ctx, iterator, stream, i.logger) } func sendPatternSample(ctx context.Context, it pattern_iter.Iterator, stream logproto.Pattern_QueryServer) error { @@ -323,9 +323,10 @@ func sendMetricSamples( ctx context.Context, it loki_iter.SampleIterator, stream logproto.Pattern_QuerySampleServer, + logger log.Logger, ) error { for ctx.Err() == nil { - batch, err := pattern_iter.ReadMetricsBatch(it, readBatchSize) + batch, err := pattern_iter.ReadMetricsBatch(it, readBatchSize, logger) if err != nil { return err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index a68707abf07c..dae1905af18a 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -105,7 +105,7 @@ func (q *IngesterQuerier) Samples( } // TODO(twhitney): what should batch size be here? - resp, err := pattern_iter.ReadMetricsBatch(loki_iter.NewSortSampleIterator(iterators), math.MaxInt32) + resp, err := pattern_iter.ReadMetricsBatch(loki_iter.NewSortSampleIterator(iterators), math.MaxInt32, q.logger) if err != nil { return nil, err } @@ -125,7 +125,7 @@ func (q *IngesterQuerier) querySample(ctx context.Context, req *logproto.QuerySa iterators := make([]loki_iter.SampleIterator, len(resps)) for i := range resps { - iterators[i] = pattern_iter.NewQuerySamplesClientIterator(resps[i].response.(logproto.Pattern_QuerySampleClient)) + iterators[i] = pattern_iter.NewQuerySamplesClientIterator(resps[i].response.(logproto.Pattern_QuerySampleClient), q.logger) } return iterators, nil } diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index 35fb37577e80..2af622bc44fa 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -21,9 +21,15 @@ import ( var lbls = labels.New(labels.Label{Name: "test", Value: "test"}) func setup(t *testing.T) *instance { - inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ - Enabled: true, - }) + inst, err := newInstance( + "foo", + log.NewNopLogger(), + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: true, + }, + ) require.NoError(t, err) return inst diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index a67e1506e9ff..1a7d49d07267 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -149,7 +149,7 @@ func (i *instance) QuerySample( return nil, err } - return loki_iter.NewSortSampleIterator(iters), nil + return pattern_iter.NewSumMergeSampleIterator(iters), nil } // forMatchingStreams will execute a function for each stream that matches the given matchers. diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 974e17403ce5..71d90717b15d 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -18,6 +18,22 @@ import ( ) func TestInstance_QuerySample(t *testing.T) { + setupInstance := func() *instance { + instance, err := newInstance( + "test", + log.NewNopLogger(), + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: true, + }, + ) + + require.NoError(t, err) + + return instance + } + ctx := context.Background() thirtySeconds := int64(30000) @@ -32,97 +48,310 @@ func TestInstance_QuerySample(t *testing.T) { Step: oneMin, } - instance, err := newInstance("test", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ - Enabled: true, - }) - require.NoError(t, err) - labels := model.LabelSet{ model.LabelName("foo"): model.LabelValue("bar"), } lastTsMilli := (then + oneMin + thirtySeconds) // 0 + 60000 + 30000 = 90000 - err = instance.Push(ctx, &logproto.PushRequest{ - Streams: []push.Stream{ - { - Labels: labels.String(), - Entries: []push.Entry{ - { - Timestamp: time.UnixMilli(then), - Line: "this=that color=blue", + t.Run("single push", func(t *testing.T) { + instance := setupInstance() + err := instance.Push(ctx, &logproto.PushRequest{ + Streams: []push.Stream{ + { + Labels: labels.String(), + Entries: []push.Entry{ + { + Timestamp: time.UnixMilli(then), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + thirtySeconds), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(lastTsMilli), + Line: "this=that color=blue", + }, }, - { - Timestamp: time.UnixMilli(then + thirtySeconds), - Line: "this=that color=blue", + Hash: uint64(labels.Fingerprint()), + }, + }, + }) + require.NoError(t, err) + + // 5 min query range + // 1 min step + // 1 min selection range + + // first: -60000 to 0 + // second: 0 to 60000 + // third: 60000 to 120000 + // fourth: 120000 to 180000 + // fifth: 180000 to 240000 + // sixth: 240000 to 300000 + + // lastTsMilli is 90000 + // would expect it in the 3rd bucket, but since there's only push, it return + // on the first iteration + start := then + secondPoint := start + oneMin + thirdPoint := secondPoint + oneMin + + t.Run("successful count over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[60s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + next := iter.Next() + require.True(t, next) + + sample := iter.Sample() + require.Equal(t, float64(4), sample.Value) + require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) + + t.Run("successful bytes over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[60s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + next := iter.Next() + require.True(t, next) + + sample := iter.Sample() + require.Equal(t, float64(80), sample.Value) + require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) + }) + + t.Run("multiple streams, multiple pushes", func(t *testing.T) { + instance := setupInstance() + stream1 := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + model.LabelName("fizz"): model.LabelValue("buzz"), + } + stream2 := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + model.LabelName("fizz"): model.LabelValue("bang"), + } + + err := instance.Push(ctx, &logproto.PushRequest{ + Streams: []push.Stream{ + { + Labels: stream1.String(), + Entries: []push.Entry{ + { + Timestamp: time.UnixMilli(then), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + thirtySeconds), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + thirtySeconds), // 60000 + 30000 = 90000 + Line: "this=that color=blue", + }, }, - { - Timestamp: time.UnixMilli(then + oneMin), - Line: "this=that color=blue", + Hash: uint64(stream1.Fingerprint()), + }, + { + Labels: stream2.String(), + Entries: []push.Entry{ + { + Timestamp: time.UnixMilli(then), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + thirtySeconds), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + thirtySeconds), + Line: "this=that color=blue", + }, }, - { - Timestamp: time.UnixMilli(lastTsMilli), - Line: "this=that color=blue", + Hash: uint64(stream2.Fingerprint()), + }, + }, + }) + require.NoError(t, err) + + err = instance.Push(ctx, &logproto.PushRequest{ + Streams: []push.Stream{ + { + Labels: stream1.String(), + Entries: []push.Entry{ + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + thirtySeconds), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + oneMin), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + oneMin + thirtySeconds), // 180000 + 30000 = 210000 + Line: "this=that color=blue", + }, }, + Hash: uint64(stream1.Fingerprint()), + }, + { + Labels: stream2.String(), + Entries: []push.Entry{ + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + thirtySeconds), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + oneMin), + Line: "this=that color=blue", + }, + { + Timestamp: time.UnixMilli(then + oneMin + oneMin + oneMin + thirtySeconds), + Line: "this=that color=blue", + }, + }, + Hash: uint64(stream2.Fingerprint()), }, - Hash: uint64(labels.Fingerprint()), }, - }, - }) - require.NoError(t, err) - - // 5 min query range - // 1 min step - // 1 min selection range - - // first: -60000 to 0 - // second: 0 to 60000 - // third: 60000 to 120000 - // fourth: 120000 to 180000 - // fifth: 180000 to 240000 - // sixth: 240000 to 300000 - - // lastTsMilli is 90000 - // would expect it in the 3rd bucket - start := then - secondPoint := start + oneMin - thirdPoint := secondPoint + oneMin - - t.Run("successful count over time query", func(t *testing.T) { - expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[60s])`) + }) require.NoError(t, err) - iter, err := instance.QuerySample(ctx, expr, mockReq) - assert.NoError(t, err) - assert.NotNil(t, iter) + // steps + start := then + secondStep := start + oneMin // 60000 + thirdStep := secondStep + oneMin // 120000 + fourthStep := thirdStep + oneMin // 180000 + fifthStep := fourthStep + oneMin // 240000 + sixthStep := fifthStep + oneMin // 300000 - next := iter.Next() - require.True(t, next) + // our first push had a timestamp of 90000 (equal to the timestamp of it's last entry) + // therefore our first datapoint will be at 120000, since we have nothing for + // the first step - sample := iter.Sample() - require.Equal(t, float64(4), sample.Value) - require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) + t.Run("successful count over time query with grouping and 1 sample per step and selection range", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`sum by(foo) (count_over_time({foo="bar"}[60s]))`) + require.NoError(t, err) - next = iter.Next() - require.False(t, next) - }) + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) - t.Run("successful bytes over time query", func(t *testing.T) { - expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[60s])`) - require.NoError(t, err) + // test that the grouping is correct + expectedLabels := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + } + + // first sample is at 120000, and should be a sum of the first push to both streams, + // due to the grouping + next := iter.Next() + require.True(t, next) + + sample := iter.Sample() + require.Equal(t, model.Time(thirdStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(8), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) - iter, err := instance.QuerySample(ctx, expr, mockReq) - assert.NoError(t, err) - assert.NotNil(t, iter) + // next should be at 180000 (fourth step), but because of our selection range, we'll have no datapoint here + // so actual next will be 240000 (fifth step), which is late enough to see the second push at 210000 + // this point will be the sum of the second push to both streams + next = iter.Next() + require.True(t, next) - next := iter.Next() - require.True(t, next) + sample = iter.Sample() + require.Equal(t, model.Time(fifthStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(6), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) - sample := iter.Sample() - require.Equal(t, float64(80), sample.Value) - require.Equal(t, model.Time(thirdPoint).UnixNano(), sample.Timestamp) + next = iter.Next() + require.False(t, next) + }) - next = iter.Next() - require.False(t, next) + t.Run( + "successful count over time query with grouping and multiple samples per step and selection range", + func(t *testing.T) { + // with a 5m slection range we should get samples from both pushes + expr, err := syntax.ParseSampleExpr( + `sum by(foo) (count_over_time({foo="bar"}[5m]))`, + ) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + // test that the grouping is correct + expectedLabels := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + } + + // the first datapoint is again at 1200000, but from the second stream + next := iter.Next() + require.True(t, next) + + sample := iter.Sample() + require.Equal(t, model.Time(thirdStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(8), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) + + // next will be the second step, which still only has the first push in it's selection range + next = iter.Next() + require.True(t, next) + + sample = iter.Sample() + require.Equal(t, model.Time(fourthStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(8), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) + + // next will be the second push, which has both pushes in it's selection range + next = iter.Next() + require.True(t, next) + + sample = iter.Sample() + require.Equal(t, model.Time(fifthStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(14), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) + + // since our timr range through goes to 310000, we will have 1 more data point + next = iter.Next() + require.True(t, next) + + sample = iter.Sample() + require.Equal(t, model.Time(sixthStep).UnixNano(), sample.Timestamp) + require.Equal(t, float64(14), sample.Value) + require.Equal(t, expectedLabels.String(), iter.Labels()) + + // there should be no more samples + next = iter.Next() + require.False(t, next) + }, + ) }) } diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index fee835d16208..0d99fb741f7a 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -1,8 +1,11 @@ package iter import ( + "fmt" "math" + "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) @@ -34,7 +37,7 @@ func ReadAll(it Iterator) (*logproto.QueryPatternsResponse, error) { return ReadBatch(it, math.MaxInt32) } -func ReadMetricsBatch(it iter.SampleIterator, batchSize int) (*logproto.QuerySamplesResponse, error) { +func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) (*logproto.QuerySamplesResponse, error) { var ( series = map[uint64]*logproto.Series{} respSize int @@ -59,11 +62,13 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int) (*logproto.QuerySam Series: make([]logproto.Series, 0, len(series)), } for _, s := range series { + level.Debug(logger).Log("msg", "appending series", "s", fmt.Sprintf("%v", s)) result.Series = append(result.Series, *s) } return &result, it.Error() } +// ReadAllSamples reads all samples from the given iterator. It is only used in tests. func ReadAllSamples(it iter.SampleIterator) (*logproto.QuerySamplesResponse, error) { - return ReadMetricsBatch(it, math.MaxInt32) + return ReadMetricsBatch(it, math.MaxInt32, log.NewNopLogger()) } diff --git a/pkg/pattern/iter/merge.go b/pkg/pattern/iter/merge_pattern.go similarity index 100% rename from pkg/pattern/iter/merge.go rename to pkg/pattern/iter/merge_pattern.go diff --git a/pkg/pattern/iter/merge_test.go b/pkg/pattern/iter/merge_pattern_test.go similarity index 100% rename from pkg/pattern/iter/merge_test.go rename to pkg/pattern/iter/merge_pattern_test.go diff --git a/pkg/pattern/iter/merge_sample.go b/pkg/pattern/iter/merge_sample.go new file mode 100644 index 000000000000..0da6f25df9e8 --- /dev/null +++ b/pkg/pattern/iter/merge_sample.go @@ -0,0 +1,204 @@ +package iter + +import ( + "container/heap" + + "github.com/grafana/loki/v3/pkg/iter" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/util" +) + +type sampleWithLabels struct { + logproto.Sample + labels string + streamHash uint64 +} + +// sumMergeSampleIterator iterates over a heap of iterators by merging samples. +type sumMergeSampleIterator struct { + heap *iter.SampleIteratorHeap + is []iter.SampleIterator + prefetched bool + // pushBuffer contains the list of iterators that needs to be pushed to the heap + // This is to avoid allocations. + pushBuffer []iter.SampleIterator + + // buffer of entries to be returned by Next() + // We buffer entries with the same timestamp to correctly dedupe them. + buffer []sampleWithLabels + curr sampleWithLabels + errs []error +} + +// NewSumMergeSampleIterator is a lot like iter.NewMergeSampleIterator, with 2 notable +// difference. For one, it does not care about the hash value of a Sample as it is +// assuming these samples are not coming from log lines, nor are in need of de-duplication. +// Second, when there are two identical samples from the same series and time, it sums the +// values. +// NewSumMergeSampleIterator returns an iterator which uses a heap to merge together samples for multiple iterators. +// The iterator only order and merge entries across given `is` iterators, it does not merge entries within individual iterator. +// This means using this iterator with a single iterator will result in the same result as the input iterator. +func NewSumMergeSampleIterator(is []iter.SampleIterator) iter.SampleIterator { + h := iter.NewSampleIteratorHeap(make([]iter.SampleIterator, 0, len(is))) + return &sumMergeSampleIterator{ + is: is, + heap: &h, + buffer: make([]sampleWithLabels, 0, len(is)), + pushBuffer: make([]iter.SampleIterator, 0, len(is)), + } +} + +// prefetch iterates over all inner iterators to merge together, calls Next() on +// each of them to prefetch the first entry and pushes of them - who are not +// empty - to the heap +func (i *sumMergeSampleIterator) prefetch() { + if i.prefetched { + return + } + + i.prefetched = true + for _, it := range i.is { + i.requeue(it, false) + } + + // We can now clear the list of input iterators to merge, given they have all + // been processed and the non empty ones have been pushed to the heap + i.is = nil +} + +// requeue pushes the input ei EntryIterator to the heap, advancing it via an ei.Next() +// call unless the advanced input parameter is true. In this latter case it expects that +// the iterator has already been advanced before calling requeue(). +// +// If the iterator has no more entries or an error occur while advancing it, the iterator +// is not pushed to the heap and any possible error captured, so that can be get via Error(). +func (i *sumMergeSampleIterator) requeue(ei iter.SampleIterator, advanced bool) { + if advanced || ei.Next() { + heap.Push(i.heap, ei) + return + } + + if err := ei.Error(); err != nil { + i.errs = append(i.errs, err) + } + util.LogError("closing iterator", ei.Close) +} + +func (i *sumMergeSampleIterator) Next() bool { + i.prefetch() + + if len(i.buffer) != 0 { + i.nextFromBuffer() + return true + } + + if i.heap.Len() == 0 { + return false + } + + // shortcut for the last iterator. + if i.heap.Len() == 1 { + i.curr.Sample = i.heap.Peek().Sample() + i.curr.labels = i.heap.Peek().Labels() + i.curr.streamHash = i.heap.Peek().StreamHash() + if !i.heap.Peek().Next() { + i.heap.Pop() + } + return true + } + + // We support multiple entries with the same timestamp, and we want to + // preserve their original order. We look at all the top entries in the + // heap with the same timestamp, and add them to the buffer to sum their values. + for i.heap.Len() > 0 { + next := i.heap.Peek() + sample := next.Sample() + + if len(i.buffer) > 0 && (i.buffer[0].streamHash != next.StreamHash() || + i.buffer[0].Timestamp != sample.Timestamp) { + break + } + heap.Pop(i.heap) + i.buffer = append(i.buffer, sampleWithLabels{ + Sample: sample, + labels: next.Labels(), + streamHash: next.StreamHash(), + }) + + if next.Next() { + i.pushBuffer = append(i.pushBuffer, next) + } + } + + for _, ei := range i.pushBuffer { + heap.Push(i.heap, ei) + } + i.pushBuffer = i.pushBuffer[:0] + + i.nextFromBuffer() + + return true +} + +func (i *sumMergeSampleIterator) nextFromBuffer() { + if len(i.buffer) == 1 { + i.curr.Sample = i.buffer[0].Sample + i.curr.labels = i.buffer[0].labels + i.curr.streamHash = i.buffer[0].streamHash + i.buffer = i.buffer[:0] + return + } + + mergedSample := i.buffer[0] + + numSamples := 1 + for _, sample := range i.buffer[1:] { + if mergedSample.labels != sample.labels || + mergedSample.streamHash != sample.streamHash || + mergedSample.Timestamp != sample.Timestamp { + i.curr = mergedSample + i.buffer = i.buffer[numSamples:] + return + } + + mergedSample.Sample.Value += sample.Value + numSamples++ + } + + i.curr = mergedSample + i.buffer = i.buffer[numSamples:] + return +} + +func (i *sumMergeSampleIterator) Sample() logproto.Sample { + return i.curr.Sample +} + +func (i *sumMergeSampleIterator) Labels() string { + return i.curr.labels +} + +func (i *sumMergeSampleIterator) StreamHash() uint64 { + return i.curr.streamHash +} + +func (i *sumMergeSampleIterator) Error() error { + switch len(i.errs) { + case 0: + return nil + case 1: + return i.errs[0] + default: + return util.MultiError(i.errs) + } +} + +func (i *sumMergeSampleIterator) Close() error { + for i.heap.Len() > 0 { + if err := i.heap.Pop().(iter.SampleIterator).Close(); err != nil { + return err + } + } + i.buffer = nil + return nil +} diff --git a/pkg/pattern/iter/merge_sample_test.go b/pkg/pattern/iter/merge_sample_test.go new file mode 100644 index 000000000000..5864f4aaafb8 --- /dev/null +++ b/pkg/pattern/iter/merge_sample_test.go @@ -0,0 +1,252 @@ +package iter + +import ( + "hash/fnv" + "testing" + + "github.com/grafana/loki/v3/pkg/iter" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/stretchr/testify/require" +) + +func TestNewSumMergeSampleIterator(t *testing.T) { + t.Run("with labels -- no merge", func(t *testing.T) { + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(varSeries), + iter.NewSeriesIterator(carSeries), + }) + + for i := 1; i < 4; i++ { + require.True(t, it.Next(), i) + require.Equal(t, `{foo="car"}`, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i)), it.Sample(), i) + require.True(t, it.Next(), i) + require.Equal(t, `{foo="var"}`, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i)), it.Sample(), i) + } + require.False(t, it.Next()) + require.NoError(t, it.Error()) + require.NoError(t, it.Close()) + }) + + t.Run("with labels -- merge", func(t *testing.T) { + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(varSeries), + iter.NewSeriesIterator(carSeries), + iter.NewSeriesIterator(varSeries), + iter.NewSeriesIterator(carSeries), + iter.NewSeriesIterator(varSeries), + iter.NewSeriesIterator(carSeries), + }) + + for i := 1; i < 4; i++ { + require.True(t, it.Next(), i) + require.Equal(t, `{foo="car"}`, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i*3)), it.Sample(), i) + require.True(t, it.Next(), i) + require.Equal(t, `{foo="var"}`, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i*3)), it.Sample(), i) + } + require.False(t, it.Next()) + require.NoError(t, it.Error()) + require.NoError(t, it.Close()) + }) + + t.Run("no labels", func(t *testing.T) { + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: carSeries.StreamHash, + Samples: carSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: varSeries.StreamHash, + Samples: varSeries.Samples, + }), + }) + + for i := 1; i < 4; i++ { + require.True(t, it.Next(), i) + require.Equal(t, ``, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i)), it.Sample(), i) + require.True(t, it.Next(), i) + require.Equal(t, ``, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i)), it.Sample(), i) + } + require.False(t, it.Next()) + require.NoError(t, it.Error()) + require.NoError(t, it.Close()) + }) + + t.Run("no labels -- merge", func(t *testing.T) { + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: carSeries.StreamHash, + Samples: carSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: varSeries.StreamHash, + Samples: varSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: carSeries.StreamHash, + Samples: carSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: varSeries.StreamHash, + Samples: varSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: carSeries.StreamHash, + Samples: carSeries.Samples, + }), + iter.NewSeriesIterator(logproto.Series{ + Labels: ``, + StreamHash: varSeries.StreamHash, + Samples: varSeries.Samples, + }), + }) + + for i := 1; i < 4; i++ { + require.True(t, it.Next(), i) + require.Equal(t, ``, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i*3)), it.Sample(), i) + require.True(t, it.Next(), i) + require.Equal(t, ``, it.Labels(), i) + require.Equal(t, sample(int64(i), float64(i*3)), it.Sample(), i) + } + require.False(t, it.Next()) + require.NoError(t, it.Error()) + require.NoError(t, it.Close()) + }) + t.Run("it sums the values from two identical points", func(t *testing.T) { + series := logproto.Series{ + Labels: `{foo="bar"}`, + StreamHash: hashLabels(`{foo="bar"}`), + Samples: []logproto.Sample{ + sample(1, 1), sample(2, 2), sample(3, 3), + }, + } + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(series), + iter.NewSeriesIterator(series), + }) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(1, 2), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(2, 4), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(3, 6), it.Sample()) + + require.False(t, it.Next()) + }) + t.Run("it sums the values from two streams with different data points", func(t *testing.T) { + series1 := logproto.Series{ + Labels: `{foo="bar"}`, + StreamHash: hashLabels(`{foo="bar"}`), + Samples: []logproto.Sample{ + sample(1, 1), sample(2, 2), sample(3, 3), + }, + } + series2 := logproto.Series{ + Labels: `{foo="baz"}`, + StreamHash: hashLabels(`{foo="baz"}`), + Samples: []logproto.Sample{ + sample(1, 1), sample(2, 2), sample(3, 4), + }, + } + series3 := logproto.Series{ + Labels: `{foo="bar"}`, + StreamHash: hashLabels(`{foo="bar"}`), + Samples: []logproto.Sample{ + sample(2, 2), sample(4, 4), + }, + } + it := NewSumMergeSampleIterator( + []iter.SampleIterator{ + iter.NewSeriesIterator(series1), + iter.NewSeriesIterator(series2), + iter.NewSeriesIterator(series3), + }) + + require.True(t, it.Next()) + require.Equal(t, `{foo="baz"}`, it.Labels()) + require.Equal(t, sample(1, 1), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(1, 1), it.Sample()) // first only + + require.True(t, it.Next()) + require.Equal(t, `{foo="baz"}`, it.Labels()) + require.Equal(t, sample(2, 2), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(2, 4), it.Sample()) // merged + + require.True(t, it.Next()) + require.Equal(t, `{foo="baz"}`, it.Labels()) + require.Equal(t, sample(3, 4), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(3, 3), it.Sample()) + + require.True(t, it.Next()) + require.Equal(t, `{foo="bar"}`, it.Labels()) + require.Equal(t, sample(4, 4), it.Sample()) // second only + + require.False(t, it.Next()) + }) +} + +var varSeries = logproto.Series{ + Labels: `{foo="var"}`, + StreamHash: hashLabels(`{foo="var"}`), + Samples: []logproto.Sample{ + sample(1, 1), sample(2, 2), sample(3, 3), + }, +} + +var carSeries = logproto.Series{ + Labels: `{foo="car"}`, + StreamHash: hashLabels(`{foo="car"}`), + Samples: []logproto.Sample{ + sample(1, 1), sample(2, 2), sample(3, 3), + }, +} + +func sample(t int64, v float64) logproto.Sample { + // The main difference between this MergeSampleIterator and the one from + // v3/pkg/iter is that this one does not care about the Sample's hash + // since it is not coming from a log line. + return logproto.Sample{ + Timestamp: t, + Hash: uint64(42), + Value: v, + } +} + +func hashLabels(lbs string) uint64 { + h := fnv.New64a() + h.Write([]byte(lbs)) + return h.Sum64() +} diff --git a/pkg/pattern/iter/query_client.go b/pkg/pattern/iter/query_client.go index f471999d62e3..243069903fdc 100644 --- a/pkg/pattern/iter/query_client.go +++ b/pkg/pattern/iter/query_client.go @@ -1,8 +1,11 @@ package iter import ( + "fmt" "io" + "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) @@ -66,20 +69,23 @@ func NewQueryResponseIterator(resp *logproto.QueryPatternsResponse) Iterator { type querySamplesClientIterator struct { client logproto.Pattern_QuerySampleClient + logger log.Logger err error curr iter.SampleIterator } // NewQueryClientIterator returns an iterator over a QueryClient. -func NewQuerySamplesClientIterator(client logproto.Pattern_QuerySampleClient) iter.SampleIterator { +func NewQuerySamplesClientIterator(client logproto.Pattern_QuerySampleClient, logger log.Logger) iter.SampleIterator { return &querySamplesClientIterator{ client: client, + logger: logger, } } func (i *querySamplesClientIterator) Next() bool { for i.curr == nil || !i.curr.Next() { batch, err := i.client.Recv() + level.Debug(i.logger).Log("msg", "received batch", "batch", fmt.Sprintf("%v", batch)) if err == io.EOF { return false } else if err != nil { diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 8125df34121b..2686642545f1 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -48,6 +48,12 @@ func NewChunks(labels labels.Labels, chunkMetrics *ChunkMetrics, logger log.Logg service = "unknown_service" } + level.Debug(logger).Log( + "msg", "creating new chunks", + "labels", labels.String(), + "service", service, + ) + return &Chunks{ chunks: []*Chunk{}, labels: labels, @@ -157,7 +163,7 @@ func (c *Chunks) Iterator( level.Debug(c.logger).Log( "msg", "found matching samples", "samples", fmt.Sprintf("%v", samples), - "num_samples", len(samples), + "found_samples", len(samples), "labels", lbls.String(), "from", from, "through", through, @@ -176,15 +182,15 @@ type Sample struct { Count float64 } -func newSample(bytes, count float64, ts model.Time) *Sample { - return &Sample{ +func newSample(bytes, count float64, ts model.Time) Sample { + return Sample{ Timestamp: ts, Bytes: bytes, Count: count, } } -type Samples []*Sample +type Samples []Sample type Chunk struct { Samples Samples @@ -195,7 +201,7 @@ func (c *Chunk) Bounds() (fromT, toT time.Time) { return time.Unix(0, c.mint), time.Unix(0, c.maxt) } -func (c *Chunk) AddSample(s *Sample) { +func (c *Chunk) AddSample(s Sample) { c.Samples = append(c.Samples, s) ts := int64(s.Timestamp) @@ -211,7 +217,7 @@ func (c *Chunk) AddSample(s *Sample) { func newChunk(bytes, count float64, ts model.Time) *Chunk { // TODO(twhitney): maybe bring this back when we introduce downsampling // maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 - v := &Chunk{Samples: Samples{}} + v := &Chunk{Samples: []Sample{}} v.Samples = append(v.Samples, newSample(bytes, count, ts)) return v } diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 1f9683d214b3..0e9c6f0f1ce8 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -26,8 +26,8 @@ func Test_SampleEvaluator(t *testing.T) { }, } - setup := func(chunks Chunks, from, through, step int64, query string) logql.StepEvaluator { - factory := NewDefaultEvaluatorFactory(&chunks) + setup := func(chunks *Chunks, from, through, step int64, query string) logql.StepEvaluator { + factory := NewDefaultEvaluatorFactory(chunks) expr, err := syntax.ParseSampleExpr(query) require.NoError(t, err) @@ -49,7 +49,7 @@ func Test_SampleEvaluator(t *testing.T) { thenTime := model.Time(then) beforeThenTime := model.Time(beforeThen) return Chunks{ - chunks: []Chunk{ + chunks: []*Chunk{ { Samples: []Sample{ { diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index a6ed4935dccd..a0fef7ebe5e8 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -2,6 +2,7 @@ package pattern import ( "context" + "fmt" "math" "sync" "time" @@ -152,7 +153,7 @@ func (s *stream) SampleIterator( // this is only 1 series since we're already on a stream // this this limit needs to also be enforced higher up maxSeries := 1000 - matrix, err := s.JoinSampleVectors( + matrix, err := s.joinSampleVectors( next, ts, r, @@ -166,7 +167,7 @@ func (s *stream) SampleIterator( return loki_iter.NewMultiSeriesIterator(matrix), nil } -func (s *stream) JoinSampleVectors( +func (s *stream) joinSampleVectors( next bool, ts int64, r logql.StepResult, @@ -228,6 +229,15 @@ func (s *stream) JoinSampleVectors( matrix = append(matrix, *s) } + level.Debug(s.logger).Log( + "msg", "joined sample vectors", + "num_series", len(matrix), + "matrix", fmt.Sprintf("%v", matrix), + "from", from, + "through", through, + "step", step, + ) + return matrix, stepEvaluator.Error() } diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index 65da8dc497a5..8f7ec19759ae 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -5,18 +5,29 @@ import ( "testing" "time" + "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/pkg/push" ) func TestAddStream(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test"), false) + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: false, + }, + log.NewNopLogger(), + ) require.NoError(t, err) err = stream.Push(context.Background(), []push.Entry{ @@ -48,7 +59,11 @@ func TestPruneStream(t *testing.T) { model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test"), - false, + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: false, + }, + log.NewNopLogger(), ) require.NoError(t, err) @@ -80,3 +95,4 @@ func TestPruneStream(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) } + From f1cfd813c9f3bcd5b95750f7d4fd67307e690640 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 11 Jun 2024 13:05:01 -0400 Subject: [PATCH 19/38] test: read metric batch test --- pkg/pattern/iter/batch_test.go | 90 ++++++++++++++++++++++++++++++++-- 1 file changed, 87 insertions(+), 3 deletions(-) diff --git a/pkg/pattern/iter/batch_test.go b/pkg/pattern/iter/batch_test.go index 3ee565892da4..be845dd15f4b 100644 --- a/pkg/pattern/iter/batch_test.go +++ b/pkg/pattern/iter/batch_test.go @@ -3,8 +3,10 @@ package iter import ( "testing" + "github.com/go-kit/log" "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) @@ -43,9 +45,13 @@ func TestReadBatch(t *testing.T) { }, }, { - name: "ReadBatch more than batchSize", - pattern: "foo", - samples: []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}}, + name: "ReadBatch more than batchSize", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, batchSize: 4, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ @@ -71,3 +77,81 @@ func TestReadBatch(t *testing.T) { }) } } + +func TestReadMetricsBatch(t *testing.T) { + tests := []struct { + name string + pattern string + series logproto.Series + batchSize int + expected *logproto.QuerySamplesResponse + }{ + { + name: "ReadBatch empty iterator", + series: logproto.Series{ + Labels: "", + Samples: []logproto.Sample{}, + }, + batchSize: 2, + expected: &logproto.QuerySamplesResponse{ + Series: []logproto.Series{}, + }, + }, + { + name: "ReadBatch less than batchSize", + series: logproto.Series{ + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + }, + batchSize: 2, + expected: &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + }, + }, + }, + }, + }, + { + name: "ReadBatch more than batchSize", + series: logproto.Series{ + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + }, + batchSize: 4, + expected: &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + it := iter.NewSeriesIterator(tt.series) + got, err := ReadMetricsBatch(it, tt.batchSize, log.NewNopLogger()) + require.NoError(t, err) + require.Equal(t, tt.expected.Series, got.Series) + }) + } +} From f42f52366d33e897095e7d23344459925b3e5334 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Wed, 12 Jun 2024 14:07:36 -0400 Subject: [PATCH 20/38] fix: formatting --- pkg/pattern/ingester.go | 2 +- pkg/pattern/iter/batch.go | 1 + pkg/pattern/iter/merge_sample_test.go | 3 ++- pkg/pattern/iter/query_client.go | 1 + pkg/pattern/metric/config.go | 4 ++-- pkg/pattern/stream.go | 1 + pkg/pattern/stream_test.go | 1 - 7 files changed, 8 insertions(+), 5 deletions(-) diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 75cd1763432f..0561f0eafe2a 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -323,7 +323,7 @@ func sendMetricSamples( ctx context.Context, it loki_iter.SampleIterator, stream logproto.Pattern_QuerySampleServer, - logger log.Logger, + logger log.Logger, ) error { for ctx.Err() == nil { batch, err := pattern_iter.ReadMetricsBatch(it, readBatchSize, logger) diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index 0d99fb741f7a..444b4c904e1b 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -6,6 +6,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) diff --git a/pkg/pattern/iter/merge_sample_test.go b/pkg/pattern/iter/merge_sample_test.go index 5864f4aaafb8..3f619ca44c3a 100644 --- a/pkg/pattern/iter/merge_sample_test.go +++ b/pkg/pattern/iter/merge_sample_test.go @@ -4,9 +4,10 @@ import ( "hash/fnv" "testing" + "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" - "github.com/stretchr/testify/require" ) func TestNewSumMergeSampleIterator(t *testing.T) { diff --git a/pkg/pattern/iter/query_client.go b/pkg/pattern/iter/query_client.go index 243069903fdc..997a2623b6f1 100644 --- a/pkg/pattern/iter/query_client.go +++ b/pkg/pattern/iter/query_client.go @@ -6,6 +6,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) diff --git a/pkg/pattern/metric/config.go b/pkg/pattern/metric/config.go index 5c5fa4a54a29..7e64d7573ffa 100644 --- a/pkg/pattern/metric/config.go +++ b/pkg/pattern/metric/config.go @@ -3,7 +3,7 @@ package metric import "flag" type AggregationConfig struct { - Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` + Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` LogPushObservations bool `yaml:"log_push_observations,omitempty" doc:"description=Wheter to log push observations."` } @@ -14,5 +14,5 @@ func (cfg *AggregationConfig) RegisterFlags(fs *flag.FlagSet) { func (cfg *AggregationConfig) RegisterFlagsWithPrefix(fs *flag.FlagSet, prefix string) { fs.BoolVar(&cfg.Enabled, prefix+"metric-aggregation.enabled", false, "Flag to enable or disable metric aggregation.") - fs.BoolVar(&cfg.LogPushObservations, prefix+"metric-aggregation.log-push-observations", false, "Flag to enable or disable logging of push observations.") + fs.BoolVar(&cfg.LogPushObservations, prefix+"metric-aggregation.log-push-observations", false, "Flag to enable or disable logging of push observations.") } diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index a0fef7ebe5e8..484e6edde0d0 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -9,6 +9,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index 8f7ec19759ae..ed08d6eebdb6 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -95,4 +95,3 @@ func TestPruneStream(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) } - From 5c0abde9a78ca01baf6a841956a5684aa6e41df4 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 13 Jun 2024 11:00:08 -0400 Subject: [PATCH 21/38] fix: more linting --- go.mod | 2 +- pkg/canary/comparator/comparator_test.go | 2 +- pkg/logproto/compat.go | 16 ++++++++++------ pkg/pattern/iter/merge_sample.go | 1 - pkg/pattern/metric/chunk.go | 2 +- pkg/pattern/metric/chunk_test.go | 4 ++-- pkg/pattern/metric/config.go | 2 +- pkg/pattern/metric/evaluator_test.go | 4 ++-- 8 files changed, 18 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index 051a18d2292f..fe9c50bcb93b 100644 --- a/go.mod +++ b/go.mod @@ -229,7 +229,7 @@ require ( github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/go-kit/kit v0.12.0 // indirect + github.com/go-kit/kit v0.12.0 github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-openapi/analysis v0.22.2 // indirect diff --git a/pkg/canary/comparator/comparator_test.go b/pkg/canary/comparator/comparator_test.go index db28545397e3..2b441b709a2a 100644 --- a/pkg/canary/comparator/comparator_test.go +++ b/pkg/canary/comparator/comparator_test.go @@ -386,7 +386,7 @@ func TestCacheTest(t *testing.T) { queryResultsDiff = &mockCounter{} // reset counter mr.countOverTime = 2.3 // value not important - mr.noCacheCountOvertime = 2.30000005 // different than `countOverTime` value but withing tolerance + mr.noCacheCountOvertime = 2.30000005 // different than `countOverTime` value but within tolerance c.cacheTest(now) assert.Equal(t, 0, queryResultsDiff.(*mockCounter).count) diff --git a/pkg/logproto/compat.go b/pkg/logproto/compat.go index 623b483257eb..fd2923b7a06c 100644 --- a/pkg/logproto/compat.go +++ b/pkg/logproto/compat.go @@ -588,11 +588,10 @@ func (m *DetectedLabelsRequest) WithStartEndForCache(start, end time.Time) resul } func (m *DetectedLabelsRequest) LogToSpan(sp opentracing.Span) { - fields := []otlog.Field{ + fields := []otlog.Field{ otlog.String("query", m.GetQuery()), otlog.String("start", m.Start.String()), otlog.String("end", m.End.String()), - otlog.String("step", time.Duration(m.Step).String()), } sp.LogFields(fields...) } @@ -600,16 +599,21 @@ func (m *DetectedLabelsRequest) LogToSpan(sp opentracing.Span) { func (m *QuerySamplesRequest) GetCachingOptions() (res definitions.CachingOptions) { return } func (m *QuerySamplesRequest) WithStartEnd(start, end time.Time) definitions.Request { + clone := *m + clone.Start = start + clone.End = end + return &clone +} func (m *QuerySamplesRequest) WithStartEndForCache(start, end time.Time) resultscache.Request { return m.WithStartEnd(start, end).(resultscache.Request) } - + func (m *QuerySamplesRequest) WithQuery(query string) definitions.Request { - clone := *m + clone := *m clone.Query = query return &clone -} +} func (m *QuerySamplesRequest) LogToSpan(sp opentracing.Span) { fields := []otlog.Field{ @@ -619,4 +623,4 @@ func (m *QuerySamplesRequest) LogToSpan(sp opentracing.Span) { otlog.String("step", time.Duration(m.Step).String()), } sp.LogFields(fields...) -} \ No newline at end of file +} diff --git a/pkg/pattern/iter/merge_sample.go b/pkg/pattern/iter/merge_sample.go index 0da6f25df9e8..7e8f250b48f2 100644 --- a/pkg/pattern/iter/merge_sample.go +++ b/pkg/pattern/iter/merge_sample.go @@ -167,7 +167,6 @@ func (i *sumMergeSampleIterator) nextFromBuffer() { i.curr = mergedSample i.buffer = i.buffer[numSamples:] - return } func (i *sumMergeSampleIterator) Sample() logproto.Sample { diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 2686642545f1..8cd8e2511ad7 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -7,8 +7,8 @@ import ( "sync" "time" - "github.com/go-kit/kit/log/level" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index b89e4942dc30..26fe33d8e408 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -303,7 +303,7 @@ func Test_Chunks_Iterator(t *testing.T) { labels.Label{Name: "container", Value: "jar"}, } chunks := Chunks{ - chunks: []Chunk{ + chunks: []*Chunk{ { Samples: []Sample{ {Timestamp: 2, Bytes: 2, Count: 1}, @@ -403,7 +403,7 @@ func Test_Chunks_Iterator(t *testing.T) { t.Run("handle slice capacity out of range", func(t *testing.T) { chunks := Chunks{ - chunks: []Chunk{ + chunks: []*Chunk{ { Samples: []Sample{}, }, diff --git a/pkg/pattern/metric/config.go b/pkg/pattern/metric/config.go index 7e64d7573ffa..7a7737368f0d 100644 --- a/pkg/pattern/metric/config.go +++ b/pkg/pattern/metric/config.go @@ -4,7 +4,7 @@ import "flag" type AggregationConfig struct { Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` - LogPushObservations bool `yaml:"log_push_observations,omitempty" doc:"description=Wheter to log push observations."` + LogPushObservations bool `yaml:"log_push_observations,omitempty" doc:"description=Whether to log push observations."` } // RegisterFlags registers pattern ingester related flags. diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index 0e9c6f0f1ce8..c88a4466bfd5 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -44,11 +44,11 @@ func Test_SampleEvaluator(t *testing.T) { return evaluator } - chunks := func(now, then, beforeThen int64) Chunks { + chunks := func(now, then, beforeThen int64) *Chunks { nowTime := model.Time(now) thenTime := model.Time(then) beforeThenTime := model.Time(beforeThen) - return Chunks{ + return &Chunks{ chunks: []*Chunk{ { Samples: []Sample{ From 0f7e473c68638988bfcc8ba093c75b30b4b166a0 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 13 Jun 2024 11:04:27 -0400 Subject: [PATCH 22/38] fix: check-mod --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index fe9c50bcb93b..051a18d2292f 100644 --- a/go.mod +++ b/go.mod @@ -229,7 +229,7 @@ require ( github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/go-kit/kit v0.12.0 + github.com/go-kit/kit v0.12.0 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-openapi/analysis v0.22.2 // indirect From ac83fcb37295a9ef4cb54d77aa506f5173314200 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 13 Jun 2024 11:27:56 -0400 Subject: [PATCH 23/38] fix: mod again --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 051a18d2292f..263c4b24c5d1 100644 --- a/go.mod +++ b/go.mod @@ -151,6 +151,7 @@ require ( require ( github.com/dlclark/regexp2 v1.4.0 // indirect + github.com/go-kit/kit v0.12.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect @@ -229,7 +230,6 @@ require ( github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/go-kit/kit v0.12.0 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-openapi/analysis v0.22.2 // indirect From e5e23c85af266cf52329266a70f7dc84e418752c Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 13 Jun 2024 12:38:40 -0400 Subject: [PATCH 24/38] fix: chunk test --- pkg/pattern/metric/chunk_test.go | 32 +++++++++++++++----------------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 26fe33d8e408..206b5c673a97 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -4,6 +4,7 @@ import ( "reflect" "testing" + "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -302,19 +303,17 @@ func Test_Chunks_Iterator(t *testing.T) { labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "container", Value: "jar"}, } - chunks := Chunks{ - chunks: []*Chunk{ - { - Samples: []Sample{ - {Timestamp: 2, Bytes: 2, Count: 1}, - {Timestamp: 4, Bytes: 4, Count: 3}, - {Timestamp: 6, Bytes: 6, Count: 5}, - }, - mint: 2, - maxt: 6, + chunks := NewChunks(lbls, NewChunkMetrics(nil, "test"), log.NewNopLogger()) + chunks.chunks = []*Chunk{ + { + Samples: []Sample{ + {Timestamp: 2, Bytes: 2, Count: 1}, + {Timestamp: 4, Bytes: 4, Count: 3}, + {Timestamp: 6, Bytes: 6, Count: 5}, }, + mint: 2, + maxt: 6, }, - labels: lbls, } t.Run("without grouping", func(t *testing.T) { @@ -402,14 +401,13 @@ func Test_Chunks_Iterator(t *testing.T) { }) t.Run("handle slice capacity out of range", func(t *testing.T) { - chunks := Chunks{ - chunks: []*Chunk{ - { - Samples: []Sample{}, - }, + chunks := NewChunks(lbls, NewChunkMetrics(nil, "test"), log.NewNopLogger()) + chunks.chunks = []*Chunk{ + { + Samples: []Sample{}, }, - labels: lbls, } + it, err := chunks.Iterator(Bytes, nil, 5e4, 0, 1e4) require.NoError(t, err) From 721e7c1bbf3e73ba4e226037228d21e470d5831e Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 13 Jun 2024 12:49:00 -0400 Subject: [PATCH 25/38] fix: docs --- docs/sources/shared/configuration.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 94aaf44ee476..444734d94147 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -323,6 +323,10 @@ pattern_ingester: # CLI flag: -pattern-ingester.metric-aggregation.enabled [enabled: | default = false] + # Whether to log push observations. + # CLI flag: -pattern-ingester.metric-aggregation.log-push-observations + [log_push_observations: | default = false] + # The index_gateway block configures the Loki index gateway server, responsible # for serving index queries without the need to constantly interact with the # object store. From 12b36361f48e3201d9132a11daefec8bafc70caa Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 14 Jun 2024 08:43:27 -0400 Subject: [PATCH 26/38] fix: logger in sample evaluator tests --- pkg/pattern/metric/evaluator_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go index c88a4466bfd5..ee8001a1e10f 100644 --- a/pkg/pattern/metric/evaluator_test.go +++ b/pkg/pattern/metric/evaluator_test.go @@ -4,6 +4,7 @@ import ( "context" "testing" + "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -73,6 +74,7 @@ func Test_SampleEvaluator(t *testing.T) { }, }, labels: stream, + logger: log.NewNopLogger(), } } From 1133763ba71f2583b36a7b4af17abeadc34b1a01 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 17 Jun 2024 09:03:17 -0600 Subject: [PATCH 27/38] chore: more debug logging --- pkg/pattern/ingester.go | 1 + pkg/pattern/instance.go | 2 ++ 2 files changed, 3 insertions(+) diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 0561f0eafe2a..4d898a1a8524 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -275,6 +275,7 @@ func (i *Ingester) QuerySample( return err } + level.Info(i.logger).Log("msg", "QuerySample", "instanceID", instanceID, "expr", expr) iterator, err := instance.QuerySample(ctx, expr, req) // this is returning a first value of 0,0 if err != nil { return err diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 1a7d49d07267..a6efe09e057f 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -6,6 +6,7 @@ import ( "net/http" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/multierror" "github.com/prometheus/common/model" @@ -159,6 +160,7 @@ func (i *instance) forMatchingStreams( ) error { filters, matchers := util.SplitFiltersAndMatchers(matchers) ids, err := i.index.Lookup(matchers, nil) + level.Debug(i.logger).Log("msg", "forMatchingStreams", "ids", ids) if err != nil { return err } From d59dd4c2229be9be42bf395262d971c6a153f103 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 17 Jun 2024 15:04:04 -0600 Subject: [PATCH 28/38] use sum merge sample iterator --- pkg/pattern/ingester_querier.go | 2 +- pkg/pattern/instance.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 214f42e474aa..5e323f17dc45 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -107,7 +107,7 @@ func (q *IngesterQuerier) Samples( } // TODO(twhitney): what should batch size be here? - resp, err := pattern_iter.ReadMetricsBatch(loki_iter.NewSortSampleIterator(iterators), math.MaxInt32, q.logger) + resp, err := pattern_iter.ReadMetricsBatch(pattern_iter.NewSumMergeSampleIterator(iterators), math.MaxInt32, q.logger) if err != nil { return nil, err } diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index a6efe09e057f..5b0515c096ca 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -160,7 +160,7 @@ func (i *instance) forMatchingStreams( ) error { filters, matchers := util.SplitFiltersAndMatchers(matchers) ids, err := i.index.Lookup(matchers, nil) - level.Debug(i.logger).Log("msg", "forMatchingStreams", "ids", ids) + level.Debug(i.logger).Log("msg", "forMatchingStreams", "ids", fmt.Sprintf("%v", ids)) if err != nil { return err } From e4b5dd71fc80361c337090b4547afdf49bba9ccf Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 17 Jun 2024 16:04:16 -0600 Subject: [PATCH 29/38] more debug logging --- pkg/pattern/ingester.go | 2 +- pkg/pattern/instance.go | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 4d898a1a8524..ddff98115d22 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -275,7 +275,7 @@ func (i *Ingester) QuerySample( return err } - level.Info(i.logger).Log("msg", "QuerySample", "instanceID", instanceID, "expr", expr) + level.Debug(i.logger).Log("msg", "QuerySample", "instanceID", instanceID, "expr", expr) iterator, err := instance.QuerySample(ctx, expr, req) // this is returning a first value of 0,0 if err != nil { return err diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 5b0515c096ca..38edda6dfb00 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -150,6 +150,12 @@ func (i *instance) QuerySample( return nil, err } + level.Debug(i.logger).Log( + "msg", "summing results of querying streams", + "num_iters", len(iters), + "iters", fmt.Sprintf("%v", iters), + ) + return pattern_iter.NewSumMergeSampleIterator(iters), nil } From 426d143318009544f3f03a087d5cbf00d00a3a24 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 17 Jun 2024 16:20:17 -0600 Subject: [PATCH 30/38] more debug logging --- pkg/pattern/instance.go | 4 +++- pkg/pattern/stream.go | 9 +++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 38edda6dfb00..0465d7576bc6 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -19,6 +19,7 @@ import ( "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" + "github.com/grafana/loki/v3/pkg/util/spanlogger" loki_iter "github.com/grafana/loki/v3/pkg/iter" pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" @@ -150,7 +151,8 @@ func (i *instance) QuerySample( return nil, err } - level.Debug(i.logger).Log( + spanLogger := spanlogger.FromContext(ctx) + level.Debug(spanLogger).Log( "msg", "summing results of querying streams", "num_iters", len(iters), "iters", fmt.Sprintf("%v", iters), diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 484e6edde0d0..515829a8b909 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -16,6 +16,7 @@ import ( "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/pattern/drain" "github.com/grafana/loki/v3/pkg/pattern/metric" + "github.com/grafana/loki/v3/pkg/util/spanlogger" loki_iter "github.com/grafana/loki/v3/pkg/iter" pattern_iter "github.com/grafana/loki/v3/pkg/pattern/iter" @@ -165,6 +166,14 @@ func (s *stream) SampleIterator( return nil, err } + spanLogger := spanlogger.FromContext(ctx) + level.Debug(spanLogger).Log( + "msg", "sample iterator for stream", + "stream", s.labelsString, + "num_results", len(matrix), + "matrix", fmt.Sprintf("%v", matrix), + ) + return loki_iter.NewMultiSeriesIterator(matrix), nil } From 0c5a4367bff20e2821b188c5482dc9df6305de56 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Mon, 17 Jun 2024 21:26:48 -0600 Subject: [PATCH 31/38] test: add test coverage around reading batches --- pkg/pattern/instance.go | 20 ++++-- pkg/pattern/iter/batch.go | 7 +- pkg/pattern/iter/batch_test.go | 86 ++++++++++++++++++---- pkg/pattern/stream.go | 30 +++++--- pkg/pattern/stream_test.go | 128 +++++++++++++++++++++++++++++++++ 5 files changed, 239 insertions(+), 32 deletions(-) diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 0465d7576bc6..2b26c4876ddf 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -151,12 +151,20 @@ func (i *instance) QuerySample( return nil, err } - spanLogger := spanlogger.FromContext(ctx) - level.Debug(spanLogger).Log( - "msg", "summing results of querying streams", - "num_iters", len(iters), - "iters", fmt.Sprintf("%v", iters), - ) + spanLogger := spanlogger.FromContext(ctx) + if spanLogger != nil { + level.Debug(spanLogger).Log( + "msg", "summing results of querying streams", + "num_iters", len(iters), + "iters", fmt.Sprintf("%v", iters), + ) + } else { + level.Debug(i.logger).Log( + "msg", "summing results of querying streams", + "num_iters", len(iters), + "iters", fmt.Sprintf("%v", iters), + ) + } return pattern_iter.NewSumMergeSampleIterator(iters), nil } diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index 444b4c904e1b..ef765b6d618b 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -40,7 +40,7 @@ func ReadAll(it Iterator) (*logproto.QueryPatternsResponse, error) { func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) (*logproto.QuerySamplesResponse, error) { var ( - series = map[uint64]*logproto.Series{} + series = map[uint64]logproto.Series{} respSize int ) @@ -48,7 +48,7 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) hash := it.StreamHash() s, ok := series[hash] if !ok { - s = &logproto.Series{ + s = logproto.Series{ Labels: it.Labels(), Samples: []logproto.Sample{}, StreamHash: hash, @@ -57,6 +57,7 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) } s.Samples = append(s.Samples, it.Sample()) + series[hash] = s } result := logproto.QuerySamplesResponse{ @@ -64,7 +65,7 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) } for _, s := range series { level.Debug(logger).Log("msg", "appending series", "s", fmt.Sprintf("%v", s)) - result.Series = append(result.Series, *s) + result.Series = append(result.Series, s) } return &result, it.Error() } diff --git a/pkg/pattern/iter/batch_test.go b/pkg/pattern/iter/batch_test.go index be845dd15f4b..a4457b95e7f9 100644 --- a/pkg/pattern/iter/batch_test.go +++ b/pkg/pattern/iter/batch_test.go @@ -4,9 +4,10 @@ import ( "testing" "github.com/go-kit/log" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" - "github.com/grafana/loki/v3/pkg/iter" + loki_iter "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" ) @@ -78,20 +79,26 @@ func TestReadBatch(t *testing.T) { } } +func singleSeriesIterator(series logproto.Series) []loki_iter.SampleIterator { + return []loki_iter.SampleIterator{ + loki_iter.NewSeriesIterator(series), + } +} + func TestReadMetricsBatch(t *testing.T) { tests := []struct { - name string - pattern string - series logproto.Series - batchSize int - expected *logproto.QuerySamplesResponse + name string + pattern string + seriesIter []loki_iter.SampleIterator + batchSize int + expected *logproto.QuerySamplesResponse }{ { name: "ReadBatch empty iterator", - series: logproto.Series{ + seriesIter: singleSeriesIterator(logproto.Series{ Labels: "", Samples: []logproto.Sample{}, - }, + }), batchSize: 2, expected: &logproto.QuerySamplesResponse{ Series: []logproto.Series{}, @@ -99,14 +106,14 @@ func TestReadMetricsBatch(t *testing.T) { }, { name: "ReadBatch less than batchSize", - series: logproto.Series{ + seriesIter: singleSeriesIterator(logproto.Series{ Labels: `{foo="bar"}`, Samples: []logproto.Sample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, }, - }, + }), batchSize: 2, expected: &logproto.QuerySamplesResponse{ Series: []logproto.Series{ @@ -122,14 +129,14 @@ func TestReadMetricsBatch(t *testing.T) { }, { name: "ReadBatch more than batchSize", - series: logproto.Series{ + seriesIter: singleSeriesIterator(logproto.Series{ Labels: `{foo="bar"}`, Samples: []logproto.Sample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, }, - }, + }), batchSize: 4, expected: &logproto.QuerySamplesResponse{ Series: []logproto.Series{ @@ -144,11 +151,64 @@ func TestReadMetricsBatch(t *testing.T) { }, }, }, + { + name: "ReadBatch multiple series", + seriesIter: []loki_iter.SampleIterator{ + loki_iter.NewSeriesIterator(logproto.Series{ + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + StreamHash: labels.StableHash([]labels.Label{{Name: "foo", Value: "bar"}}), + }), + loki_iter.NewSeriesIterator(logproto.Series{ + Labels: `{fizz="buzz"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 3}, + {Timestamp: 20, Value: 5}, + {Timestamp: 30, Value: 7}, + }, + StreamHash: labels.StableHash([]labels.Label{{Name: "fizz", Value: "buzz"}}), + }), + loki_iter.NewSeriesIterator(logproto.Series{ + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 20, Value: 2}, + }, + StreamHash: labels.StableHash([]labels.Label{{Name: "foo", Value: "bar"}}), + }), + }, + batchSize: 8, + expected: &logproto.QuerySamplesResponse{ + Series: []logproto.Series{ + { + Labels: `{foo="bar"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 6}, // from the second series + {Timestamp: 30, Value: 6}, + }, + StreamHash: labels.StableHash([]labels.Label{{Name: "foo", Value: "bar"}}), + }, + { + Labels: `{fizz="buzz"}`, + Samples: []logproto.Sample{ + {Timestamp: 10, Value: 3}, + {Timestamp: 20, Value: 5}, // from the second series + {Timestamp: 30, Value: 7}, + }, + StreamHash: labels.StableHash([]labels.Label{{Name: "fizz", Value: "buzz"}}), + }, + }, + }, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - it := iter.NewSeriesIterator(tt.series) + it := NewSumMergeSampleIterator(tt.seriesIter) got, err := ReadMetricsBatch(it, tt.batchSize, log.NewNopLogger()) require.NoError(t, err) require.Equal(t, tt.expected.Series, got.Series) diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 515829a8b909..4f6d5632de30 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -167,12 +167,21 @@ func (s *stream) SampleIterator( } spanLogger := spanlogger.FromContext(ctx) - level.Debug(spanLogger).Log( - "msg", "sample iterator for stream", - "stream", s.labelsString, - "num_results", len(matrix), - "matrix", fmt.Sprintf("%v", matrix), - ) + if spanLogger != nil { + level.Debug(spanLogger).Log( + "msg", "sample iterator for stream", + "stream", s.labelsString, + "num_results", len(matrix), + "matrix", fmt.Sprintf("%v", matrix), + ) + } else { + level.Debug(s.logger).Log( + "msg", "sample iterator for stream", + "stream", s.labelsString, + "num_results", len(matrix), + "matrix", fmt.Sprintf("%v", matrix), + ) + } return loki_iter.NewMultiSeriesIterator(matrix), nil } @@ -200,7 +209,7 @@ func (s *stream) joinSampleVectors( return nil, logqlmodel.NewSeriesLimitError(maxSeries) } - series := map[uint64]*logproto.Series{} + series := map[uint64]logproto.Series{} // step evaluator logic is slightly different than the normal contract in Loki // when evaluating a selection range, it's counts datapoints within (start, end] @@ -214,7 +223,7 @@ func (s *stream) joinSampleVectors( hash := p.Metric.Hash() s, ok := series[hash] if !ok { - s = &logproto.Series{ + s = logproto.Series{ Labels: p.Metric.String(), Samples: make([]logproto.Sample, 0, stepCount), StreamHash: hash, @@ -226,6 +235,7 @@ func (s *stream) joinSampleVectors( Timestamp: ts * 1e6, // convert milliseconds to nanoseconds Value: p.F, }) + series[hash] = s } next, ts, r = stepEvaluator.Next() @@ -235,8 +245,8 @@ func (s *stream) joinSampleVectors( } matrix := make([]logproto.Series, 0, len(series)) - for _, s := range series { - matrix = append(matrix, *s) + for i, s := range series { + matrix[i] = s } level.Debug(s.logger).Log( diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index ed08d6eebdb6..c7ddb911b59a 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -10,6 +10,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/iter" "github.com/grafana/loki/v3/pkg/pattern/metric" @@ -95,3 +96,130 @@ func TestPruneStream(t *testing.T) { require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) } + +func TestSampleIterator(t *testing.T) { + lbs := labels.New( + labels.Label{Name: "foo", Value: "bar"}, + labels.Label{Name: "fizz", Value: "buzz"}, + ) + + t.Run("single stream single push", func(t *testing.T) { + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: true, + }, + log.NewNopLogger(), + ) + require.NoError(t, err) + + err = stream.Push(context.Background(), []push.Entry{ + { + Timestamp: time.Unix(20, 0), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(20, 0), + Line: "ts=2 msg=hello", + }, + }) + + require.NoError(t, err) + + expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + it, err := stream.SampleIterator( + context.Background(), + expr, + 0, + model.Time(1*time.Minute/1e6), + model.Time(5*time.Second/1e6), + ) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.Equal(t, 1, len(res.Series)) + require.Equal(t, 1, len(res.Series[0].Samples)) + require.Equal(t, float64(2), res.Series[0].Samples[0].Value) + }) + + t.Run("single stream multiple pushes", func(t *testing.T) { + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: true, + }, + log.NewNopLogger(), + ) + require.NoError(t, err) + + err = stream.Push(context.Background(), []push.Entry{ + { + Timestamp: time.Unix(20, 0), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(20, 0), + Line: "ts=2 msg=hello", + }, + }) + require.NoError(t, err) + + err = stream.Push(context.Background(), []push.Entry{ + { + Timestamp: time.Unix(40, 0), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(40, 0), + Line: "ts=2 msg=hello", + }, + }) + require.NoError(t, err) + + t.Run("non-overlapping timestamps", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + it, err := stream.SampleIterator( + context.Background(), + expr, + 0, + model.Time(1*time.Minute/1e6), + model.Time(5*time.Second/1e6), + ) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.Equal(t, 1, len(res.Series)) + require.Equal(t, 2, len(res.Series[0].Samples)) + require.Equal(t, float64(2), res.Series[0].Samples[0].Value) + require.Equal(t, float64(2), res.Series[0].Samples[1].Value) + }) + + t.Run("overlapping timestamps", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[1m])") + it, err := stream.SampleIterator( + context.Background(), + expr, + 0, + model.Time(1*time.Minute/1e6), + model.Time(5*time.Second/1e6), + ) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.Equal(t, 1, len(res.Series)) + require.Equal( + t, + 8, + len(res.Series[0].Samples), + ) // bigger selection range keeps pushes in winodow longer + require.Equal(t, float64(2), res.Series[0].Samples[0].Value) + require.Equal(t, float64(4), res.Series[0].Samples[7].Value) + }) + }) +} From 96cccef5102315efdfdd2df5fa222fc8371c3212 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 18 Jun 2024 12:14:21 +0200 Subject: [PATCH 32/38] various fixes for building series --- pkg/pattern/flush.go | 1 + pkg/pattern/instance.go | 1 - pkg/pattern/iter/batch.go | 3 +- pkg/pattern/metric/chunk.go | 84 ++++++++++++++++++------------------- pkg/pattern/stream.go | 7 ++-- 5 files changed, 49 insertions(+), 47 deletions(-) diff --git a/pkg/pattern/flush.go b/pkg/pattern/flush.go index d53b486a168c..e36025094d88 100644 --- a/pkg/pattern/flush.go +++ b/pkg/pattern/flush.go @@ -66,6 +66,7 @@ func (i *Ingester) sweepInstance(instance *instance, _, mayRemoveStreams bool) { if mayRemoveStreams { instance.streams.WithLock(func() { if s.prune(retainSampleFor) { + level.Debug(i.logger).Log("msg", "pruned stream", "stream", s.labelsString) instance.removeStream(s) } }) diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index 2b26c4876ddf..cb5f8ae25314 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -138,7 +138,6 @@ func (i *instance) QuerySample( var iter loki_iter.SampleIterator var err error iter, err = stream.SampleIterator(ctx, expr, from, through, step) - if err != nil { return err } diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index ef765b6d618b..9d091429ce09 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -57,13 +57,14 @@ func ReadMetricsBatch(it iter.SampleIterator, batchSize int, logger log.Logger) } s.Samples = append(s.Samples, it.Sample()) - series[hash] = s + series[hash] = s } result := logproto.QuerySamplesResponse{ Series: make([]logproto.Series, 0, len(series)), } for _, s := range series { + s := s level.Debug(logger).Log("msg", "appending series", "s", fmt.Sprintf("%v", s)) result.Series = append(result.Series, s) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 8cd8e2511ad7..8b6279a0771e 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -13,11 +13,11 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/chunk" - - "github.com/grafana/loki/v3/pkg/iter" + util_log "github.com/grafana/loki/v3/pkg/util/log" ) type Type int @@ -245,44 +245,44 @@ func (c *Chunk) ForTypeAndRange( if len(c.Samples) == 0 { return nil, nil } - - first := c.Samples[0].Timestamp - last := c.Samples[len(c.Samples)-1].Timestamp - startBeforeEnd := start >= end - samplesAreAfterRange := first >= end - samplesAreBeforeRange := last < start - if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { - return nil, nil - } - - lo := 0 - hi := len(c.Samples) - - for i, sample := range c.Samples { - if first >= start { - break - } - - if first < start && sample.Timestamp >= start { - lo = i - first = sample.Timestamp - } - } - - for i := hi - 1; i >= 0; i-- { - if last < end { - break - } - - sample := c.Samples[i] - if last >= end && sample.Timestamp < end { - hi = i + 1 - last = sample.Timestamp - } - } - - aggregatedSamples := make([]logproto.Sample, len(c.Samples[lo:hi])) - for i, sample := range c.Samples[lo:hi] { + level.Debug(util_log.Logger).Log("msg", "ForTypeAndRange", "start", start, "end", end, "samples", fmt.Sprintf("%v", c.Samples)) + // first := c.Samples[0].Timestamp + // last := c.Samples[len(c.Samples)-1].Timestamp + // startBeforeEnd := start >= end + // samplesAreAfterRange := first >= end + // samplesAreBeforeRange := last < start + // if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { + // return nil, nil + // } + + // lo := 0 + // hi := len(c.Samples) + + // for i, sample := range c.Samples { + // if first >= start { + // break + // } + + // if first < start && sample.Timestamp >= start { + // lo = i + // first = sample.Timestamp + // } + // } + + // for i := hi - 1; i >= 0; i-- { + // if last < end { + // break + // } + + // sample := c.Samples[i] + // if last >= end && sample.Timestamp < end { + // hi = i + 1 + // last = sample.Timestamp + // } + // } + + aggregatedSamples := make([]logproto.Sample, 0, len(c.Samples)) + for _, sample := range c.Samples { if sample.Timestamp >= start && sample.Timestamp < end { var v float64 if typ == Bytes { @@ -290,10 +290,10 @@ func (c *Chunk) ForTypeAndRange( } else { v = sample.Count } - aggregatedSamples[i] = logproto.Sample{ + aggregatedSamples = append(aggregatedSamples, logproto.Sample{ Timestamp: sample.Timestamp.UnixNano(), Value: v, - } + }) } } diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 4f6d5632de30..66434ddf572e 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -64,7 +64,7 @@ func newStream( cfg: cfg, logger: logger, } - + level.Debug(logger).Log("msg", "creating new stream", "labels", stream.labelsString) if cfg.Enabled { chunks := metric.NewChunks(labels, chunkMetrics, logger) stream.chunks = chunks @@ -245,8 +245,9 @@ func (s *stream) joinSampleVectors( } matrix := make([]logproto.Series, 0, len(series)) - for i, s := range series { - matrix[i] = s + for _, s := range series { + s := s + matrix = append(matrix, s) } level.Debug(s.logger).Log( From 2d13e8a712cbacd5d35379d91c3d7b7a58ffb7a4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 18 Jun 2024 12:16:58 +0200 Subject: [PATCH 33/38] Removes comments --- pkg/pattern/flush.go | 1 - pkg/pattern/stream.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/pattern/flush.go b/pkg/pattern/flush.go index e36025094d88..d53b486a168c 100644 --- a/pkg/pattern/flush.go +++ b/pkg/pattern/flush.go @@ -66,7 +66,6 @@ func (i *Ingester) sweepInstance(instance *instance, _, mayRemoveStreams bool) { if mayRemoveStreams { instance.streams.WithLock(func() { if s.prune(retainSampleFor) { - level.Debug(i.logger).Log("msg", "pruned stream", "stream", s.labelsString) instance.removeStream(s) } }) diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 66434ddf572e..7840e592aa17 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -64,7 +64,7 @@ func newStream( cfg: cfg, logger: logger, } - level.Debug(logger).Log("msg", "creating new stream", "labels", stream.labelsString) + if cfg.Enabled { chunks := metric.NewChunks(labels, chunkMetrics, logger) stream.chunks = chunks From 74c245df295e74163c9a6928ee8fb31ebb37e26c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 18 Jun 2024 13:54:29 +0200 Subject: [PATCH 34/38] fix: Prune chunks older than a specified duration and only delete stream when empty --- pkg/pattern/metric/chunk.go | 22 ++++++++++++++++++++++ pkg/pattern/stream.go | 2 +- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 8b6279a0771e..57c804bc4059 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -88,6 +88,28 @@ func (c *Chunks) Observe(bytes, count float64, ts model.Time) { last.AddSample(newSample(bytes, count, ts)) } +func (c *Chunks) Prune(olderThan time.Duration) bool { + c.lock.Lock() + defer c.lock.Unlock() + + if len(c.chunks) == 0 { + return true + } + + oldest := time.Now().Add(-olderThan).UnixNano() + // keep the last chunk + for i := 0; i < len(c.chunks)-1; { + if c.chunks[i].maxt < oldest { + c.chunks = append(c.chunks[:i], c.chunks[i+1:]...) + c.metrics.chunks.Set(float64(len(c.chunks))) + continue + } + i++ + } + + return len(c.chunks) == 0 +} + func (c *Chunks) Iterator( typ Type, grouping *syntax.Grouping, diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 7840e592aa17..cf8cb1234fc0 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -274,5 +274,5 @@ func (s *stream) prune(olderThan time.Duration) bool { } } - return len(s.patterns.Clusters()) == 0 + return len(s.patterns.Clusters()) == 0 && s.chunks.Prune(olderThan) } From fefa6b50d2bcbe09cb989845868b0125180808b9 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 18 Jun 2024 11:37:55 -0600 Subject: [PATCH 35/38] test: more test coverage --- pkg/pattern/ingester_test.go | 464 +++++++++++++++++++++++++++++++ pkg/pattern/metric/chunk.go | 44 +-- pkg/pattern/metric/chunk_test.go | 38 ++- 3 files changed, 501 insertions(+), 45 deletions(-) diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index 2af622bc44fa..c4d9bcf0b9e7 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -2,7 +2,9 @@ package pattern import ( "context" + "fmt" "math" + "sort" "testing" "time" @@ -270,4 +272,466 @@ func TestInstancePushQuerySamples(t *testing.T) { require.Equal(t, float64(64), res.Series[0].Samples[6].Value) require.Equal(t, float64(64), res.Series[0].Samples[expectedDataPoints-1].Value) }) + + t.Run("test count_over_time samples, multiple streams", func(t *testing.T) { + lbls2 := labels.New( + labels.Label{Name: "fizz", Value: "buzz"}, + labels.Label{Name: "test", Value: "test"}, + labels.Label{Name: "foo", Value: "bar"}, + ) + lbls3 := labels.New( + labels.Label{Name: "fizz", Value: "buzz"}, + labels.Label{Name: "test", Value: "test"}, + ) + lbls4 := labels.New( + labels.Label{Name: "fizz", Value: "buzz"}, + labels.Label{Name: "foo", Value: "baz"}, + ) + + inst := setup(t) + err := inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbls.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=hello", + }, + }, + }, + { + Labels: lbls2.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=goodbye", + }, + }, + }, + { + Labels: lbls3.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=goodbye", + }, + }, + }, + { + Labels: lbls4.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=goodbye", + }, + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=shalom", + }, + }, + }, + }, + }) + for i := 1; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbls.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo bar foo bar", + }, + }, + }, + { + Labels: lbls2.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo", + }, + }, + }, + { + Labels: lbls3.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo", + }, + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "bar", + }, + }, + }, + { + Labels: lbls4.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo", + }, + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "bar", + }, + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "baz", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + + for _, tt := range []struct { + name string + expr func(string) syntax.SampleExpr + req func(syntax.SampleExpr) logproto.QuerySamplesRequest + verifySeries20sStep func([]logproto.Series) + verifySeries80sStep func([]logproto.Series) + }{ + { + // test="test" will capture lbls - lbls3 + name: `{test="test"}`, + expr: func(selRange string) syntax.SampleExpr { + expr, err := syntax.ParseSampleExpr(fmt.Sprintf(`count_over_time({test="test"}[%s])`, selRange)) + require.NoError(t, err) + return expr + }, + req: func(expr syntax.SampleExpr) logproto.QuerySamplesRequest { + return logproto.QuerySamplesRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + } + }, + verifySeries20sStep: func(series []logproto.Series) { + require.Equal(t, 3, len(series)) + require.Equal(t, lbls2.String(), series[0].GetLabels()) + require.Equal(t, lbls3.String(), series[1].GetLabels()) + require.Equal(t, lbls.String(), series[2].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + require.Equal(t, expectedDataPoints, len(series[2].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line per step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(1), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 2 lines per step for lbls3 + require.Equal(t, float64(2), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(2), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 1 linee per step for lbls3 + require.Equal(t, float64(1), series[2].Samples[0].Value, fmt.Sprintf("series: %v, samples: %v", series[2].Labels, series[2].Samples)) + require.Equal(t, float64(1), series[2].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, samples: %v", series[2].Labels, series[2].Samples[expectedDataPoints-1])) + }, + verifySeries80sStep: func(series []logproto.Series) { + require.Equal(t, 3, len(series)) + require.Equal(t, lbls2.String(), series[0].GetLabels()) + require.Equal(t, lbls3.String(), series[1].GetLabels()) + require.Equal(t, lbls.String(), series[2].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + require.Equal(t, expectedDataPoints, len(series[2].Samples)) + + // pushes are spaced 80s apart, and there's 10s step, + // so we expect to see a single line for the first step + // and 4 lines for the last for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(4), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 2 lines for the first step and + // 8 lines for the last for lbls3 + require.Equal(t, float64(2), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(8), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 1 linee for the first step + // and 4 lintes for the last step for lbls3 + require.Equal(t, float64(1), series[2].Samples[0].Value, fmt.Sprintf("series: %v, samples: %v", series[2].Labels, series[2].Samples)) + require.Equal(t, float64(4), series[2].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, samples: %v", series[2].Labels, series[2].Samples[expectedDataPoints-1])) + }, + }, + { + // fizz="buzz" will capture lbls2 - lbls4 + name: `{fizz="buzz"}`, + expr: func(selRange string) syntax.SampleExpr { + expr, err := syntax.ParseSampleExpr(fmt.Sprintf(`count_over_time({fizz="buzz"}[%s])`, selRange)) + require.NoError(t, err) + return expr + }, + req: func(expr syntax.SampleExpr) logproto.QuerySamplesRequest { + return logproto.QuerySamplesRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + } + }, + verifySeries20sStep: func(series []logproto.Series) { + require.Equal(t, 3, len(series)) + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls4.String(), series[1].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls3.String(), series[2].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + require.Equal(t, expectedDataPoints, len(series[2].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line per step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(1), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 3 lines per step for lbls4 + require.Equal(t, float64(3), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(3), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 2 lines per step for lbls3 + require.Equal(t, float64(2), series[2].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[2].Labels, series[2].Samples[0])) + require.Equal(t, float64(2), series[2].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[2].Labels, series[2].Samples[expectedDataPoints-1])) + }, + verifySeries80sStep: func(series []logproto.Series) { + require.Equal(t, 3, len(series)) + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls4.String(), series[1].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls3.String(), series[2].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + require.Equal(t, expectedDataPoints, len(series[2].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line for the first step + // and 4 lines for the last step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(4), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 3 lines for the first step + // and 12 lines for the last step for lbls4 + require.Equal(t, float64(3), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(12), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a 2 lines for the first step + // and 8 lines for the last step for lbls3 + require.Equal(t, float64(2), series[2].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[2].Labels, series[2].Samples[0])) + require.Equal(t, float64(8), series[2].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[2].Labels, series[2].Samples[expectedDataPoints-1])) + }, + }, + { + // foo="bar" will capture only lbls2 + name: `{foo="bar"}`, + expr: func(selRange string) syntax.SampleExpr { + expr, err := syntax.ParseSampleExpr(fmt.Sprintf(`count_over_time({foo="bar"}[%s])`, selRange)) + require.NoError(t, err) + return expr + }, + req: func(expr syntax.SampleExpr) logproto.QuerySamplesRequest { + return logproto.QuerySamplesRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + } + }, + verifySeries20sStep: func(series []logproto.Series) { + require.Equal(t, 1, len(series)) + + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line per step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(1), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + }, + verifySeries80sStep: func(series []logproto.Series) { + require.Equal(t, 1, len(series)) + + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line for the first step + // and 4 lines for the last step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(4), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + }, + }, + { + // foo=~".+" will capture lbls2 and lbls4 + name: `{foo=~".+"}`, + expr: func(selRange string) syntax.SampleExpr { + expr, err := syntax.ParseSampleExpr(fmt.Sprintf(`count_over_time({foo=~".+"}[%s])`, selRange)) + require.NoError(t, err) + return expr + }, + req: func(expr syntax.SampleExpr) logproto.QuerySamplesRequest { + return logproto.QuerySamplesRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + } + }, + verifySeries20sStep: func(series []logproto.Series) { + require.Equal(t, 2, len(series)) + + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls4.String(), series[1].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line per step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(1), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see 3 lines per step for lbls4 + require.Equal(t, float64(3), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(3), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + }, + verifySeries80sStep: func(series []logproto.Series) { + require.Equal(t, 2, len(series)) + + sereisLabels := make([]string, 0, len(series)) + for _, s := range series { + sereisLabels = append(sereisLabels, s.GetLabels()) + } + + require.Equal(t, lbls2.String(), series[0].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + require.Equal(t, lbls4.String(), series[1].GetLabels(), fmt.Sprintf("series: %v", sereisLabels)) + + // end - start / step -- (start is 0, step is 10s) + expectedDataPoints := ((20 * 30) / 10) + require.Equal(t, expectedDataPoints, len(series[0].Samples)) + require.Equal(t, expectedDataPoints, len(series[1].Samples)) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see a single line for the first step + // and 4 lines for the last step for lbls2 + require.Equal(t, float64(1), series[0].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[0])) + require.Equal(t, float64(4), series[0].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[0].Labels, series[0].Samples[expectedDataPoints-1])) + + // pushes are spaced 20s apart, and there's 10s step, + // so we expect to see 3 lines for the first step + // and 12 lines for the last step for lbls4 + require.Equal(t, float64(3), series[1].Samples[0].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[0])) + require.Equal(t, float64(12), series[1].Samples[expectedDataPoints-1].Value, fmt.Sprintf("series: %v, sample: %v", series[1].Labels, series[1].Samples[expectedDataPoints-1])) + }, + }, + } { + t.Run(tt.name, func(t *testing.T) { + expr := tt.expr("20s") + req := tt.req(expr) + + it, err := inst.QuerySample(context.Background(), expr, &req) + require.NoError(t, err) + res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + + ss := make([]logproto.Series, 0, len(res.Series)) + for _, s := range res.Series { + ss = append(ss, s) + } + + sort.Slice(ss, func(i, j int) bool { + return ss[i].Labels < ss[j].Labels + }) + + tt.verifySeries20sStep(ss) + + expr = tt.expr("80s") + req = tt.req(expr) + + it, err = inst.QuerySample(context.Background(), expr, &req) + require.NoError(t, err) + res, err = iter.ReadAllSamples(it) + require.NoError(t, err) + + ss = make([]logproto.Series, 0, len(res.Series)) + for _, s := range res.Series { + ss = append(ss, s) + } + + sort.Slice(ss, func(i, j int) bool { + return ss[i].Labels < ss[j].Labels + }) + + if tt.verifySeries80sStep != nil { + tt.verifySeries80sStep(ss) + } + }) + } + }) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go index 57c804bc4059..d4dfcb65d061 100644 --- a/pkg/pattern/metric/chunk.go +++ b/pkg/pattern/metric/chunk.go @@ -264,46 +264,16 @@ func (c *Chunk) ForTypeAndRange( return nil, fmt.Errorf("unsupported metric type") } + aggregatedSamples := make([]logproto.Sample, 0, len(c.Samples)) if len(c.Samples) == 0 { - return nil, nil + return aggregatedSamples, nil } - level.Debug(util_log.Logger).Log("msg", "ForTypeAndRange", "start", start, "end", end, "samples", fmt.Sprintf("%v", c.Samples)) - // first := c.Samples[0].Timestamp - // last := c.Samples[len(c.Samples)-1].Timestamp - // startBeforeEnd := start >= end - // samplesAreAfterRange := first >= end - // samplesAreBeforeRange := last < start - // if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { - // return nil, nil - // } - - // lo := 0 - // hi := len(c.Samples) - - // for i, sample := range c.Samples { - // if first >= start { - // break - // } - - // if first < start && sample.Timestamp >= start { - // lo = i - // first = sample.Timestamp - // } - // } - - // for i := hi - 1; i >= 0; i-- { - // if last < end { - // break - // } - - // sample := c.Samples[i] - // if last >= end && sample.Timestamp < end { - // hi = i + 1 - // last = sample.Timestamp - // } - // } - aggregatedSamples := make([]logproto.Sample, 0, len(c.Samples)) + level.Debug(util_log.Logger).Log("msg", "finding chunk samples for type and range", + "start", start, + "end", end, + "samples", fmt.Sprintf("%v", c.Samples)) + for _, sample := range c.Samples { if sample.Timestamp >= start && sample.Timestamp < end { var v float64 diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go index 206b5c673a97..40701744da9b 100644 --- a/pkg/pattern/metric/chunk_test.go +++ b/pkg/pattern/metric/chunk_test.go @@ -29,7 +29,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Count, start: 1, end: 10, - expected: nil, + expected: []logproto.Sample{}, }, { name: "Empty bytes", @@ -37,7 +37,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Bytes, start: 1, end: 10, - expected: nil, + expected: []logproto.Sample{}, }, { name: "No Overlap -- count", @@ -49,7 +49,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Count, start: 10, end: 20, - expected: nil, + expected: []logproto.Sample{}, }, { name: "No Overlap -- bytes", @@ -61,7 +61,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Bytes, start: 10, end: 20, - expected: nil, + expected: []logproto.Sample{}, }, { name: "Complete Overlap -- count", @@ -269,7 +269,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Count, start: 0, end: 1, - expected: nil, + expected: []logproto.Sample{}, }, { name: "Start and End before First Element -- bytes", @@ -281,7 +281,7 @@ func TestForTypeAndRange(t *testing.T) { metricType: Bytes, start: 0, end: 1, - expected: nil, + expected: []logproto.Sample{}, }, } @@ -292,12 +292,10 @@ func TestForTypeAndRange(t *testing.T) { if !reflect.DeepEqual(result, tc.expected) { t.Errorf("Expected %v, got %v", tc.expected, result) } - require.Equal(t, len(result), cap(result), "Returned slice wasn't created at the correct capacity") }) } } -// TODO(twhitney): test the maximum steps logic func Test_Chunks_Iterator(t *testing.T) { lbls := labels.Labels{ labels.Label{Name: "foo", Value: "bar"}, @@ -424,4 +422,28 @@ func Test_Chunks_Iterator(t *testing.T) { require.Equal(t, 0, len(res.Series)) }) + + t.Run("correctly sets capacity for samples slice to range / step", func(t *testing.T) { + it, err := chunks.Iterator(Bytes, nil, 0, 10, 2) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) + require.Equal(t, 3, len(res.Series[0].Samples)) + require.Equal(t, 4, cap(res.Series[0].Samples)) + + it, err = chunks.Iterator(Count, nil, 0, 10, 2) + require.NoError(t, err) + + res, err = iter.ReadAllSamples(it) + require.NoError(t, err) + + require.Equal(t, 1, len(res.Series)) + require.Equal(t, lbls.String(), res.Series[0].GetLabels()) + require.Equal(t, 3, len(res.Series[0].Samples)) + require.Equal(t, 4, cap(res.Series[0].Samples)) + }) } From 796d8d9c48380e91739077f526e7482a4c0725dc Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 18 Jun 2024 12:38:37 -0600 Subject: [PATCH 36/38] fix: truncate timestamp to nearest step --- pkg/pattern/ingester_querier_test.go | 20 ++++++++++ pkg/pattern/stream.go | 4 ++ pkg/pattern/stream_test.go | 57 ++++++++++++++++++++++++++++ 3 files changed, 81 insertions(+) diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index 77562cc163cb..c2d0a8913507 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -250,6 +250,26 @@ func (f *fakeRingClient) Ring() ring.ReadRing { type fakeRing struct{} +// InstancesWithTokensCount returns the number of instances in the ring that have tokens. +func (f *fakeRing) InstancesWithTokensCount() int { + panic("not implemented") // TODO: Implement +} + +// InstancesInZoneCount returns the number of instances in the ring that are registered in given zone. +func (f *fakeRing) InstancesInZoneCount(zone string) int { + panic("not implemented") // TODO: Implement +} + +// InstancesWithTokensInZoneCount returns the number of instances in the ring that are registered in given zone and have tokens. +func (f *fakeRing) InstancesWithTokensInZoneCount(zone string) int { + panic("not implemented") // TODO: Implement +} + +// ZonesCount returns the number of zones for which there's at least 1 instance registered in the ring. +func (f *fakeRing) ZonesCount() int { + panic("not implemented") // TODO: Implement +} + func (f *fakeRing) Get( _ uint32, _ ring.Operation, diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index cf8cb1234fc0..202efc89873a 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -14,6 +14,7 @@ import ( "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/logqlmodel" + "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/drain" "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util/spanlogger" @@ -134,6 +135,9 @@ func (s *stream) SampleIterator( s.mtx.Lock() defer s.mtx.Unlock() + from = chunk.TruncateTimestamp(from, step) + through = chunk.TruncateTimestamp(through, step) + stepEvaluator, err := s.evaluator.NewStepEvaluator( ctx, s.evaluator, diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index c7ddb911b59a..2745bbb9cf16 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -222,4 +222,61 @@ func TestSampleIterator(t *testing.T) { require.Equal(t, float64(4), res.Series[0].Samples[7].Value) }) }) + + t.Run("truncates timestamps to nearest step", func(t *testing.T) { + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + metric.NewChunkMetrics(nil, "test"), + metric.AggregationConfig{ + Enabled: true, + }, + log.NewNopLogger(), + ) + require.NoError(t, err) + + err = stream.Push(context.Background(), []push.Entry{ + { + Timestamp: time.Unix(26, 999), + Line: "ts=1 msg=hello", + }, + { + Timestamp: time.Unix(26, 999), + Line: "ts=2 msg=hello", + }, + }) + + require.NoError(t, err) + + expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[1s])") + it, err := stream.SampleIterator( + context.Background(), + expr, + 789, + model.Time(1*time.Minute/1e6), + model.Time(time.Second/1e6), + ) + require.NoError(t, err) + + res, err := iter.ReadAllSamples(it) + require.Equal(t, 1, len(res.Series)) + require.Equal(t, 1, len(res.Series[0].Samples)) + require.Equal(t, int64(26000000000), res.Series[0].Samples[0].Timestamp) + + expr, err = syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + it, err = stream.SampleIterator( + context.Background(), + expr, + 4789, + model.Time(1*time.Minute/1e6), + model.Time(5*time.Second/1e6), + ) + require.NoError(t, err) + + res, err = iter.ReadAllSamples(it) + require.Equal(t, 1, len(res.Series)) + require.Equal(t, 1, len(res.Series[0].Samples)) + require.Equal(t, int64(30000000000), res.Series[0].Samples[0].Timestamp) + }) } From b2d72a64cb84210af2141adc865a8d89601cbeb4 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 18 Jun 2024 15:34:06 -0600 Subject: [PATCH 37/38] fix: tests --- pkg/pattern/instance_test.go | 12 +----------- pkg/pattern/stream.go | 15 +++++++++------ 2 files changed, 10 insertions(+), 17 deletions(-) diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go index 71d90717b15d..07351089ca8d 100644 --- a/pkg/pattern/instance_test.go +++ b/pkg/pattern/instance_test.go @@ -250,7 +250,6 @@ func TestInstance_QuerySample(t *testing.T) { thirdStep := secondStep + oneMin // 120000 fourthStep := thirdStep + oneMin // 180000 fifthStep := fourthStep + oneMin // 240000 - sixthStep := fifthStep + oneMin // 300000 // our first push had a timestamp of 90000 (equal to the timestamp of it's last entry) // therefore our first datapoint will be at 120000, since we have nothing for @@ -339,16 +338,7 @@ func TestInstance_QuerySample(t *testing.T) { require.Equal(t, float64(14), sample.Value) require.Equal(t, expectedLabels.String(), iter.Labels()) - // since our timr range through goes to 310000, we will have 1 more data point - next = iter.Next() - require.True(t, next) - - sample = iter.Sample() - require.Equal(t, model.Time(sixthStep).UnixNano(), sample.Timestamp) - require.Equal(t, float64(14), sample.Value) - require.Equal(t, expectedLabels.String(), iter.Labels()) - - // there should be no more samples + // our time range through goes to 310000, but will be truncated, so this is the end next = iter.Next() require.False(t, next) }, diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index 202efc89873a..ec075fece739 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -66,11 +66,9 @@ func newStream( logger: logger, } - if cfg.Enabled { - chunks := metric.NewChunks(labels, chunkMetrics, logger) - stream.chunks = chunks - stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) - } + chunks := metric.NewChunks(labels, chunkMetrics, logger) + stream.chunks = chunks + stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) return stream, nil } @@ -278,5 +276,10 @@ func (s *stream) prune(olderThan time.Duration) bool { } } - return len(s.patterns.Clusters()) == 0 && s.chunks.Prune(olderThan) + chunksPruned := true + if s.chunks != nil { + chunksPruned = s.chunks.Prune(olderThan) + } + + return len(s.patterns.Clusters()) == 0 && chunksPruned } From fa7016b2d2ad070f4f25ecd4b22225c770cf3ab0 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Tue, 18 Jun 2024 16:14:16 -0600 Subject: [PATCH 38/38] fix: lint --- pkg/pattern/ingester_querier_test.go | 4 ++-- pkg/pattern/ingester_test.go | 8 ++------ pkg/pattern/stream_test.go | 19 +++++++++++++++++++ 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index c2d0a8913507..ec64221274a0 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -256,12 +256,12 @@ func (f *fakeRing) InstancesWithTokensCount() int { } // InstancesInZoneCount returns the number of instances in the ring that are registered in given zone. -func (f *fakeRing) InstancesInZoneCount(zone string) int { +func (f *fakeRing) InstancesInZoneCount(_ string) int { panic("not implemented") // TODO: Implement } // InstancesWithTokensInZoneCount returns the number of instances in the ring that are registered in given zone and have tokens. -func (f *fakeRing) InstancesWithTokensInZoneCount(zone string) int { +func (f *fakeRing) InstancesWithTokensInZoneCount(_ string) int { panic("not implemented") // TODO: Implement } diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index c4d9bcf0b9e7..1e683f51ece0 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -701,9 +701,7 @@ func TestInstancePushQuerySamples(t *testing.T) { require.NoError(t, err) ss := make([]logproto.Series, 0, len(res.Series)) - for _, s := range res.Series { - ss = append(ss, s) - } + ss = append(ss, res.Series...) sort.Slice(ss, func(i, j int) bool { return ss[i].Labels < ss[j].Labels @@ -720,9 +718,7 @@ func TestInstancePushQuerySamples(t *testing.T) { require.NoError(t, err) ss = make([]logproto.Series, 0, len(res.Series)) - for _, s := range res.Series { - ss = append(ss, s) - } + ss = append(ss, res.Series...) sort.Slice(ss, func(i, j int) bool { return ss[i].Labels < ss[j].Labels diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index 2745bbb9cf16..cee3df791319 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -130,6 +130,8 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + require.NoError(t, err) + it, err := stream.SampleIterator( context.Background(), expr, @@ -140,6 +142,7 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) res, err := iter.ReadAllSamples(it) + require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, 1, len(res.Series[0].Samples)) require.Equal(t, float64(2), res.Series[0].Samples[0].Value) @@ -184,6 +187,8 @@ func TestSampleIterator(t *testing.T) { t.Run("non-overlapping timestamps", func(t *testing.T) { expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + require.NoError(t, err) + it, err := stream.SampleIterator( context.Background(), expr, @@ -194,6 +199,8 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) require.Equal(t, 2, len(res.Series[0].Samples)) require.Equal(t, float64(2), res.Series[0].Samples[0].Value) @@ -202,6 +209,8 @@ func TestSampleIterator(t *testing.T) { t.Run("overlapping timestamps", func(t *testing.T) { expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[1m])") + require.NoError(t, err) + it, err := stream.SampleIterator( context.Background(), expr, @@ -212,6 +221,8 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) require.Equal( t, @@ -250,6 +261,8 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) expr, err := syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[1s])") + require.NoError(t, err) + it, err := stream.SampleIterator( context.Background(), expr, @@ -260,11 +273,15 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) res, err := iter.ReadAllSamples(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) require.Equal(t, 1, len(res.Series[0].Samples)) require.Equal(t, int64(26000000000), res.Series[0].Samples[0].Timestamp) expr, err = syntax.ParseSampleExpr("count_over_time({foo=\"bar\"}[5s])") + require.NoError(t, err) + it, err = stream.SampleIterator( context.Background(), expr, @@ -275,6 +292,8 @@ func TestSampleIterator(t *testing.T) { require.NoError(t, err) res, err = iter.ReadAllSamples(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) require.Equal(t, 1, len(res.Series[0].Samples)) require.Equal(t, int64(30000000000), res.Series[0].Samples[0].Timestamp)