diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 90d54d30c2989..c81d6f5dfb5dc 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -130,6 +130,11 @@ querier_rf1: # CLI flag: -querier-rf1.engine.max-lookback-period [max_look_back_period: | default = 30s] + # The maximum number of labels the heap of a topk query using a count min + # sketch can track. + # CLI flag: -querier-rf1.engine.max-count-min-sketch-heap-size + [max_count_min_sketch_heap_size: | default = 10000] + # The maximum number of queries that can be simultaneously processed by the # querier. # CLI flag: -querier-rf1.max-concurrent @@ -3841,6 +3846,9 @@ otlp_config: # CLI flag: -limits.ingestion-partition-tenant-shard-size [ingestion_partitions_tenant_shard_size: | default = 0] +# List of LogQL vector and range aggregations that should be sharded. +[shard_aggregations: ] + # Enable metric aggregation. When enabled, pushed streams will be sampled for # bytes and count, and these metric will be written back into Loki as a special # __aggregated_metric__ stream, which can be queried for faster histogram @@ -4245,6 +4253,11 @@ engine: # CLI flag: -querier.engine.max-lookback-period [max_look_back_period: | default = 30s] + # The maximum number of labels the heap of a topk query using a count min + # sketch can track. + # CLI flag: -querier.engine.max-count-min-sketch-heap-size + [max_count_min_sketch_heap_size: | default = 10000] + # The maximum number of queries that can be simultaneously processed by the # querier. # CLI flag: -querier.max-concurrent diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index cfa46c1e9d85b..c462115020959 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -94,7 +94,7 @@ func TestMicroServicesIngestQuery(t *testing.T) { "-common.compactor-address="+tCompactor.HTTPURL(), "-querier.per-request-limits-enabled=true", "-frontend.encoding=protobuf", - "-querier.shard-aggregations=quantile_over_time", + "-querier.shard-aggregations=quantile_over_time,approx_topk", "-frontend.tail-proxy-url="+tQuerier.HTTPURL(), ) ) @@ -784,6 +784,115 @@ func TestOTLPLogsIngestQuery(t *testing.T) { }) } +func TestProbabilisticQuery(t *testing.T) { + clu := cluster.New(nil, cluster.SchemaWithTSDBAndTSDB, func(c *cluster.Cluster) { + c.SetSchemaVer("v13") + }) + defer func() { + assert.NoError(t, clu.Cleanup()) + }() + + // run initially the compactor, indexgateway, and distributor. + var ( + tCompactor = clu.AddComponent( + "compactor", + "-target=compactor", + "-compactor.compaction-interval=1s", + "-compactor.retention-delete-delay=1s", + // By default, a minute is added to the delete request start time. This compensates for that. + "-compactor.delete-request-cancel-period=-60s", + "-compactor.deletion-mode=filter-and-delete", + ) + tIndexGateway = clu.AddComponent( + "index-gateway", + "-target=index-gateway", + ) + tDistributor = clu.AddComponent( + "distributor", + "-target=distributor", + ) + ) + require.NoError(t, clu.Run()) + + // then, run only the ingester and query scheduler. + var ( + tIngester = clu.AddComponent( + "ingester", + "-target=ingester", + "-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + tQueryScheduler = clu.AddComponent( + "query-scheduler", + "-target=query-scheduler", + "-query-scheduler.use-scheduler-ring=false", + "-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + ) + require.NoError(t, clu.Run()) + + // the run querier. + var ( + tQuerier = clu.AddComponent( + "querier", + "-target=querier", + "-querier.scheduler-address="+tQueryScheduler.GRPCURL(), + "-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + "-common.compactor-address="+tCompactor.HTTPURL(), + ) + ) + require.NoError(t, clu.Run()) + + // finally, run the query-frontend. + var ( + tQueryFrontend = clu.AddComponent( + "query-frontend", + "-target=query-frontend", + "-frontend.scheduler-address="+tQueryScheduler.GRPCURL(), + "-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + "-common.compactor-address="+tCompactor.HTTPURL(), + "-querier.per-request-limits-enabled=true", + "-frontend.encoding=protobuf", + "-querier.shard-aggregations=quantile_over_time,approx_topk", + "-frontend.tail-proxy-url="+tQuerier.HTTPURL(), + ) + ) + require.NoError(t, clu.Run()) + + tenantID := randStringRunes() + + now := time.Now() + cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL()) + cliDistributor.Now = now + cliIngester := client.New(tenantID, "", tIngester.HTTPURL()) + cliIngester.Now = now + cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL()) + cliQueryFrontend.Now = now + + t.Run("ingest-logs", func(t *testing.T) { + // ingest some log lines + require.NoError(t, cliDistributor.PushLogLine("lineA", now.Add(-45*time.Minute), nil, map[string]string{"job": "one"})) + require.NoError(t, cliDistributor.PushLogLine("lineB", now.Add(-45*time.Minute), nil, map[string]string{"job": "one"})) + + require.NoError(t, cliDistributor.PushLogLine("lineC", now, nil, map[string]string{"job": "one"})) + require.NoError(t, cliDistributor.PushLogLine("lineD", now, nil, map[string]string{"job": "two"})) + }) + + t.Run("query", func(t *testing.T) { + resp, err := cliQueryFrontend.RunQuery(context.Background(), `approx_topk(1, count_over_time({job=~".+"}[1h]))`) + require.NoError(t, err) + assert.Equal(t, "vector", resp.Data.ResultType) + + var values []string + var labels []string + for _, value := range resp.Data.Vector { + values = append(values, value.Value) + labels = append(labels, value.Metric["job"]) + } + assert.ElementsMatch(t, []string{"3"}, values) + assert.ElementsMatch(t, []string{"one"}, labels) + }) +} + func TestCategorizedLabels(t *testing.T) { clu := cluster.New(nil, cluster.SchemaWithTSDB, func(c *cluster.Cluster) { c.SetSchemaVer("v13") diff --git a/pkg/logproto/sketch.pb.go b/pkg/logproto/sketch.pb.go index ef0fa986b1e33..d3e2328594366 100644 --- a/pkg/logproto/sketch.pb.go +++ b/pkg/logproto/sketch.pb.go @@ -379,7 +379,8 @@ type CountMinSketch struct { Depth uint32 `protobuf:"varint,1,opt,name=depth,proto3" json:"depth,omitempty"` Width uint32 `protobuf:"varint,2,opt,name=width,proto3" json:"width,omitempty"` // counters is a matrix of depth * width. - Counters []uint32 `protobuf:"varint,3,rep,packed,name=counters,proto3" json:"counters,omitempty"` + Counters []float64 `protobuf:"fixed64,3,rep,packed,name=counters,proto3" json:"counters,omitempty"` + Hyperloglog []byte `protobuf:"bytes,4,opt,name=hyperloglog,proto3" json:"hyperloglog,omitempty"` } func (m *CountMinSketch) Reset() { *m = CountMinSketch{} } @@ -428,13 +429,122 @@ func (m *CountMinSketch) GetWidth() uint32 { return 0 } -func (m *CountMinSketch) GetCounters() []uint32 { +func (m *CountMinSketch) GetCounters() []float64 { if m != nil { return m.Counters } return nil } +func (m *CountMinSketch) GetHyperloglog() []byte { + if m != nil { + return m.Hyperloglog + } + return nil +} + +type CountMinSketchVector struct { + TimestampMs int64 `protobuf:"varint,1,opt,name=timestamp_ms,json=timestampMs,proto3" json:"timestamp_ms,omitempty"` + Sketch *CountMinSketch `protobuf:"bytes,2,opt,name=sketch,proto3" json:"sketch,omitempty"` + Metrics []*Labels `protobuf:"bytes,3,rep,name=metrics,proto3" json:"metrics,omitempty"` +} + +func (m *CountMinSketchVector) Reset() { *m = CountMinSketchVector{} } +func (*CountMinSketchVector) ProtoMessage() {} +func (*CountMinSketchVector) Descriptor() ([]byte, []int) { + return fileDescriptor_7f9fd40e59b87ff3, []int{6} +} +func (m *CountMinSketchVector) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CountMinSketchVector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CountMinSketchVector.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 *CountMinSketchVector) XXX_Merge(src proto.Message) { + xxx_messageInfo_CountMinSketchVector.Merge(m, src) +} +func (m *CountMinSketchVector) XXX_Size() int { + return m.Size() +} +func (m *CountMinSketchVector) XXX_DiscardUnknown() { + xxx_messageInfo_CountMinSketchVector.DiscardUnknown(m) +} + +var xxx_messageInfo_CountMinSketchVector proto.InternalMessageInfo + +func (m *CountMinSketchVector) GetTimestampMs() int64 { + if m != nil { + return m.TimestampMs + } + return 0 +} + +func (m *CountMinSketchVector) GetSketch() *CountMinSketch { + if m != nil { + return m.Sketch + } + return nil +} + +func (m *CountMinSketchVector) GetMetrics() []*Labels { + if m != nil { + return m.Metrics + } + return nil +} + +type Labels struct { + Metric []*LabelPair `protobuf:"bytes,1,rep,name=metric,proto3" json:"metric,omitempty"` +} + +func (m *Labels) Reset() { *m = Labels{} } +func (*Labels) ProtoMessage() {} +func (*Labels) Descriptor() ([]byte, []int) { + return fileDescriptor_7f9fd40e59b87ff3, []int{7} +} +func (m *Labels) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Labels) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Labels.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 *Labels) XXX_Merge(src proto.Message) { + xxx_messageInfo_Labels.Merge(m, src) +} +func (m *Labels) XXX_Size() int { + return m.Size() +} +func (m *Labels) XXX_DiscardUnknown() { + xxx_messageInfo_Labels.DiscardUnknown(m) +} + +var xxx_messageInfo_Labels proto.InternalMessageInfo + +func (m *Labels) GetMetric() []*LabelPair { + if m != nil { + return m.Metric + } + return nil +} + type TopK struct { Cms *CountMinSketch `protobuf:"bytes,1,opt,name=cms,proto3" json:"cms,omitempty"` List []*TopK_Pair `protobuf:"bytes,2,rep,name=list,proto3" json:"list,omitempty"` @@ -444,7 +554,7 @@ type TopK struct { func (m *TopK) Reset() { *m = TopK{} } func (*TopK) ProtoMessage() {} func (*TopK) Descriptor() ([]byte, []int) { - return fileDescriptor_7f9fd40e59b87ff3, []int{6} + return fileDescriptor_7f9fd40e59b87ff3, []int{8} } func (m *TopK) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -495,14 +605,14 @@ func (m *TopK) GetHyperloglog() []byte { } type TopK_Pair struct { - Event string `protobuf:"bytes,1,opt,name=event,proto3" json:"event,omitempty"` - Count uint32 `protobuf:"varint,2,opt,name=count,proto3" json:"count,omitempty"` + Event string `protobuf:"bytes,1,opt,name=event,proto3" json:"event,omitempty"` + Count float64 `protobuf:"fixed64,2,opt,name=count,proto3" json:"count,omitempty"` } func (m *TopK_Pair) Reset() { *m = TopK_Pair{} } func (*TopK_Pair) ProtoMessage() {} func (*TopK_Pair) Descriptor() ([]byte, []int) { - return fileDescriptor_7f9fd40e59b87ff3, []int{6, 0} + return fileDescriptor_7f9fd40e59b87ff3, []int{8, 0} } func (m *TopK_Pair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -538,7 +648,7 @@ func (m *TopK_Pair) GetEvent() string { return "" } -func (m *TopK_Pair) GetCount() uint32 { +func (m *TopK_Pair) GetCount() float64 { if m != nil { return m.Count } @@ -552,7 +662,7 @@ type TopKMatrix struct { func (m *TopKMatrix) Reset() { *m = TopKMatrix{} } func (*TopKMatrix) ProtoMessage() {} func (*TopKMatrix) Descriptor() ([]byte, []int) { - return fileDescriptor_7f9fd40e59b87ff3, []int{7} + return fileDescriptor_7f9fd40e59b87ff3, []int{9} } func (m *TopKMatrix) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -596,7 +706,7 @@ type TopKMatrix_Vector struct { func (m *TopKMatrix_Vector) Reset() { *m = TopKMatrix_Vector{} } func (*TopKMatrix_Vector) ProtoMessage() {} func (*TopKMatrix_Vector) Descriptor() ([]byte, []int) { - return fileDescriptor_7f9fd40e59b87ff3, []int{7, 0} + return fileDescriptor_7f9fd40e59b87ff3, []int{9, 0} } func (m *TopKMatrix_Vector) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -647,6 +757,8 @@ func init() { proto.RegisterType((*TDigest)(nil), "logproto.TDigest") proto.RegisterType((*TDigest_Centroid)(nil), "logproto.TDigest.Centroid") proto.RegisterType((*CountMinSketch)(nil), "logproto.CountMinSketch") + proto.RegisterType((*CountMinSketchVector)(nil), "logproto.CountMinSketchVector") + proto.RegisterType((*Labels)(nil), "logproto.Labels") proto.RegisterType((*TopK)(nil), "logproto.TopK") proto.RegisterType((*TopK_Pair)(nil), "logproto.TopK.Pair") proto.RegisterType((*TopKMatrix)(nil), "logproto.TopKMatrix") @@ -656,47 +768,50 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/sketch.proto", fileDescriptor_7f9fd40e59b87ff3) } var fileDescriptor_7f9fd40e59b87ff3 = []byte{ - // 626 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0x41, 0x4f, 0xd4, 0x4e, - 0x14, 0xef, 0xfc, 0x77, 0xff, 0xcb, 0xf2, 0x16, 0x88, 0x8e, 0xc4, 0xd4, 0xc5, 0x4c, 0xd6, 0x1e, - 0x84, 0x68, 0xdc, 0x4d, 0x20, 0x21, 0x9c, 0xc1, 0x03, 0x89, 0xa2, 0x38, 0x10, 0x63, 0x4c, 0x8c, - 0x19, 0xda, 0xa1, 0x3b, 0xd9, 0xb6, 0xd3, 0x74, 0x66, 0x01, 0x6f, 0x7e, 0x02, 0x63, 0xfc, 0x14, - 0x5e, 0xfd, 0x08, 0xde, 0x3c, 0x72, 0xe4, 0x28, 0xe5, 0xe2, 0x91, 0x8f, 0x60, 0x66, 0xda, 0x2e, - 0x74, 0x89, 0xd1, 0xd3, 0xce, 0xfb, 0xbd, 0xdf, 0xfb, 0xcd, 0x6f, 0xde, 0xeb, 0x5b, 0xb8, 0x97, - 0x8e, 0xc2, 0x41, 0x24, 0xc3, 0x34, 0x93, 0x5a, 0x0e, 0xd4, 0x88, 0x6b, 0x7f, 0xd8, 0xb7, 0x01, - 0x6e, 0x57, 0x70, 0x77, 0xa9, 0x46, 0xaa, 0x0e, 0x05, 0xcd, 0x7b, 0x01, 0x8b, 0xaf, 0xc6, 0x2c, - 0xd1, 0x22, 0xe2, 0x7b, 0xb6, 0x7c, 0x87, 0xe9, 0x4c, 0x9c, 0xe0, 0x75, 0x68, 0x1d, 0xb1, 0x68, - 0xcc, 0x95, 0x8b, 0x7a, 0x8d, 0x95, 0xce, 0x2a, 0xe9, 0x4f, 0x0a, 0xeb, 0xfc, 0xd7, 0xdc, 0xd7, - 0x32, 0xa3, 0x25, 0xdb, 0xdb, 0x9d, 0xd6, 0x2b, 0xf2, 0x78, 0x03, 0x66, 0x14, 0x8b, 0xd3, 0xe8, - 0xef, 0x82, 0x7b, 0x96, 0x46, 0x2b, 0xba, 0xf7, 0x09, 0x4d, 0x4b, 0x16, 0x0c, 0xfc, 0x10, 0xd0, - 0xa1, 0x8b, 0x7a, 0x68, 0xa5, 0xb3, 0xea, 0xfe, 0x49, 0x8c, 0xa2, 0x43, 0xfc, 0x00, 0xe6, 0xb4, - 0x88, 0xb9, 0xd2, 0x2c, 0x4e, 0xdf, 0xc7, 0xca, 0xfd, 0xaf, 0x87, 0x56, 0x1a, 0xb4, 0x33, 0xc1, - 0x76, 0x14, 0x7e, 0x0c, 0xad, 0x98, 0xeb, 0x4c, 0xf8, 0x6e, 0xc3, 0x9a, 0xbb, 0x73, 0xa5, 0xf7, - 0x9c, 0x1d, 0xf0, 0x68, 0x97, 0x89, 0x8c, 0x96, 0x14, 0x2f, 0x84, 0x85, 0xfa, 0x25, 0xf8, 0x09, - 0xcc, 0xe8, 0x40, 0x84, 0x5c, 0xe9, 0xd2, 0xcf, 0xed, 0xab, 0xfa, 0xfd, 0xa7, 0x36, 0xb1, 0xed, - 0xd0, 0x8a, 0x83, 0xef, 0x43, 0x3b, 0x08, 0x8a, 0x61, 0x59, 0x33, 0x73, 0xdb, 0x0e, 0x9d, 0x20, - 0x9b, 0x6d, 0x68, 0x15, 0x27, 0xef, 0x3b, 0x82, 0x99, 0xb2, 0x1c, 0xdf, 0x82, 0x46, 0x2c, 0x12, - 0x2b, 0x8f, 0xa8, 0x39, 0x5a, 0x84, 0x9d, 0x58, 0x01, 0x83, 0xb0, 0x13, 0xdc, 0x83, 0x8e, 0x2f, - 0xe3, 0x34, 0xe3, 0x4a, 0x09, 0x99, 0xb8, 0x0d, 0x9b, 0xb9, 0x0e, 0xe1, 0x0d, 0x98, 0x4d, 0x33, - 0xe9, 0x73, 0xa5, 0x78, 0xe0, 0x36, 0xed, 0x53, 0xbb, 0x37, 0xac, 0xf6, 0xb7, 0x78, 0xa2, 0x33, - 0x29, 0x02, 0x7a, 0x45, 0xee, 0xae, 0x43, 0xbb, 0x82, 0x31, 0x86, 0x66, 0xcc, 0x59, 0x65, 0xc6, - 0x9e, 0xf1, 0x5d, 0x68, 0x1d, 0x73, 0x11, 0x0e, 0x75, 0x69, 0xa8, 0x8c, 0xbc, 0x37, 0xb0, 0xb0, - 0x25, 0xc7, 0x89, 0xde, 0x11, 0x49, 0xd9, 0xac, 0x45, 0xf8, 0x3f, 0xe0, 0xa9, 0x1e, 0xda, 0xf2, - 0x79, 0x5a, 0x04, 0x06, 0x3d, 0x16, 0x81, 0x2e, 0x1a, 0x32, 0x4f, 0x8b, 0x00, 0x77, 0xa1, 0xed, - 0x9b, 0x6a, 0x9e, 0x29, 0x3b, 0x99, 0x79, 0x3a, 0x89, 0xbd, 0x6f, 0x08, 0x9a, 0xfb, 0x32, 0x7d, - 0x86, 0x1f, 0x41, 0xc3, 0x8f, 0xd5, 0xcd, 0x2f, 0xa1, 0x7e, 0x2f, 0x35, 0x24, 0xbc, 0x0c, 0xcd, - 0x48, 0x28, 0x63, 0x72, 0x6a, 0xcc, 0x46, 0xa9, 0x6f, 0xc7, 0x6c, 0x09, 0xa6, 0x97, 0xc3, 0x0f, - 0x29, 0xcf, 0x22, 0x19, 0x46, 0x32, 0xb4, 0xbd, 0x9c, 0xa3, 0xd7, 0xa1, 0xee, 0x2a, 0x34, 0x0d, - 0xdf, 0x38, 0xe7, 0x47, 0x3c, 0x29, 0x46, 0x3f, 0x4b, 0x8b, 0xc0, 0xa0, 0xd6, 0x69, 0xf5, 0x1e, - 0x1b, 0x78, 0x5f, 0x10, 0x80, 0xb9, 0xa9, 0x5c, 0xb2, 0xb5, 0xa9, 0x25, 0x5b, 0xaa, 0xfb, 0x29, - 0x58, 0xfd, 0xfa, 0x86, 0x75, 0x5f, 0x42, 0xab, 0xdc, 0x29, 0x0f, 0x9a, 0x5a, 0xa6, 0xa3, 0xf2, - 0xe5, 0x0b, 0xf5, 0x62, 0x6a, 0x73, 0xff, 0xf0, 0xf1, 0x6f, 0xbe, 0x3b, 0x3d, 0x27, 0xce, 0xd9, - 0x39, 0x71, 0x2e, 0xcf, 0x09, 0xfa, 0x98, 0x13, 0xf4, 0x35, 0x27, 0xe8, 0x47, 0x4e, 0xd0, 0x69, - 0x4e, 0xd0, 0xcf, 0x9c, 0xa0, 0x5f, 0x39, 0x71, 0x2e, 0x73, 0x82, 0x3e, 0x5f, 0x10, 0xe7, 0xf4, - 0x82, 0x38, 0x67, 0x17, 0xc4, 0x79, 0xbb, 0x1c, 0x0a, 0x3d, 0x1c, 0x1f, 0xf4, 0x7d, 0x19, 0x0f, - 0xc2, 0x8c, 0x1d, 0xb2, 0x84, 0x0d, 0x22, 0x39, 0x12, 0x83, 0xa3, 0xb5, 0xc1, 0xf5, 0x3f, 0x9c, - 0x83, 0x96, 0xfd, 0x59, 0xfb, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x9e, 0x82, 0x05, 0x2f, 0xac, 0x04, - 0x00, 0x00, + // 681 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0x41, 0x4f, 0x13, 0x41, + 0x18, 0xdd, 0xb1, 0xb5, 0x2d, 0x5f, 0x81, 0xe0, 0x48, 0xcc, 0x5a, 0xcc, 0xa6, 0xee, 0x41, 0x08, + 0xc6, 0xd6, 0x40, 0x24, 0x9c, 0xc1, 0x03, 0x89, 0xa2, 0x38, 0x10, 0x0f, 0x26, 0xc6, 0x2c, 0xdb, + 0x61, 0x3b, 0xe9, 0xee, 0xce, 0x66, 0x67, 0x0a, 0xe8, 0xc9, 0x5f, 0x60, 0x8c, 0x17, 0xff, 0x82, + 0x57, 0x7f, 0x82, 0x37, 0x8f, 0x1c, 0x39, 0x4a, 0xb9, 0x78, 0xe4, 0x27, 0x98, 0x9d, 0x99, 0x2d, + 0xdd, 0x25, 0x8a, 0xa7, 0xce, 0xf7, 0xe6, 0x7d, 0x6f, 0xdf, 0xcc, 0xf7, 0x3a, 0x70, 0x37, 0x19, + 0x04, 0xdd, 0x90, 0x07, 0x49, 0xca, 0x25, 0xef, 0x8a, 0x01, 0x95, 0x7e, 0xbf, 0xa3, 0x0a, 0xdc, + 0xc8, 0xe1, 0xd6, 0x42, 0x81, 0x94, 0x2f, 0x34, 0xcd, 0x7d, 0x01, 0xf3, 0xaf, 0x86, 0x5e, 0x2c, + 0x59, 0x48, 0x77, 0x55, 0xfb, 0xb6, 0x27, 0x53, 0x76, 0x8c, 0xd7, 0xa0, 0x76, 0xe8, 0x85, 0x43, + 0x2a, 0x6c, 0xd4, 0xae, 0x2c, 0x35, 0x57, 0x9c, 0xce, 0xb8, 0xb1, 0xc8, 0x7f, 0x4d, 0x7d, 0xc9, + 0x53, 0x62, 0xd8, 0xee, 0x4e, 0x59, 0x4f, 0xef, 0xe3, 0x75, 0xa8, 0x0b, 0x2f, 0x4a, 0xc2, 0xeb, + 0x05, 0x77, 0x15, 0x8d, 0xe4, 0x74, 0xf7, 0x13, 0x2a, 0x4b, 0x6a, 0x06, 0x7e, 0x00, 0xe8, 0xc0, + 0x46, 0x6d, 0xb4, 0xd4, 0x5c, 0xb1, 0xff, 0x26, 0x46, 0xd0, 0x01, 0xbe, 0x0f, 0xd3, 0x92, 0x45, + 0x54, 0x48, 0x2f, 0x4a, 0xde, 0x45, 0xc2, 0xbe, 0xd1, 0x46, 0x4b, 0x15, 0xd2, 0x1c, 0x63, 0xdb, + 0x02, 0x3f, 0x84, 0x5a, 0x44, 0x65, 0xca, 0x7c, 0xbb, 0xa2, 0xcc, 0xdd, 0xbe, 0xd4, 0x7b, 0xee, + 0xed, 0xd3, 0x70, 0xc7, 0x63, 0x29, 0x31, 0x14, 0x37, 0x80, 0xd9, 0xe2, 0x47, 0xf0, 0x23, 0xa8, + 0xcb, 0x1e, 0x0b, 0xa8, 0x90, 0xc6, 0xcf, 0xad, 0xcb, 0xfe, 0xbd, 0xa7, 0x6a, 0x63, 0xcb, 0x22, + 0x39, 0x07, 0xdf, 0x83, 0x46, 0xaf, 0xa7, 0x87, 0xa5, 0xcc, 0x4c, 0x6f, 0x59, 0x64, 0x8c, 0x6c, + 0x34, 0xa0, 0xa6, 0x57, 0xee, 0x0f, 0x04, 0x75, 0xd3, 0x8e, 0xe7, 0xa0, 0x12, 0xb1, 0x58, 0xc9, + 0x23, 0x92, 0x2d, 0x15, 0xe2, 0x1d, 0x2b, 0x81, 0x0c, 0xf1, 0x8e, 0x71, 0x1b, 0x9a, 0x3e, 0x8f, + 0x92, 0x94, 0x0a, 0xc1, 0x78, 0x6c, 0x57, 0xd4, 0xce, 0x24, 0x84, 0xd7, 0x61, 0x2a, 0x49, 0xb9, + 0x4f, 0x85, 0xa0, 0x3d, 0xbb, 0xaa, 0x8e, 0xda, 0xba, 0x62, 0xb5, 0xb3, 0x49, 0x63, 0x99, 0x72, + 0xd6, 0x23, 0x97, 0xe4, 0xd6, 0x1a, 0x34, 0x72, 0x18, 0x63, 0xa8, 0x46, 0xd4, 0xcb, 0xcd, 0xa8, + 0x35, 0xbe, 0x03, 0xb5, 0x23, 0xca, 0x82, 0xbe, 0x34, 0x86, 0x4c, 0xe5, 0x7e, 0x80, 0xd9, 0x4d, + 0x3e, 0x8c, 0xe5, 0x36, 0x8b, 0xcd, 0x65, 0xcd, 0xc3, 0xcd, 0x1e, 0x4d, 0x64, 0x5f, 0xb5, 0xcf, + 0x10, 0x5d, 0x64, 0xe8, 0x11, 0xeb, 0x49, 0x7d, 0x21, 0x33, 0x44, 0x17, 0xb8, 0x05, 0x0d, 0x3f, + 0xeb, 0xa6, 0xa9, 0x50, 0x93, 0x41, 0x64, 0x5c, 0x67, 0xa7, 0xed, 0xbf, 0x4f, 0x68, 0x1a, 0xf2, + 0x20, 0xe4, 0x81, 0x5d, 0xcd, 0x2e, 0x92, 0x4c, 0x42, 0xee, 0x57, 0x04, 0xf3, 0xc5, 0x8f, 0x9b, + 0x30, 0x96, 0x13, 0x81, 0xae, 0x26, 0xe2, 0x71, 0x3e, 0x05, 0x65, 0xa8, 0x90, 0xb0, 0xa2, 0x24, + 0x31, 0x3c, 0xbc, 0x0c, 0x75, 0x1d, 0x10, 0x61, 0x42, 0x34, 0x57, 0x0a, 0x91, 0x20, 0x39, 0xc1, + 0x7d, 0x02, 0x35, 0x0d, 0x4d, 0x24, 0x0f, 0x5d, 0x9f, 0xbc, 0xef, 0x08, 0xaa, 0x7b, 0x3c, 0x79, + 0x86, 0x97, 0xa1, 0xe2, 0x1b, 0xdf, 0xff, 0xb2, 0x96, 0x91, 0xf0, 0x22, 0x54, 0x43, 0x26, 0xb2, + 0xb9, 0x94, 0xf4, 0x33, 0xa5, 0x8e, 0xd2, 0x57, 0x84, 0xf2, 0x85, 0x56, 0xae, 0x5c, 0x68, 0x6b, + 0x05, 0xaa, 0x19, 0x3f, 0x1b, 0x16, 0x3d, 0xa4, 0xb1, 0x4e, 0xfb, 0x14, 0xd1, 0x45, 0x86, 0xaa, + 0xe1, 0x98, 0x04, 0xe8, 0xc2, 0xfd, 0x82, 0x00, 0xb2, 0x2f, 0x99, 0x77, 0x65, 0xb5, 0xf4, 0xae, + 0x2c, 0x14, 0xfd, 0x68, 0x56, 0xa7, 0xf8, 0xa8, 0xb4, 0x5e, 0x42, 0xcd, 0x4c, 0xce, 0x85, 0xaa, + 0xe4, 0xc9, 0xc0, 0x9c, 0x7c, 0xb6, 0xd8, 0x4c, 0xd4, 0xde, 0x7f, 0xfc, 0xdf, 0x37, 0xde, 0x9e, + 0x9c, 0x39, 0xd6, 0xe9, 0x99, 0x63, 0x5d, 0x9c, 0x39, 0xe8, 0xe3, 0xc8, 0x41, 0xdf, 0x46, 0x0e, + 0xfa, 0x39, 0x72, 0xd0, 0xc9, 0xc8, 0x41, 0xbf, 0x46, 0x0e, 0xfa, 0x3d, 0x72, 0xac, 0x8b, 0x91, + 0x83, 0x3e, 0x9f, 0x3b, 0xd6, 0xc9, 0xb9, 0x63, 0x9d, 0x9e, 0x3b, 0xd6, 0x9b, 0xc5, 0x80, 0xc9, + 0xfe, 0x70, 0xbf, 0xe3, 0xf3, 0xa8, 0x1b, 0xa4, 0xde, 0x81, 0x17, 0x7b, 0xdd, 0x90, 0x0f, 0x58, + 0xf7, 0x70, 0xb5, 0x3b, 0xf9, 0xc6, 0xee, 0xd7, 0xd4, 0xcf, 0xea, 0x9f, 0x00, 0x00, 0x00, 0xff, + 0xff, 0xab, 0x2f, 0x32, 0x74, 0x9f, 0x05, 0x00, 0x00, } func (this *QuantileSketchMatrix) Equal(that interface{}) bool { @@ -968,6 +1083,73 @@ func (this *CountMinSketch) Equal(that interface{}) bool { return false } } + if !bytes.Equal(this.Hyperloglog, that1.Hyperloglog) { + return false + } + return true +} +func (this *CountMinSketchVector) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CountMinSketchVector) + if !ok { + that2, ok := that.(CountMinSketchVector) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TimestampMs != that1.TimestampMs { + return false + } + if !this.Sketch.Equal(that1.Sketch) { + return false + } + if len(this.Metrics) != len(that1.Metrics) { + return false + } + for i := range this.Metrics { + if !this.Metrics[i].Equal(that1.Metrics[i]) { + return false + } + } + return true +} +func (this *Labels) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Labels) + if !ok { + that2, ok := that.(Labels) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Metric) != len(that1.Metric) { + return false + } + for i := range this.Metric { + if !this.Metric[i].Equal(that1.Metric[i]) { + return false + } + } return true } func (this *TopK) Equal(that interface{}) bool { @@ -1186,11 +1368,40 @@ func (this *CountMinSketch) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 7) + s := make([]string, 0, 8) s = append(s, "&logproto.CountMinSketch{") s = append(s, "Depth: "+fmt.Sprintf("%#v", this.Depth)+",\n") s = append(s, "Width: "+fmt.Sprintf("%#v", this.Width)+",\n") s = append(s, "Counters: "+fmt.Sprintf("%#v", this.Counters)+",\n") + s = append(s, "Hyperloglog: "+fmt.Sprintf("%#v", this.Hyperloglog)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *CountMinSketchVector) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&logproto.CountMinSketchVector{") + s = append(s, "TimestampMs: "+fmt.Sprintf("%#v", this.TimestampMs)+",\n") + if this.Sketch != nil { + s = append(s, "Sketch: "+fmt.Sprintf("%#v", this.Sketch)+",\n") + } + if this.Metrics != nil { + s = append(s, "Metrics: "+fmt.Sprintf("%#v", this.Metrics)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Labels) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&logproto.Labels{") + if this.Metric != nil { + s = append(s, "Metric: "+fmt.Sprintf("%#v", this.Metric)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -1559,21 +1770,20 @@ func (m *CountMinSketch) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.Hyperloglog) > 0 { + i -= len(m.Hyperloglog) + copy(dAtA[i:], m.Hyperloglog) + i = encodeVarintSketch(dAtA, i, uint64(len(m.Hyperloglog))) + i-- + dAtA[i] = 0x22 + } if len(m.Counters) > 0 { - dAtA4 := make([]byte, len(m.Counters)*10) - var j3 int - for _, num := range m.Counters { - for num >= 1<<7 { - dAtA4[j3] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j3++ - } - dAtA4[j3] = uint8(num) - j3++ - } - i -= j3 - copy(dAtA[i:], dAtA4[:j3]) - i = encodeVarintSketch(dAtA, i, uint64(j3)) + for iNdEx := len(m.Counters) - 1; iNdEx >= 0; iNdEx-- { + f3 := math.Float64bits(float64(m.Counters[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f3)) + } + i = encodeVarintSketch(dAtA, i, uint64(len(m.Counters)*8)) i-- dAtA[i] = 0x1a } @@ -1590,6 +1800,97 @@ func (m *CountMinSketch) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *CountMinSketchVector) 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 *CountMinSketchVector) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CountMinSketchVector) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Metrics) > 0 { + for iNdEx := len(m.Metrics) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Metrics[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSketch(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.Sketch != nil { + { + size, err := m.Sketch.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSketch(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.TimestampMs != 0 { + i = encodeVarintSketch(dAtA, i, uint64(m.TimestampMs)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Labels) 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 *Labels) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Labels) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Metric) > 0 { + for iNdEx := len(m.Metric) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Metric[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSketch(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *TopK) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1667,9 +1968,10 @@ func (m *TopK_Pair) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.Count != 0 { - i = encodeVarintSketch(dAtA, i, uint64(m.Count)) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Count)))) i-- - dAtA[i] = 0x10 + dAtA[i] = 0x11 } if len(m.Event) > 0 { i -= len(m.Event) @@ -1909,23 +2211,60 @@ func (m *CountMinSketch) Size() (n int) { n += 1 + sovSketch(uint64(m.Width)) } if len(m.Counters) > 0 { - l = 0 - for _, e := range m.Counters { - l += sovSketch(uint64(e)) - } - n += 1 + sovSketch(uint64(l)) + l + n += 1 + sovSketch(uint64(len(m.Counters)*8)) + len(m.Counters)*8 + } + l = len(m.Hyperloglog) + if l > 0 { + n += 1 + l + sovSketch(uint64(l)) } return n } -func (m *TopK) Size() (n int) { +func (m *CountMinSketchVector) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.Cms != nil { - l = m.Cms.Size() + if m.TimestampMs != 0 { + n += 1 + sovSketch(uint64(m.TimestampMs)) + } + if m.Sketch != nil { + l = m.Sketch.Size() + n += 1 + l + sovSketch(uint64(l)) + } + if len(m.Metrics) > 0 { + for _, e := range m.Metrics { + l = e.Size() + n += 1 + l + sovSketch(uint64(l)) + } + } + return n +} + +func (m *Labels) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Metric) > 0 { + for _, e := range m.Metric { + l = e.Size() + n += 1 + l + sovSketch(uint64(l)) + } + } + return n +} + +func (m *TopK) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Cms != nil { + l = m.Cms.Size() n += 1 + l + sovSketch(uint64(l)) } if len(m.List) > 0 { @@ -1952,7 +2291,7 @@ func (m *TopK_Pair) Size() (n int) { n += 1 + l + sovSketch(uint64(l)) } if m.Count != 0 { - n += 1 + sovSketch(uint64(m.Count)) + n += 9 } return n } @@ -2108,6 +2447,39 @@ func (this *CountMinSketch) String() string { `Depth:` + fmt.Sprintf("%v", this.Depth) + `,`, `Width:` + fmt.Sprintf("%v", this.Width) + `,`, `Counters:` + fmt.Sprintf("%v", this.Counters) + `,`, + `Hyperloglog:` + fmt.Sprintf("%v", this.Hyperloglog) + `,`, + `}`, + }, "") + return s +} +func (this *CountMinSketchVector) String() string { + if this == nil { + return "nil" + } + repeatedStringForMetrics := "[]*Labels{" + for _, f := range this.Metrics { + repeatedStringForMetrics += strings.Replace(f.String(), "Labels", "Labels", 1) + "," + } + repeatedStringForMetrics += "}" + s := strings.Join([]string{`&CountMinSketchVector{`, + `TimestampMs:` + fmt.Sprintf("%v", this.TimestampMs) + `,`, + `Sketch:` + strings.Replace(this.Sketch.String(), "CountMinSketch", "CountMinSketch", 1) + `,`, + `Metrics:` + repeatedStringForMetrics + `,`, + `}`, + }, "") + return s +} +func (this *Labels) String() string { + if this == nil { + return "nil" + } + repeatedStringForMetric := "[]*LabelPair{" + for _, f := range this.Metric { + repeatedStringForMetric += strings.Replace(fmt.Sprintf("%v", f), "LabelPair", "LabelPair", 1) + "," + } + repeatedStringForMetric += "}" + s := strings.Join([]string{`&Labels{`, + `Metric:` + repeatedStringForMetric + `,`, `}`, }, "") return s @@ -2874,23 +3246,15 @@ func (m *CountMinSketch) Unmarshal(dAtA []byte) error { } } case 3: - if wireType == 0 { - var v uint32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSketch - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF } - m.Counters = append(m.Counters, v) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.Counters = append(m.Counters, v2) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -2918,37 +3282,286 @@ func (m *CountMinSketch) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count + elementCount = packedLen / 8 if elementCount != 0 && len(m.Counters) == 0 { - m.Counters = make([]uint32, 0, elementCount) + m.Counters = make([]float64, 0, elementCount) } for iNdEx < postIndex { - var v uint32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSketch - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF } - m.Counters = append(m.Counters, v) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.Counters = append(m.Counters, v2) } } else { return fmt.Errorf("proto: wrong wireType = %d for field Counters", wireType) } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hyperloglog", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSketch + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSketch + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSketch + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hyperloglog = append(m.Hyperloglog[:0], dAtA[iNdEx:postIndex]...) + if m.Hyperloglog == nil { + m.Hyperloglog = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSketch(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSketch + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthSketch + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CountMinSketchVector) 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 ErrIntOverflowSketch + } + 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: CountMinSketchVector: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CountMinSketchVector: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + } + m.TimestampMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSketch + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimestampMs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sketch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSketch + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSketch + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSketch + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Sketch == nil { + m.Sketch = &CountMinSketch{} + } + if err := m.Sketch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metrics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSketch + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSketch + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSketch + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Metrics = append(m.Metrics, &Labels{}) + if err := m.Metrics[len(m.Metrics)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSketch(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSketch + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthSketch + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Labels) 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 ErrIntOverflowSketch + } + 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: Labels: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Labels: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metric", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSketch + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSketch + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSketch + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Metric = append(m.Metric, &LabelPair{}) + if err := m.Metric[len(m.Metric)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSketch(dAtA[iNdEx:]) @@ -3192,24 +3805,16 @@ func (m *TopK_Pair) Unmarshal(dAtA []byte) error { m.Event = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 0 { + if wireType != 1 { return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) } - m.Count = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSketch - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Count |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = float64(math.Float64frombits(v)) default: iNdEx = preIndex skippy, err := skipSketch(dAtA[iNdEx:]) diff --git a/pkg/logproto/sketch.proto b/pkg/logproto/sketch.proto index e551716db3742..dc89fd73d1df8 100644 --- a/pkg/logproto/sketch.proto +++ b/pkg/logproto/sketch.proto @@ -45,7 +45,20 @@ message CountMinSketch { uint32 width = 2; // counters is a matrix of depth * width. - repeated uint32 counters = 3; + repeated double counters = 3; + + bytes hyperloglog = 4; +} + +message CountMinSketchVector { + int64 timestamp_ms = 1; + CountMinSketch sketch = 2; + + repeated Labels metrics = 3; +} + +message Labels { + repeated LabelPair metric = 1; } message TopK { @@ -53,7 +66,7 @@ message TopK { message Pair { string event = 1; - uint32 count = 2; + double count = 2; } repeated Pair list = 2; diff --git a/pkg/logql/accumulator.go b/pkg/logql/accumulator.go index 3313e9c17d422..4091875a18375 100644 --- a/pkg/logql/accumulator.go +++ b/pkg/logql/accumulator.go @@ -115,6 +115,82 @@ func (a *QuantileSketchAccumulator) Result() []logqlmodel.Result { } } +type CountMinSketchAccumulator struct { + vec *CountMinSketchVector + + stats stats.Result // for accumulating statistics from downstream requests + headers map[string][]string // for accumulating headers from downstream requests + warnings map[string]struct{} // for accumulating warnings from downstream requests} +} + +// newCountMinSketchAccumulator returns an accumulator for sharded +// count min sketch queries that merges results as they come in. +func newCountMinSketchAccumulator() *CountMinSketchAccumulator { + return &CountMinSketchAccumulator{ + headers: make(map[string][]string), + warnings: make(map[string]struct{}), + } +} + +func (a *CountMinSketchAccumulator) Accumulate(_ context.Context, res logqlmodel.Result, _ int) error { + if res.Data.Type() != CountMinSketchVectorType { + return fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), CountMinSketchVectorType) + } + data, ok := res.Data.(CountMinSketchVector) + if !ok { + return fmt.Errorf("unexpected matrix type: got (%T), want (CountMinSketchVector)", res.Data) + } + + // TODO(owen-d/ewelch): Shard counts should be set by the querier + // so we don't have to do it in tricky ways in multiple places. + // See pkg/logql/downstream.go:DownstreamEvaluator.Downstream + // for another example. + if res.Statistics.Summary.Shards == 0 { + res.Statistics.Summary.Shards = 1 + } + a.stats.Merge(res.Statistics) + metadata.ExtendHeaders(a.headers, res.Headers) + + for _, w := range res.Warnings { + a.warnings[w] = struct{}{} + } + + if a.vec == nil { + a.vec = &data // TODO: maybe the matrix should already be a pointeer + return nil + } + + var err error + a.vec, err = a.vec.Merge(&data) + a.stats.Merge(res.Statistics) + return err +} + +func (a *CountMinSketchAccumulator) Result() []logqlmodel.Result { + headers := make([]*definitions.PrometheusResponseHeader, 0, len(a.headers)) + for name, vals := range a.headers { + headers = append( + headers, + &definitions.PrometheusResponseHeader{ + Name: name, + Values: vals, + }, + ) + } + + warnings := maps.Keys(a.warnings) + sort.Strings(warnings) + + return []logqlmodel.Result{ + { + Data: a.vec, + Headers: headers, + Warnings: warnings, + Statistics: a.stats, + }, + } +} + // heap impl for keeping only the top n results across m streams // importantly, AccumulatedStreams is _bounded_, so it will only // store the top `limit` results across all streams. @@ -334,7 +410,6 @@ func (acc *AccumulatedStreams) appendTo(dst, src *logproto.Stream) { acc.count += len(src.Entries) heap.Fix(acc, acc.labelmap[dst.Labels]) - } // Pop returns a stream with one entry. It pops the first entry of the first stream diff --git a/pkg/logql/count_min_sketch.go b/pkg/logql/count_min_sketch.go new file mode 100644 index 0000000000000..2d2e607f09598 --- /dev/null +++ b/pkg/logql/count_min_sketch.go @@ -0,0 +1,332 @@ +package logql + +import ( + "container/heap" + "fmt" + + "github.com/axiomhq/hyperloglog" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" + promql_parser "github.com/prometheus/prometheus/promql/parser" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/sketch" + "github.com/grafana/loki/v3/pkg/logql/syntax" +) + +const ( + CountMinSketchVectorType = "CountMinSketchVector" + + epsilon = 0.0001 + // delta of 0.01 results in a sketch size of 27183 * 7 * 4 bytes = 761,124 bytes, 0.05 would yield 543,660 bytes + delta = 0.01 +) + +// CountMinSketchVector tracks the count or sum of values of a metric, ie list of label value pairs. It's storage for +// the values is upper bound bu delta and epsilon. To limit the storage for labels see HeapCountMinSketchVector. +// The main use case is for a topk approximation. +type CountMinSketchVector struct { + T int64 + F *sketch.CountMinSketch + + Metrics []labels.Labels +} + +func (CountMinSketchVector) SampleVector() promql.Vector { + return promql.Vector{} +} + +func (CountMinSketchVector) QuantileSketchVec() ProbabilisticQuantileVector { + return ProbabilisticQuantileVector{} +} + +func (v CountMinSketchVector) CountMinSketchVec() CountMinSketchVector { + return v +} + +func (v *CountMinSketchVector) Merge(right *CountMinSketchVector) (*CountMinSketchVector, error) { + // The underlying CMS implementation already merges the HLL sketches that are part of that structure. + err := v.F.Merge(right.F) + if err != nil { + return v, err + } + + // Merge labels without duplication. Note: the CMS does not limit the number of labels as the + // HeapCountMinSketchVector does. + processed := map[string]struct{}{} + for _, l := range v.Metrics { + processed[l.String()] = struct{}{} + } + + for _, r := range right.Metrics { + if _, duplicate := processed[r.String()]; !duplicate { + processed[r.String()] = struct{}{} + v.Metrics = append(v.Metrics, r) + } + } + + return v, nil +} + +func (CountMinSketchVector) String() string { + return "CountMinSketchVector()" +} + +func (CountMinSketchVector) Type() promql_parser.ValueType { return CountMinSketchVectorType } + +func (v CountMinSketchVector) ToProto() (*logproto.CountMinSketchVector, error) { + p := &logproto.CountMinSketchVector{ + TimestampMs: v.T, + Metrics: make([]*logproto.Labels, len(v.Metrics)), + Sketch: &logproto.CountMinSketch{ + Depth: v.F.Depth, + Width: v.F.Width, + }, + } + + // insert the hll sketch + hllBytes, err := v.F.HyperLogLog.MarshalBinary() + if err != nil { + return nil, err + } + p.Sketch.Hyperloglog = hllBytes + + // Serialize CMS + p.Sketch.Counters = make([]float64, 0, v.F.Depth*v.F.Width) + for row := uint32(0); row < v.F.Depth; row++ { + p.Sketch.Counters = append(p.Sketch.Counters, v.F.Counters[row]...) + } + + // Serialize metric labels + for i, metric := range v.Metrics { + p.Metrics[i] = &logproto.Labels{ + Metric: make([]*logproto.LabelPair, len(metric)), + } + for j, pair := range metric { + p.Metrics[i].Metric[j] = &logproto.LabelPair{ + Name: pair.Name, + Value: pair.Value, + } + } + } + + return p, nil +} + +func CountMinSketchVectorFromProto(p *logproto.CountMinSketchVector) (CountMinSketchVector, error) { + vec := CountMinSketchVector{ + T: p.TimestampMs, + Metrics: make([]labels.Labels, len(p.Metrics)), + } + + // Deserialize CMS + var err error + vec.F, err = sketch.NewCountMinSketch(p.Sketch.Width, p.Sketch.Depth) + if err != nil { + return vec, err + } + + hll := hyperloglog.New() + if err := hll.UnmarshalBinary(p.Sketch.Hyperloglog); err != nil { + return vec, err + } + vec.F.HyperLogLog = hll + + for row := 0; row < int(vec.F.Depth); row++ { + s := row * int(vec.F.Width) + e := s + int(vec.F.Width) + copy(vec.F.Counters[row], p.Sketch.Counters[s:e]) + } + + // Deserialize metric labels + for i, in := range p.Metrics { + lbls := make(labels.Labels, len(in.Metric)) + for j, labelPair := range in.Metric { + lbls[j].Name = labelPair.Name + lbls[j].Value = labelPair.Value + } + vec.Metrics[i] = lbls + } + + return vec, nil +} + +// HeapCountMinSketchVector is a CountMinSketchVector that keeps the number of metrics to a defined maximum. +type HeapCountMinSketchVector struct { + CountMinSketchVector + + // internal set of observed events + observed map[string]struct{} + maxLabels int +} + +func NewHeapCountMinSketchVector(ts int64, metricsLength, maxLabels int) HeapCountMinSketchVector { + f, _ := sketch.NewCountMinSketchFromErrorAndProbability(epsilon, delta) + + if metricsLength >= maxLabels { + metricsLength = maxLabels + } + + return HeapCountMinSketchVector{ + CountMinSketchVector: CountMinSketchVector{ + T: ts, + F: f, + Metrics: make([]labels.Labels, 0, metricsLength), + }, + observed: make(map[string]struct{}), + maxLabels: maxLabels, + } +} + +func (v *HeapCountMinSketchVector) Add(metric labels.Labels, value float64) { + // TODO: we save a lot of allocations by reusing the buffer inside metric.String + metricString := metric.String() + v.F.Add(metricString, value) + + // Add our metric if we haven't seen it + if _, ok := v.observed[metricString]; !ok { + heap.Push(v, metric) + v.observed[metricString] = struct{}{} + } else if v.Metrics[0].String() == metricString { + // The smalles element has been updated to fix the heap. + heap.Fix(v, 0) + } + + // The maximum number of labels has been reached, so drop the smallest element. + if len(v.Metrics) > v.maxLabels { + metric := heap.Pop(v).(labels.Labels) + delete(v.observed, metric.String()) + } +} + +func (v HeapCountMinSketchVector) Len() int { + return len(v.Metrics) +} + +func (v HeapCountMinSketchVector) Less(i, j int) bool { + left := v.F.Count(v.Metrics[i].String()) + right := v.F.Count(v.Metrics[j].String()) + return left < right +} + +func (v HeapCountMinSketchVector) Swap(i, j int) { + v.Metrics[i], v.Metrics[j] = v.Metrics[j], v.Metrics[i] +} + +func (v *HeapCountMinSketchVector) Push(x any) { + v.Metrics = append(v.Metrics, x.(labels.Labels)) +} + +func (v *HeapCountMinSketchVector) Pop() any { + old := v.Metrics + n := len(old) + x := old[n-1] + v.Metrics = old[0 : n-1] + return x +} + +// JoinCountMinSketchVector joins the results from stepEvaluator into a CountMinSketchVector. +func JoinCountMinSketchVector(_ bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { + vec := r.CountMinSketchVec() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + if GetRangeType(params) != InstantType { + return nil, fmt.Errorf("count min sketches are only supported on instant queries") + } + + return vec, nil +} + +func newCountMinSketchVectorAggEvaluator(nextEvaluator StepEvaluator, expr *syntax.VectorAggregationExpr, maxLabels int) (*countMinSketchVectorAggEvaluator, error) { + if expr.Grouping.Groups != nil { + return nil, fmt.Errorf("count min sketch vector aggregation does not support any grouping") + } + + return &countMinSketchVectorAggEvaluator{ + nextEvaluator: nextEvaluator, + expr: expr, + buf: make([]byte, 0, 1024), + lb: labels.NewBuilder(nil), + maxLabels: maxLabels, + }, nil +} + +// countMinSketchVectorAggEvaluator processes sample vectors and aggregates them in a count min sketch with a heap. +type countMinSketchVectorAggEvaluator struct { + nextEvaluator StepEvaluator + expr *syntax.VectorAggregationExpr + buf []byte + lb *labels.Builder + maxLabels int +} + +func (e *countMinSketchVectorAggEvaluator) Next() (bool, int64, StepResult) { + next, ts, r := e.nextEvaluator.Next() + + if !next { + return false, 0, CountMinSketchVector{} + } + vec := r.SampleVector() + + result := NewHeapCountMinSketchVector(ts, len(vec), e.maxLabels) + for _, s := range vec { + result.Add(s.Metric, s.F) + } + return next, ts, result +} + +func (e *countMinSketchVectorAggEvaluator) Explain(parent Node) { + b := parent.Child("CountMinSketchVectorAgg") + e.nextEvaluator.Explain(b) +} + +func (e *countMinSketchVectorAggEvaluator) Close() error { + return e.nextEvaluator.Close() +} + +func (e *countMinSketchVectorAggEvaluator) Error() error { + return e.nextEvaluator.Error() +} + +// CountMinSketchVectorStepEvaluator evaluates a count min sketch into a promql.Vector. +type CountMinSketchVectorStepEvaluator struct { + exhausted bool + vec *CountMinSketchVector +} + +var _ StepEvaluator = NewQuantileSketchVectorStepEvaluator(nil, 0) + +func NewCountMinSketchVectorStepEvaluator(vec *CountMinSketchVector) *CountMinSketchVectorStepEvaluator { + return &CountMinSketchVectorStepEvaluator{ + exhausted: false, + vec: vec, + } +} + +func (e *CountMinSketchVectorStepEvaluator) Next() (bool, int64, StepResult) { + if e.exhausted { + return false, 0, SampleVector{} + } + + vec := make(promql.Vector, len(e.vec.Metrics)) + + for i, labels := range e.vec.Metrics { + + f := e.vec.F.Count(labels.String()) + + vec[i] = promql.Sample{ + T: e.vec.T, + F: float64(f), + Metric: labels, + } + } + + return true, e.vec.T, SampleVector(vec) +} + +func (*CountMinSketchVectorStepEvaluator) Close() error { return nil } + +func (*CountMinSketchVectorStepEvaluator) Error() error { return nil } diff --git a/pkg/logql/count_min_sketch_test.go b/pkg/logql/count_min_sketch_test.go new file mode 100644 index 0000000000000..2b2c2960e4515 --- /dev/null +++ b/pkg/logql/count_min_sketch_test.go @@ -0,0 +1,88 @@ +package logql + +import ( + "testing" + + "github.com/grafana/loki/v3/pkg/logproto" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logql/sketch" +) + +func TestHeapCountMinSketchVectorHeap(t *testing.T) { + v := NewHeapCountMinSketchVector(0, 0, 3) + + a := labels.Labels{{Name: "event", Value: "a"}} + b := labels.Labels{{Name: "event", Value: "b"}} + c := labels.Labels{{Name: "event", Value: "c"}} + d := labels.Labels{{Name: "event", Value: "d"}} + + v.Add(a, 2.0) + v.Add(b, 4.0) + v.Add(d, 5.5) + require.Equal(t, "a", v.Metrics[0][0].Value) + + // Adding c drops a + v.Add(c, 3.0) + require.Equal(t, "c", v.Metrics[0][0].Value) + require.Len(t, v.Metrics, v.maxLabels) + require.NotContains(t, v.observed, a.String()) + + // Increasing c to 6.0 should make b with 4,0 the smallest + v.Add(c, 3.0) + require.Equal(t, "b", v.Metrics[0][0].Value) + + // Increasing a to 5.0 drops b because it's the smallest + v.Add(a, 3.0) + require.Equal(t, "a", v.Metrics[0][0].Value) + require.Len(t, v.Metrics, v.maxLabels) + require.NotContains(t, v.observed, b.String()) + + // Verify final list + final := make([]string, v.maxLabels) + for i, metric := range v.Metrics { + final[i] = metric[0].Value + } + require.ElementsMatch(t, []string{"a", "d", "c"}, final) +} + +func TestCountMinSketchSerialization(t *testing.T) { + metric := []labels.Label{{Name: "foo", Value: "bar"}} + cms, err := sketch.NewCountMinSketch(4, 2) + require.NoError(t, err) + vec := HeapCountMinSketchVector{ + CountMinSketchVector: CountMinSketchVector{ + T: 42, + F: cms, + }, + observed: make(map[string]struct{}, 0), + maxLabels: 10_000, + } + vec.Add(metric, 42.0) + + hllBytes, _ := vec.F.HyperLogLog.MarshalBinary() + proto := &logproto.CountMinSketchVector{ + TimestampMs: 42, + Sketch: &logproto.CountMinSketch{ + Depth: 2, + Width: 4, + Counters: []float64{0, 0, 0, 42, 0, 42, 0, 0}, + Hyperloglog: hllBytes, + }, + Metrics: []*logproto.Labels{ + {Metric: []*logproto.LabelPair{{Name: "foo", Value: "bar"}}}, + }, + } + + actual, err := vec.ToProto() + require.NoError(t, err) + require.Equal(t, proto, actual) + + round, err := CountMinSketchVectorFromProto(actual) + require.NoError(t, err) + + // The HeapCountMinSketchVector is serialized to a CountMinSketchVector. + require.Equal(t, round, vec.CountMinSketchVector) +} diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 615ddd0f3f47d..6a577026a49b2 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -357,6 +357,34 @@ func (e *MergeLastOverTimeExpr) Walk(f syntax.WalkFn) { } } +type CountMinSketchEvalExpr struct { + syntax.SampleExpr + downstreams []DownstreamSampleExpr +} + +func (e CountMinSketchEvalExpr) String() string { + var sb strings.Builder + for i, d := range e.downstreams { + if i >= defaultMaxDepth { + break + } + + if i > 0 { + sb.WriteString(" ++ ") + } + + sb.WriteString(d.String()) + } + return fmt.Sprintf("CountMinSketchEval<%s>", sb.String()) +} + +func (e *CountMinSketchEvalExpr) Walk(f syntax.WalkFn) { + f(e) + for _, d := range e.downstreams { + d.Walk(f) + } +} + type Downstreamable interface { Downstreamer(context.Context) Downstreamer } @@ -435,7 +463,7 @@ func (errorQuerier) SelectSamples(_ context.Context, _ SelectSampleParams) (iter func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator { return &DownstreamEvaluator{ Downstreamer: downstreamer, - defaultEvaluator: NewDefaultEvaluator(&errorQuerier{}, 0), + defaultEvaluator: NewDefaultEvaluator(&errorQuerier{}, 0, 0), } } @@ -554,7 +582,6 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( xs := make([]promql.Matrix, 0, len(queries)) for _, res := range results { - switch data := res.Data.(type) { case promql.Matrix: xs = append(xs, data) @@ -591,7 +618,6 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( xs := make([]promql.Matrix, 0, len(queries)) for _, res := range results { - switch data := res.Data.(type) { case promql.Matrix: xs = append(xs, data) @@ -600,6 +626,40 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( } } return NewMergeLastOverTimeStepEvaluator(params, xs), nil + case *CountMinSketchEvalExpr: + queries := make([]DownstreamQuery, len(e.downstreams)) + + for i, d := range e.downstreams { + qry := DownstreamQuery{ + Params: ParamsWithExpressionOverride{ + Params: params, + ExpressionOverride: d.SampleExpr, + }, + } + if shard := d.shard; shard != nil { + qry.Params = ParamsWithShardsOverride{ + Params: qry.Params, + ShardsOverride: Shards{shard.Shard}.Encode(), + } + } + queries[i] = qry + } + + acc := newCountMinSketchAccumulator() + results, err := ev.Downstream(ctx, queries, acc) + if err != nil { + return nil, err + } + + if len(results) != 1 { + return nil, fmt.Errorf("unexpected results length for sharded count min sketch: got (%d), want (1)", len(results)) + } + + vector, ok := results[0].Data.(*CountMinSketchVector) + if !ok { + return nil, fmt.Errorf("unexpected matrix type: got (%T), want (CountMinSketchVector)", results[0].Data) + } + return NewCountMinSketchVectorStepEvaluator(vector), nil default: return ev.defaultEvaluator.NewStepEvaluator(ctx, nextEvFactory, e, params) } diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index b1082f3d6ad76..615bf80a9c276 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -2,6 +2,7 @@ package logql import ( "context" + "fmt" "math" "testing" "time" @@ -18,8 +19,10 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) -var nilShardMetrics = NewShardMapperMetrics(nil) -var nilRangeMetrics = NewRangeMapperMetrics(nil) +var ( + nilShardMetrics = NewShardMapperMetrics(nil) + nilRangeMetrics = NewRangeMapperMetrics(nil) +) func TestMappingEquivalence(t *testing.T) { var ( @@ -193,7 +196,9 @@ func TestMappingEquivalenceSketches(t *testing.T) { streams, ) - opts := EngineOpts{} + opts := EngineOpts{ + MaxCountMinSketchHeapSize: 10_000, + } regular := NewEngine(opts, q, NoLimits, log.NewNopLogger()) sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, NoLimits, log.NewNopLogger()) @@ -236,8 +241,8 @@ func TestMappingEquivalenceSketches(t *testing.T) { // plus set step and interval to 0 params, err := NewLiteralParams( tc.query, - time.Unix(0, int64(rounds+1)), - time.Unix(0, int64(rounds+1)), + time.Unix(1, 0), + time.Unix(1, 0), 0, 0, logproto.FORWARD, @@ -246,11 +251,12 @@ func TestMappingEquivalenceSketches(t *testing.T) { nil, ) require.NoError(t, err) - qry := regular.Query(params) + qry := regular.Query(params.Copy()) ctx := user.InjectOrgID(context.Background(), "fake") strategy := NewPowerOfTwoStrategy(ConstantShards(shards)) - mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk}) + _, _, mapped, err := mapper.Parse(params.GetExpression()) require.NoError(t, err) @@ -261,6 +267,7 @@ func TestMappingEquivalenceSketches(t *testing.T) { res, err := qry.Exec(ctx) require.NoError(t, err) + require.NotEmpty(t, res.Data.(promql.Vector)) shardedRes, err := shardedQry.Exec(ctx) require.NoError(t, err) @@ -270,6 +277,132 @@ func TestMappingEquivalenceSketches(t *testing.T) { } } +func TestApproxTopkSketches(t *testing.T) { + var ( + rounds = 20 + limit = 100 + ) + + limits := &fakeLimits{ + maxSeries: math.MaxInt64, + timeout: time.Hour, + } + + for _, tc := range []struct { + labelShards int + totalStreams int + shardedQuery string + regularQuery string + realtiveError float64 + //cardinalityEstimate int + }{ + // Note:our data generation results in less spread between topk things for 10k streams than for 100k streams + // if we have 1k streams, we can get much more accurate results for topk 10 than topk 100 + { + labelShards: 3, + totalStreams: 100, + shardedQuery: `approx_topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.0012, + //cardinalityEstimate: 3, + }, + { + labelShards: 10, + totalStreams: 100, + shardedQuery: `approx_topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(3, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.005, + }, + { + labelShards: 10, + totalStreams: 1_000, + shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.0015, + }, + { + labelShards: 100, + totalStreams: 1_000, + shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.022, + }, + { + labelShards: 100, + totalStreams: 10_000, + shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.008, + }, + { + labelShards: 100, + totalStreams: 100_000, + shardedQuery: `approx_topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + regularQuery: `topk(100, sum by (a) (sum_over_time ({a=~".+"} | logfmt | unwrap value [1s])))`, + realtiveError: 0.0015, + }, + } { + t.Run(fmt.Sprintf("%s/%d/%d", tc.shardedQuery, tc.labelShards, tc.totalStreams), func(t *testing.T) { + streams := randomStreams(tc.totalStreams, rounds+1, tc.labelShards, []string{"a", "b", "c", "d"}, true) + + q := NewMockQuerier( + tc.labelShards, + streams, + ) + + opts := EngineOpts{ + MaxCountMinSketchHeapSize: 10_000, + } + regular := NewEngine(opts, q, limits, log.NewNopLogger()) + sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, limits, log.NewNopLogger()) + + // for an instant query we set the start and end to the same timestamp + // plus set step and interval to 0 + params, err := NewLiteralParams( + tc.regularQuery, + time.Unix(1, 0), + time.Unix(1, 0), + 0, + 0, + logproto.FORWARD, + uint32(limit), + nil, + nil, + ) + require.NoError(t, err) + qry := regular.Query(params.Copy()) + ctx := user.InjectOrgID(context.Background(), "fake") + + strategy := NewPowerOfTwoStrategy(ConstantShards(tc.labelShards)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk}) + + params.queryString = tc.shardedQuery + params.queryExpr, err = syntax.ParseExpr(params.queryString) + require.NoError(t, err) + + _, _, mapped, err := mapper.Parse(params.GetExpression()) + require.NoError(t, err) + + shardedQry := sharded.Query(ctx, ParamsWithExpressionOverride{ + Params: params, + ExpressionOverride: mapped, + }) + + res, err := qry.Exec(ctx) + require.NoError(t, err) + require.NotEmpty(t, res.Data.(promql.Vector)) + + shardedRes, err := shardedQry.Exec(ctx) + require.NoError(t, err) + relativeErrorVector(t, res.Data.(promql.Vector), shardedRes.Data.(promql.Vector), tc.realtiveError) + + // we can't check this here currently because the CMS vector step evaluators Next function translates + // each steps probabilistic result into just a promql.Vector + // require.Equal(t, tc.cardinalityEstimate, res.Data.(CountMinSketchVector).F.HyperLogLog.Estimate()) + }) + } +} + func TestShardCounter(t *testing.T) { var ( shards = 3 @@ -591,7 +724,7 @@ func TestRangeMappingEquivalence(t *testing.T) { rangeQry := downstreamEngine.Query(ctx, ParamsWithExpressionOverride{Params: params, ExpressionOverride: rangeExpr}) rangeRes, err := rangeQry.Exec(ctx) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, res.Data, rangeRes.Data) }) @@ -661,14 +794,16 @@ func relativeErrorVector(t *testing.T, expected, actual promql.Vector, alpha flo e := make([]float64, len(expected)) a := make([]float64, len(expected)) + inTopk := 0 for i := 0; i < len(expected); i++ { - require.Equal(t, expected[i].Metric, actual[i].Metric) - - e[i] = expected[i].F - a[i] = expected[i].F + if labels.Equal(expected[i].Metric, actual[i].Metric) { + e[i] = expected[i].F + a[i] = actual[i].F + inTopk++ + } } + require.True(t, float64(inTopk/len(expected)) > 0.9, "not enough of the real topk elements were in the output %f", float64(inTopk/len(expected))) require.InEpsilonSlice(t, e, a, alpha) - } func TestFormat_ShardedExpr(t *testing.T) { diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 17e90520e7de5..3d7cb541bb9dc 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -146,10 +146,15 @@ type EngineOpts struct { // LogExecutingQuery will control if we log the query when Exec is called. LogExecutingQuery bool `yaml:"-"` + + // MaxCountMinSketchHeapSize is the maximum number of labels the heap for a topk query using a count min sketch + // can track. This impacts the memory usage and accuracy of a sharded probabilistic topk query. + MaxCountMinSketchHeapSize int `yaml:"max_count_min_sketch_heap_size"` } func (opts *EngineOpts) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.DurationVar(&opts.MaxLookBackPeriod, prefix+".engine.max-lookback-period", 30*time.Second, "The maximum amount of time to look back for log lines. Used only for instant log queries.") + f.IntVar(&opts.MaxCountMinSketchHeapSize, prefix+".engine.max-count-min-sketch-heap-size", 10_000, "The maximum number of labels the heap of a topk query using a count min sketch can track.") // Log executing query by default opts.LogExecutingQuery = true } @@ -176,7 +181,7 @@ func NewEngine(opts EngineOpts, q Querier, l Limits, logger log.Logger) *Engine } return &Engine{ logger: logger, - evaluatorFactory: NewDefaultEvaluator(q, opts.MaxLookBackPeriod), + evaluatorFactory: NewDefaultEvaluator(q, opts.MaxLookBackPeriod, opts.MaxCountMinSketchHeapSize), limits: l, opts: opts, } @@ -386,7 +391,11 @@ func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_ } return q.JoinSampleVector(next, vec, stepEvaluator, maxSeries, mfl) case ProbabilisticQuantileVector: - return MergeQuantileSketchVector(next, vec, stepEvaluator, q.params) + return JoinQuantileSketchVector(next, vec, stepEvaluator, q.params) + case CountMinSketchVector: + return JoinCountMinSketchVector(next, vec, stepEvaluator, q.params) + case HeapCountMinSketchVector: + return JoinCountMinSketchVector(next, vec.CountMinSketchVector, stepEvaluator, q.params) default: return nil, fmt.Errorf("unsupported result type: %T", r) } diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 491a0b2ff1168..c9b0340f6e102 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -282,15 +282,17 @@ func EvaluatorUnsupportedType(expr syntax.Expr, ev EvaluatorFactory) error { } type DefaultEvaluator struct { - maxLookBackPeriod time.Duration - querier Querier + maxLookBackPeriod time.Duration + maxCountMinSketchHeapSize int + querier Querier } // NewDefaultEvaluator constructs a DefaultEvaluator -func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) *DefaultEvaluator { +func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration, maxCountMinSketchHeapSize int) *DefaultEvaluator { return &DefaultEvaluator{ - querier: querier, - maxLookBackPeriod: maxLookBackPeriod, + querier: querier, + maxLookBackPeriod: maxLookBackPeriod, + maxCountMinSketchHeapSize: maxCountMinSketchHeapSize, } } @@ -350,7 +352,7 @@ func (ev *DefaultEvaluator) NewStepEvaluator( return newRangeAggEvaluator(iter.NewPeekingSampleIterator(it), rangExpr, q, rangExpr.Left.Offset) }) } - return newVectorAggEvaluator(ctx, nextEvFactory, e, q) + return newVectorAggEvaluator(ctx, nextEvFactory, e, q, ev.maxCountMinSketchHeapSize) case *syntax.RangeAggregationExpr: it, err := ev.querier.SelectSamples(ctx, SelectSampleParams{ &logproto.SampleQueryRequest{ @@ -391,7 +393,8 @@ func newVectorAggEvaluator( evFactory SampleEvaluatorFactory, expr *syntax.VectorAggregationExpr, q Params, -) (*VectorAggEvaluator, error) { + maxCountMinSketchHeapSize int, +) (StepEvaluator, error) { if expr.Grouping == nil { return nil, errors.Errorf("aggregation operator '%q' without grouping", expr.Operation) } @@ -401,6 +404,10 @@ func newVectorAggEvaluator( } sort.Strings(expr.Grouping.Groups) + if expr.Operation == syntax.OpTypeCountMinSketch { + return newCountMinSketchVectorAggEvaluator(nextEvaluator, expr, maxCountMinSketchHeapSize) + } + return &VectorAggEvaluator{ nextEvaluator: nextEvaluator, expr: expr, @@ -1198,7 +1205,8 @@ type VectorIterator struct { } func newVectorIterator(val float64, - stepMs, startMs, endMs int64) *VectorIterator { + stepMs, startMs, endMs int64, +) *VectorIterator { if stepMs == 0 { stepMs = 1 } @@ -1294,6 +1302,7 @@ func (e *LabelReplaceEvaluator) Next() (bool, int64, StepResult) { func (e *LabelReplaceEvaluator) Close() error { return e.nextEvaluator.Close() } + func (e *LabelReplaceEvaluator) Error() error { return e.nextEvaluator.Error() } diff --git a/pkg/logql/explain.go b/pkg/logql/explain.go index 22240f5804b35..82ad0d74edd54 100644 --- a/pkg/logql/explain.go +++ b/pkg/logql/explain.go @@ -67,6 +67,10 @@ func (e *mergeOverTimeStepEvaluator) Explain(parent Node) { parent.Child("MergeFirstOverTime") } +func (e *CountMinSketchVectorStepEvaluator) Explain(parent Node) { + parent.Child("CountMinSketchVector") +} + func (EmptyEvaluator[SampleVector]) Explain(parent Node) { parent.Child("Empty") } diff --git a/pkg/logql/explain_test.go b/pkg/logql/explain_test.go index d6984683aec1b..01b2e7a5f3161 100644 --- a/pkg/logql/explain_test.go +++ b/pkg/logql/explain_test.go @@ -14,7 +14,6 @@ import ( ) func TestExplain(t *testing.T) { - query := `topk(5, avg_over_time({app="loki"} |= "caller=metrics.go" | logfmt | unwrap bytes [5s]))` // TODO(karsten): Ideally the querier and downstreamer are not required @@ -25,7 +24,7 @@ func TestExplain(t *testing.T) { ctx := user.InjectOrgID(context.Background(), "fake") - defaultEv := NewDefaultEvaluator(querier, 30*time.Second) + defaultEv := NewDefaultEvaluator(querier, 30*time.Second, 10_000) downEv := &DownstreamEvaluator{Downstreamer: MockDownstreamer{regular}, defaultEvaluator: defaultEv} strategy := NewPowerOfTwoStrategy(ConstantShards(4)) @@ -46,8 +45,7 @@ func TestExplain(t *testing.T) { tree := NewTree() ev.Explain(tree) - expected := - `[topk, by ()] VectorAgg + expected := `[topk, by ()] VectorAgg └── Concat ├── VectorStep ├── ... diff --git a/pkg/logql/limits.go b/pkg/logql/limits.go index f9742dac53ef0..6ea7bc1c8a89c 100644 --- a/pkg/logql/limits.go +++ b/pkg/logql/limits.go @@ -8,9 +8,10 @@ import ( "github.com/grafana/loki/v3/pkg/util/validation" ) -var ( - NoLimits = &fakeLimits{maxSeries: math.MaxInt32} -) +var NoLimits = &fakeLimits{ + maxSeries: math.MaxInt32, + timeout: math.MaxInt32, +} // Limits allow the engine to fetch limits for a given users. type Limits interface { diff --git a/pkg/logql/log/vector.go b/pkg/logql/log/vector.go new file mode 100644 index 0000000000000..f145fcbcc81a5 --- /dev/null +++ b/pkg/logql/log/vector.go @@ -0,0 +1,48 @@ +package log + +type VectorInt []int64 + +type VectorString struct { + Offsets VectorInt + Lines []byte +} + +// TODO: we might want an interface to support different types of batches. https://github.com/jeschkies/loki/blob/065a34a1afb765e45d15430c143ac522d0308646/pkg/logql/vectorized.go#L54 +type Batch struct { + Timestamps VectorInt + Entries VectorString + Selection []int + // TODO: Add selection +} + +// Returns the timestamp and line for index i or false +func (b *Batch) Get(i int) (int64, []byte, bool) { + if i < 0 || i >= len(b.Timestamps) { + return 0, nil, false + } + + prevOffset := 0 + if i > 0 { + prevOffset = int(b.Entries.Offsets[i-1]) + } + return b.Timestamps[i], b.Entries.Lines[prevOffset:b.Entries.Offsets[i]], true +} + +func (b *Batch) Iter(yield func(int64, []byte) bool) { + prevOffset := 0 + for i, ts := range b.Timestamps { + if i > 0 { + prevOffset = int(b.Entries.Offsets[i-1]) + } + line := b.Entries.Lines[prevOffset:b.Entries.Offsets[i]] + if !yield(ts, line) { + return + } + } +} + +func (b *Batch) Append(ts int64, line []byte) { + b.Timestamps = append(b.Timestamps, ts) + b.Entries.Offsets = append(b.Entries.Offsets, int64(len(b.Entries.Lines))) + b.Entries.Lines = append(b.Entries.Lines, line...) +} diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 5bbe9d2e5ebe8..f02b0902f35ab 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -95,12 +95,13 @@ func RecordRangeAndInstantQueryMetrics( result promql_parser.Value, ) { var ( - logger = fixLogger(ctx, log) - rangeType = GetRangeType(p) - rt = string(rangeType) - latencyType = latencyTypeFast - returnedLines = 0 - queryTags, _ = ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty. + logger = fixLogger(ctx, log) + rangeType = GetRangeType(p) + rt = string(rangeType) + latencyType = latencyTypeFast + returnedLines = 0 + cardinalityEstimate = uint64(0) + queryTags, _ = ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty. ) queryType, err := QueryType(p.GetExpression()) @@ -141,6 +142,10 @@ func RecordRangeAndInstantQueryMetrics( bloomRatio = float64(stats.Index.TotalChunks-stats.Index.PostFilterChunks) / float64(stats.Index.TotalChunks) } + if r, ok := result.(CountMinSketchVector); ok { + cardinalityEstimate = r.F.HyperLogLog.Estimate() + } + logValues = append(logValues, []interface{}{ "latency", latencyType, // this can be used to filter log lines. "query", query, @@ -187,6 +192,8 @@ func RecordRangeAndInstantQueryMetrics( "cache_result_hit", resultCache.EntriesFound, "cache_result_download_time", resultCache.CacheDownloadTime(), "cache_result_query_length_served", resultCache.CacheQueryLengthServed(), + // Cardinality estimate for some approximate query types + "cardinality_estimate", cardinalityEstimate, // The total of chunk reference fetched from index. "ingester_chunk_refs", stats.Ingester.Store.GetTotalChunksRef(), // Total number of chunks fetched. diff --git a/pkg/logql/quantile_over_time_sketch.go b/pkg/logql/quantile_over_time_sketch.go index a14bf303ab943..49eca3453e081 100644 --- a/pkg/logql/quantile_over_time_sketch.go +++ b/pkg/logql/quantile_over_time_sketch.go @@ -64,6 +64,10 @@ func (q ProbabilisticQuantileVector) QuantileSketchVec() ProbabilisticQuantileVe return q } +func (ProbabilisticQuantileVector) CountMinSketchVec() CountMinSketchVector { + return CountMinSketchVector{} +} + func (q ProbabilisticQuantileVector) ToProto() *logproto.QuantileSketchVector { samples := make([]*logproto.QuantileSketchSample, len(q)) for i, sample := range q { @@ -265,8 +269,8 @@ func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sk return s } -// MergeQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. -func MergeQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { +// JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. +func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { vec := r.QuantileSketchVec() if stepEvaluator.Error() != nil { return nil, stepEvaluator.Error() diff --git a/pkg/logql/quantile_over_time_sketch_test.go b/pkg/logql/quantile_over_time_sketch_test.go index 5692575bd2904..7cccfd7479dc7 100644 --- a/pkg/logql/quantile_over_time_sketch_test.go +++ b/pkg/logql/quantile_over_time_sketch_test.go @@ -16,7 +16,7 @@ import ( "github.com/grafana/loki/v3/pkg/logqlmodel" ) -func TestProbabilisticMQuantileMatrixSerialization(t *testing.T) { +func TestProbabilisticQuantileMatrixSerialization(t *testing.T) { emptySketch := sketch.NewDDSketch() ddsketchBytes := make([]byte, 0) emptySketch.Encode(&ddsketchBytes, false) @@ -69,7 +69,7 @@ func TestJoinQuantileSketchVectorError(t *testing.T) { ev := errorStepEvaluator{ err: errors.New("could not evaluate"), } - _, err := MergeQuantileSketchVector(true, result, ev, LiteralParams{}) + _, err := JoinQuantileSketchVector(true, result, ev, LiteralParams{}) require.ErrorContains(t, err, "could not evaluate") } @@ -113,7 +113,6 @@ func (e errorStepEvaluator) Error() error { func (e errorStepEvaluator) Explain(Node) {} func BenchmarkJoinQuantileSketchVector(b *testing.B) { - selRange := (5 * time.Second).Nanoseconds() step := (30 * time.Second) offset := int64(0) @@ -136,7 +135,7 @@ func BenchmarkJoinQuantileSketchVector(b *testing.B) { iter: iter, } _, _, r := ev.Next() - m, err := MergeQuantileSketchVector(true, r.QuantileSketchVec(), ev, params) + m, err := JoinQuantileSketchVector(true, r.QuantileSketchVec(), ev, params) require.NoError(b, err) m.(ProbabilisticQuantileMatrix).Release() } diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 003362913171d..fd07ec0a73ba3 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -16,6 +16,7 @@ const ( ShardLastOverTime = "last_over_time" ShardFirstOverTime = "first_over_time" ShardQuantileOverTime = "quantile_over_time" + SupportApproxTopk = "approx_topk" ) type ShardMapper struct { @@ -24,29 +25,32 @@ type ShardMapper struct { quantileOverTimeSharding bool lastOverTimeSharding bool firstOverTimeSharding bool + approxTopkSupport bool } func NewShardMapper(strategy ShardingStrategy, metrics *MapperMetrics, shardAggregation []string) ShardMapper { - quantileOverTimeSharding := false - lastOverTimeSharding := false - firstOverTimeSharding := false + mapper := ShardMapper{ + shards: strategy, + metrics: metrics, + quantileOverTimeSharding: false, + lastOverTimeSharding: false, + firstOverTimeSharding: false, + approxTopkSupport: false, + } for _, a := range shardAggregation { switch a { case ShardQuantileOverTime: - quantileOverTimeSharding = true + mapper.quantileOverTimeSharding = true case ShardLastOverTime: - lastOverTimeSharding = true + mapper.lastOverTimeSharding = true case ShardFirstOverTime: - firstOverTimeSharding = true + mapper.firstOverTimeSharding = true + case SupportApproxTopk: + mapper.approxTopkSupport = true } } - return ShardMapper{ - shards: strategy, - metrics: metrics, - quantileOverTimeSharding: quantileOverTimeSharding, - firstOverTimeSharding: firstOverTimeSharding, - lastOverTimeSharding: lastOverTimeSharding, - } + + return mapper } func NewShardMapperMetrics(registerer prometheus.Registerer) *MapperMetrics { @@ -185,7 +189,6 @@ func (m ShardMapper) mapLogSelectorExpr(expr syntax.LogSelectorExpr, r *downstre func (m ShardMapper) mapSampleExpr(expr syntax.SampleExpr, r *downstreamRecorder) (syntax.SampleExpr, uint64, error) { var head *ConcatSampleExpr shards, maxBytesPerShard, err := m.shards.Shards(expr) - if err != nil { return nil, 0, err } @@ -233,7 +236,6 @@ func (m ShardMapper) wrappedShardedVectorAggr(expr *syntax.VectorAggregationExpr // in descendent nodes in the AST. This optimization is currently avoided for simplicity. func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr, r *downstreamRecorder, topLevel bool) (syntax.SampleExpr, uint64, error) { if expr.Shardable(topLevel) { - switch expr.Operation { case syntax.OpTypeSum: @@ -286,6 +288,71 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr Grouping: expr.Grouping, Operation: syntax.OpTypeSum, }, bytesPerShard, nil + case syntax.OpTypeApproxTopK: + if !m.approxTopkSupport { + return nil, 0, fmt.Errorf("approx_topk is not enabled. See -limits.shard_aggregations") + } + + // TODO(owen-d): integrate bounded sharding with approx_topk + // I'm not doing this now because it uses a separate code path and may not handle + // bounded shards in the same way + shards, bytesPerShard, err := m.shards.Resolver().Shards(expr) + if err != nil { + return nil, 0, err + } + + // approx_topk(k, inner) -> + // topk( + // k, + // eval_cms( + // __count_min_sketch__(inner, shard=1) ++ __count_min_sketch__(inner, shard=2)... + // ) + // ) + + countMinSketchExpr := syntax.MustClone(expr) + countMinSketchExpr.Operation = syntax.OpTypeCountMinSketch + countMinSketchExpr.Params = 0 + + // Even if this query is not sharded the user wants an approximation. This is helpful if some + // inferred label has a very high cardinality. Note that the querier does not support CountMinSketchEvalExpr + // which is why it's evaluated on the front end. + if shards == 0 { + return &syntax.VectorAggregationExpr{ + Left: &CountMinSketchEvalExpr{ + downstreams: []DownstreamSampleExpr{{ + SampleExpr: countMinSketchExpr, + }}, + }, + Grouping: expr.Grouping, + Operation: syntax.OpTypeTopK, + Params: expr.Params, + }, bytesPerShard, nil + } + + downstreams := make([]DownstreamSampleExpr, 0, shards) + for shard := 0; shard < shards; shard++ { + s := NewPowerOfTwoShard(index.ShardAnnotation{ + Shard: uint32(shard), + Of: uint32(shards), + }) + downstreams = append(downstreams, DownstreamSampleExpr{ + shard: &ShardWithChunkRefs{ + Shard: s, + }, + SampleExpr: countMinSketchExpr, + }) + } + + sharded := &CountMinSketchEvalExpr{ + downstreams: downstreams, + } + + return &syntax.VectorAggregationExpr{ + Left: sharded, + Grouping: expr.Grouping, + Operation: syntax.OpTypeTopK, + Params: expr.Params, + }, bytesPerShard, nil default: // this should not be reachable. If an operation is shardable it should // have an optimization listed. Nonetheless, we log this as a warning @@ -300,7 +367,6 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr } return expr, exprStats.Bytes, nil } - } // if this AST contains unshardable operations, don't shard this at this level, @@ -320,7 +386,6 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr Params: expr.Params, Operation: expr.Operation, }, bytesPerShard, nil - } func (m ShardMapper) mapLabelReplaceExpr(expr *syntax.LabelReplaceExpr, r *downstreamRecorder, topLevel bool) (syntax.SampleExpr, uint64, error) { diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 9bdd128b6e493..b1842c094896c 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -52,7 +52,6 @@ func TestShardedStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - strategy := NewPowerOfTwoStrategy(ConstantShards(2)) m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) @@ -117,7 +116,7 @@ func TestMapSampleExpr(t *testing.T) { func TestMappingStrings(t *testing.T) { strategy := NewPowerOfTwoStrategy(ConstantShards(2)) - m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) + m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime, SupportApproxTopk}) for _, tc := range []struct { in string out string @@ -168,6 +167,15 @@ func TestMappingStrings(t *testing.T) { ++ downstream )`, }, + { + in: `approx_topk(3, sum by(ip)(rate({foo="bar"}[5m])))`, + out: `topk(3, + CountMinSketchEval< + downstream<__count_min_sketch__(sum by(ip)(rate({foo="bar"}[5m]))), shard=0_of_2> + ++ downstream<__count_min_sketch__(sum by(ip)(rate({foo="bar"}[5m]))), shard=1_of_2> + > + )`, + }, { in: `sum(max(rate({foo="bar"}[5m])))`, out: `sum(max( @@ -500,7 +508,6 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) { }, } { t.Run(tc.in, func(t *testing.T) { - shardedMapper := NewShardMapper(NewPowerOfTwoStrategy(ConstantShards(2)), nilShardMetrics, []string{ShardQuantileOverTime}) ast, err := syntax.ParseExpr(tc.in) @@ -526,7 +533,7 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) { func TestMapping(t *testing.T) { strategy := NewPowerOfTwoStrategy(ConstantShards(2)) - m := NewShardMapper(strategy, nilShardMetrics, []string{}) + m := NewShardMapper(strategy, nilShardMetrics, []string{SupportApproxTopk}) for _, tc := range []struct { in string @@ -776,6 +783,56 @@ func TestMapping(t *testing.T) { }, }, }, + { + in: `approx_topk(3, rate({foo="bar"}[5m]))`, + expr: &syntax.VectorAggregationExpr{ + Grouping: &syntax.Grouping{}, + Params: 3, + Operation: syntax.OpTypeTopK, + Left: &CountMinSketchEvalExpr{ + downstreams: []DownstreamSampleExpr{ + { + shard: NewPowerOfTwoShard(index.ShardAnnotation{ + Shard: 0, + Of: 2, + }).Bind(nil), + SampleExpr: &syntax.VectorAggregationExpr{ + Operation: syntax.OpTypeCountMinSketch, + Left: &syntax.RangeAggregationExpr{ + Operation: syntax.OpRangeTypeRate, + Left: &syntax.LogRange{ + Left: &syntax.MatchersExpr{ + Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, + }, + Interval: 5 * time.Minute, + }, + }, + Grouping: &syntax.Grouping{}, + }, + }, + { + shard: NewPowerOfTwoShard(index.ShardAnnotation{ + Shard: 1, + Of: 2, + }).Bind(nil), + SampleExpr: &syntax.VectorAggregationExpr{ + Operation: syntax.OpTypeCountMinSketch, + Left: &syntax.RangeAggregationExpr{ + Operation: syntax.OpRangeTypeRate, + Left: &syntax.LogRange{ + Left: &syntax.MatchersExpr{ + Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, + }, + Interval: 5 * time.Minute, + }, + }, + Grouping: &syntax.Grouping{}, + }, + }, + }, + }, + }, + }, { in: `count(rate({foo="bar"}[5m]))`, expr: &syntax.VectorAggregationExpr{ diff --git a/pkg/logql/sketch/cms.go b/pkg/logql/sketch/cms.go index 0db2a0649b299..9c8e72955ef33 100644 --- a/pkg/logql/sketch/cms.go +++ b/pkg/logql/sketch/cms.go @@ -3,44 +3,58 @@ package sketch import ( "fmt" "math" + + "github.com/axiomhq/hyperloglog" ) type CountMinSketch struct { - depth, width uint32 - counters [][]uint32 + Depth, Width uint32 + Counters [][]float64 + HyperLogLog *hyperloglog.Sketch //hyperloglog.New16(), } // NewCountMinSketch creates a new CMS for a given width and depth. func NewCountMinSketch(w, d uint32) (*CountMinSketch, error) { return &CountMinSketch{ - depth: d, - width: w, - counters: make2dslice(w, d), + Depth: d, + Width: w, + Counters: make2dslice(w, d), + HyperLogLog: hyperloglog.New16(), }, nil } -func make2dslice(col, row uint32) [][]uint32 { - ret := make([][]uint32, row) +// NewCountMinSketchFromErrorAndProbability creates a new CMS for a given epsilon and delta. The sketch width and depth +// are calculated according to the RedisBloom implementation. +// See https://github.com/RedisBloom/RedisBloom/blob/7bc047d1ea4113419b60eb6446ac3d4e61877a7b/src/cms.c#L38-L39 +func NewCountMinSketchFromErrorAndProbability(epsilon float64, delta float64) (*CountMinSketch, error) { + width := math.Ceil(math.E / epsilon) + depth := math.Ceil(math.Log(delta) / math.Log(0.5)) + return NewCountMinSketch(uint32(width), uint32(depth)) +} + +func make2dslice(col, row uint32) [][]float64 { + ret := make([][]float64, row) for i := range ret { - ret[i] = make([]uint32, col) + ret[i] = make([]float64, col) } return ret } func (s *CountMinSketch) getPos(h1, h2, row uint32) uint32 { - pos := (h1 + row*h2) % s.width + pos := (h1 + row*h2) % s.Width return pos } // Add 'count' occurrences of the given input. -func (s *CountMinSketch) Add(event string, count int) { +func (s *CountMinSketch) Add(event string, count float64) { + s.HyperLogLog.Insert(unsafeGetBytes(event)) // see the comments in the hashn function for how using only 2 // hash functions rather than a function per row still fullfils // the pairwise indendent hash functions requirement for CMS h1, h2 := hashn(event) - for i := uint32(0); i < s.depth; i++ { + for i := uint32(0); i < s.Depth; i++ { pos := s.getPos(h1, h2, i) - s.counters[i][pos] += uint32(count) + s.Counters[i][pos] += count } } @@ -55,43 +69,45 @@ func (s *CountMinSketch) Increment(event string) { // value that's less than Count(h) + count rather than all counters that h hashed to. // Returns the new estimate for the event as well as the both hashes which can be used // to identify the event for other things that need a hash. -func (s *CountMinSketch) ConservativeAdd(event string, count uint32) (uint32, uint32, uint32) { - min := uint32(math.MaxUint32) +func (s *CountMinSketch) ConservativeAdd(event string, count float64) (float64, uint32, uint32) { + s.HyperLogLog.Insert(unsafeGetBytes(event)) + + min := float64(math.MaxUint64) h1, h2 := hashn(event) // inline Count to save time/memory var pos uint32 - for i := uint32(0); i < s.depth; i++ { + for i := uint32(0); i < s.Depth; i++ { pos = s.getPos(h1, h2, i) - if s.counters[i][pos] < min { - min = s.counters[i][pos] + if s.Counters[i][pos] < min { + min = s.Counters[i][pos] } } min += count - for i := uint32(0); i < s.depth; i++ { + for i := uint32(0); i < s.Depth; i++ { pos = s.getPos(h1, h2, i) - v := s.counters[i][pos] + v := s.Counters[i][pos] if v < min { - s.counters[i][pos] = min + s.Counters[i][pos] = min } } return min, h1, h2 } -func (s *CountMinSketch) ConservativeIncrement(event string) (uint32, uint32, uint32) { - return s.ConservativeAdd(event, 1) +func (s *CountMinSketch) ConservativeIncrement(event string) (float64, uint32, uint32) { + return s.ConservativeAdd(event, float64(1)) } // Count returns the approximate min count for the given input. -func (s *CountMinSketch) Count(event string) uint32 { - min := uint32(math.MaxUint32) +func (s *CountMinSketch) Count(event string) float64 { + min := float64(math.MaxUint64) h1, h2 := hashn(event) var pos uint32 - for i := uint32(0); i < s.depth; i++ { + for i := uint32(0); i < s.Depth; i++ { pos = s.getPos(h1, h2, i) - if s.counters[i][pos] < min { - min = s.counters[i][pos] + if s.Counters[i][pos] < min { + min = s.Counters[i][pos] } } return min @@ -100,14 +116,23 @@ func (s *CountMinSketch) Count(event string) uint32 { // Merge the given sketch into this one. // The sketches must have the same dimensions. func (s *CountMinSketch) Merge(from *CountMinSketch) error { - if s.depth != from.depth || s.width != from.width { + if s.Depth != from.Depth || s.Width != from.Width { return fmt.Errorf("Can't merge different sketches with different dimensions") } - for i, l := range from.counters { + for i, l := range from.Counters { for j, v := range l { - s.counters[i][j] += v + s.Counters[i][j] += v } } + + // merge the cardinality sketches + s.HyperLogLog.Merge(from.HyperLogLog) + return nil } + +// Cardinality returns the estimated cardinality of the input to the CMS. +func (s *CountMinSketch) Cardinality() uint64 { + return s.HyperLogLog.Estimate() +} diff --git a/pkg/logql/sketch/heap.go b/pkg/logql/sketch/heap.go index fe9c3327c8a16..4abd182c5f31d 100644 --- a/pkg/logql/sketch/heap.go +++ b/pkg/logql/sketch/heap.go @@ -6,7 +6,7 @@ import ( type node struct { event string - count uint32 + count float64 // used for the container heap Fix function index uint16 sketchPositions []uint32 @@ -50,7 +50,7 @@ func (h *MinHeap) Peek() interface{} { } // update modifies the count and value of an Item in the queue. -func (h *MinHeap) update(event string, count uint32) { +func (h *MinHeap) update(event string, count float64) { updateNode := -1 for i, k := range *h { if k.event == event { diff --git a/pkg/logql/sketch/heap_test.go b/pkg/logql/sketch/heap_test.go index 57b4d9598ba83..af5515874a825 100644 --- a/pkg/logql/sketch/heap_test.go +++ b/pkg/logql/sketch/heap_test.go @@ -12,21 +12,21 @@ func TestHeap(t *testing.T) { heap.Init(&h) - heap.Push(&h, &node{event: "1", count: 70}) - assert.Equal(t, uint32(70), h.Peek().(*node).count, "expected: %d and got %d", uint32(70), h.Peek().(*node).count) + heap.Push(&h, &node{event: "1", count: 70.0}) + assert.Equal(t, 70.0, h.Peek().(*node).count, "expected: %f and got %f", 70.0, h.Peek().(*node).count) - heap.Push(&h, &node{event: "2", count: 20}) - assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count) + heap.Push(&h, &node{event: "2", count: 20.0}) + assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20, h.Peek().(*node).count) heap.Push(&h, &node{event: "3", count: 50}) - assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count) + assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20.0, h.Peek().(*node).count) - heap.Push(&h, &node{event: "4", count: 60}) - assert.Equal(t, uint32(20), h.Peek().(*node).count, "expected: %d and got %d", uint32(20), h.Peek().(*node).count) + heap.Push(&h, &node{event: "4", count: 60.0}) + assert.Equal(t, 20.0, h.Peek().(*node).count, "expected: %f and got %f", 20.0, h.Peek().(*node).count) - heap.Push(&h, &node{event: "5", count: 10}) - assert.Equal(t, uint32(10), h.Peek().(*node).count, "expected: %d and got %d", uint32(10), h.Peek().(*node).count) + heap.Push(&h, &node{event: "5", count: 10.0}) + assert.Equal(t, 10.0, h.Peek().(*node).count, "expected: %f and got %f", 10.0, h.Peek().(*node).count) - assert.Equal(t, heap.Pop(&h).(*node).count, uint32(10)) - assert.Equal(t, h.Peek().(*node).count, uint32(20)) + assert.Equal(t, heap.Pop(&h).(*node).count, 10.0) + assert.Equal(t, h.Peek().(*node).count, 20.0) } diff --git a/pkg/logql/sketch/series_test.go b/pkg/logql/sketch/series_test.go index 4878dd9724543..1c827db4f688a 100644 --- a/pkg/logql/sketch/series_test.go +++ b/pkg/logql/sketch/series_test.go @@ -36,7 +36,7 @@ func TestTopKMatrixProto(t *testing.T) { require.NoError(t, err) require.Len(t, deserialized, 1) - require.Equal(t, original.sketch.counters, deserialized[0].topk.sketch.counters) + require.Equal(t, original.sketch.Counters, deserialized[0].topk.sketch.Counters) require.Equal(t, *original.hll, *deserialized[0].topk.hll) oCardinality, _ := original.Cardinality() fmt.Println("ocardinality: ", oCardinality) @@ -74,7 +74,7 @@ func TestTopKMatrixProtoMerge(t *testing.T) { require.NoError(t, err) require.Len(t, deserialized, 1) - require.Equal(t, original.sketch.counters, deserialized[0].topk.sketch.counters) + require.Equal(t, original.sketch.Counters, deserialized[0].topk.sketch.Counters) require.Equal(t, *original.hll, *deserialized[0].topk.hll) oCardinality, _ := original.Cardinality() dCardinality, _ := deserialized[0].topk.Cardinality() diff --git a/pkg/logql/sketch/topk.go b/pkg/logql/sketch/topk.go index e6f2c036f3675..23280154daed8 100644 --- a/pkg/logql/sketch/topk.go +++ b/pkg/logql/sketch/topk.go @@ -14,7 +14,7 @@ import ( type element struct { Event string - Count int64 + Count float64 } type TopKResult []element @@ -108,13 +108,13 @@ func newCMSTopK(k int, w, d uint32) (*Topk, error) { func TopkFromProto(t *logproto.TopK) (*Topk, error) { cms := &CountMinSketch{ - depth: t.Cms.Depth, - width: t.Cms.Width, + Depth: t.Cms.Depth, + Width: t.Cms.Width, } - for row := uint32(0); row < cms.depth; row++ { - s := row * cms.width - e := s + cms.width - cms.counters = append(cms.counters, t.Cms.Counters[s:e]) + for row := uint32(0); row < cms.Depth; row++ { + s := row * cms.Width + e := s + cms.Width + cms.Counters = append(cms.Counters, t.Cms.Counters[s:e]) } hll := hyperloglog.New() @@ -143,12 +143,12 @@ func TopkFromProto(t *logproto.TopK) (*Topk, error) { func (t *Topk) ToProto() (*logproto.TopK, error) { cms := &logproto.CountMinSketch{ - Depth: t.sketch.depth, - Width: t.sketch.width, + Depth: t.sketch.Depth, + Width: t.sketch.Width, } - cms.Counters = make([]uint32, 0, cms.Depth*cms.Width) + cms.Counters = make([]float64, 0, cms.Depth*cms.Width) for row := uint32(0); row < cms.Depth; row++ { - cms.Counters = append(cms.Counters, t.sketch.counters[row]...) + cms.Counters = append(cms.Counters, t.sketch.Counters[row]...) } hllBytes, err := t.hll.MarshalBinary() @@ -175,7 +175,7 @@ func (t *Topk) ToProto() (*logproto.TopK, error) { // wrapper to bundle together updating of the bf portion of the sketch and pushing of a new element // to the heap -func (t *Topk) heapPush(h *MinHeap, event string, estimate, h1, h2 uint32) { +func (t *Topk) heapPush(h *MinHeap, event string, estimate float64, h1, h2 uint32) { var pos uint32 for i := range t.bf { pos = t.sketch.getPos(h1, h2, uint32(i)) @@ -186,7 +186,7 @@ func (t *Topk) heapPush(h *MinHeap, event string, estimate, h1, h2 uint32) { // wrapper to bundle together updating of the bf portion of the sketch for the removed and added event // as well as replacing the min heap element with the new event and it's count -func (t *Topk) heapMinReplace(event string, estimate uint32, removed string) { +func (t *Topk) heapMinReplace(event string, estimate float64, removed string) { t.updateBF(removed, event) (*t.heap)[0].event = event (*t.heap)[0].count = estimate @@ -269,7 +269,7 @@ func (t *Topk) Observe(event string) { if estimate > t.heap.Peek().(*node).count { if len(*t.heap) == t.max { e := t.heap.Peek().(*node).event - //r1, r2 := hashn(e) + // r1, r2 := hashn(e) t.heapMinReplace(event, estimate, e) return } @@ -304,11 +304,11 @@ func (t *Topk) Merge(from *Topk) error { var all TopKResult for _, e := range *t.heap { - all = append(all, element{Event: e.event, Count: int64(t.sketch.Count(e.event))}) + all = append(all, element{Event: e.event, Count: t.sketch.Count(e.event)}) } for _, e := range *from.heap { - all = append(all, element{Event: e.event, Count: int64(t.sketch.Count(e.event))}) + all = append(all, element{Event: e.event, Count: t.sketch.Count(e.event)}) } all = removeDuplicates(all) @@ -318,7 +318,7 @@ func (t *Topk) Merge(from *Topk) error { // TODO: merging should also potentially replace it's bloomfilter? or 0 everything in the bloomfilter for _, e := range all[:t.max] { h1, h2 = hashn(e.Event) - t.heapPush(temp, e.Event, uint32(e.Count), h1, h2) + t.heapPush(temp, e.Event, float64(e.Count), h1, h2) } t.heap = temp @@ -347,7 +347,7 @@ func (t *Topk) Topk() TopKResult { for _, e := range *t.heap { res = append(res, element{ Event: e.event, - Count: int64(t.sketch.Count(e.event)), + Count: t.sketch.Count(e.event), }) } sort.Sort(res) diff --git a/pkg/logql/sketch/topk_test.go b/pkg/logql/sketch/topk_test.go index d923cc60fc32f..d375d159e8c60 100644 --- a/pkg/logql/sketch/topk_test.go +++ b/pkg/logql/sketch/topk_test.go @@ -131,7 +131,7 @@ outer2: require.LessOrEqualf(t, singleMissing, 2, "more than acceptable misses: %d > %d", singleMissing, 2) // this condition is never actually true - //require.LessOrEqualf(t, mergedMissing, singleMissing, "merged sketch should be at least as accurate as a single sketch") + // require.LessOrEqualf(t, mergedMissing, singleMissing, "merged sketch should be at least as accurate as a single sketch") } // compare the accuracy of cms topk and hk to the real topk @@ -142,7 +142,7 @@ func TestRealTopK(t *testing.T) { defer f.Close() scanner := bufio.NewScanner(f) - m := make(map[string]uint32) + m := make(map[string]float64) h := MinHeap{} hll := hyperloglog.New16() @@ -170,7 +170,7 @@ func TestRealTopK(t *testing.T) { res := make(TopKResult, 0, len(h)) for i := 0; i < len(h); i++ { - res = append(res, element{h[i].event, int64(h[i].count)}) + res = append(res, element{h[i].event, h[i].count}) } sort.Sort(res) @@ -219,7 +219,7 @@ func TestRealTop_Merge(t *testing.T) { scanner := bufio.NewScanner(combined) - m := make(map[string]uint32) + m := make(map[string]float64) h := MinHeap{} hll := hyperloglog.New16() // HK gets more inaccurate with merging the more shards we have @@ -251,7 +251,7 @@ func TestRealTop_Merge(t *testing.T) { res := make(TopKResult, 0, len(h)) for i := 0; i < len(h); i++ { - res = append(res, element{h[i].event, int64(h[i].count)}) + res = append(res, element{h[i].event, h[i].count}) } sort.Sort(res) @@ -265,7 +265,7 @@ func TestRealTop_Merge(t *testing.T) { scanner = bufio.NewScanner(combined) scanner.Split(bufio.ScanWords) - var cms = make([]*Topk, shards) + cms := make([]*Topk, shards) for i := range cms { cms[i], _ = newCMSTopK(k, 2048, 5) } diff --git a/pkg/logql/step_evaluator.go b/pkg/logql/step_evaluator.go index 23b313f18da27..1420bf270f6bd 100644 --- a/pkg/logql/step_evaluator.go +++ b/pkg/logql/step_evaluator.go @@ -7,6 +7,7 @@ import ( type StepResult interface { SampleVector() promql.Vector QuantileSketchVec() ProbabilisticQuantileVector + CountMinSketchVec() CountMinSketchVector } type SampleVector promql.Vector @@ -21,6 +22,10 @@ func (p SampleVector) QuantileSketchVec() ProbabilisticQuantileVector { return ProbabilisticQuantileVector{} } +func (SampleVector) CountMinSketchVec() CountMinSketchVector { + return CountMinSketchVector{} +} + // StepEvaluator evaluate a single step of a query. type StepEvaluator interface { // while Next returns a promql.Value, the only acceptable types are Scalar and Vector. diff --git a/pkg/logql/syntax/ast.go b/pkg/logql/syntax/ast.go index 51bee23b01a9a..29cb5e548ddd6 100644 --- a/pkg/logql/syntax/ast.go +++ b/pkg/logql/syntax/ast.go @@ -47,7 +47,7 @@ func Clone[T Expr](e T) (T, error) { } func MustClone[T Expr](e T) T { - copied, err := Clone[T](e) + copied, err := Clone(e) if err != nil { panic(err) } @@ -800,6 +800,7 @@ func (e *DecolorizeExpr) Shardable(_ bool) bool { return true } func (e *DecolorizeExpr) Stage() (log.Stage, error) { return log.NewDecolorizer() } + func (e *DecolorizeExpr) String() string { return fmt.Sprintf("%s %s", OpPipe, OpDecolorize) } @@ -823,6 +824,7 @@ func (e *DropLabelsExpr) Shardable(_ bool) bool { return true } func (e *DropLabelsExpr) Stage() (log.Stage, error) { return log.NewDropLabels(e.dropLabels), nil } + func (e *DropLabelsExpr) String() string { var sb strings.Builder @@ -1234,7 +1236,7 @@ const ( OpRangeTypeLast = "last_over_time" OpRangeTypeAbsent = "absent_over_time" - //vector + // vector OpTypeVector = "vector" // binops - logical/set @@ -1305,6 +1307,11 @@ const ( OpRangeTypeQuantileSketch = "__quantile_sketch_over_time__" OpRangeTypeFirstWithTimestamp = "__first_over_time_ts__" OpRangeTypeLastWithTimestamp = "__last_over_time_ts__" + + OpTypeCountMinSketch = "__count_min_sketch__" + + // probabilistic aggregations + OpTypeApproxTopK = "approx_topk" ) func IsComparisonOperator(op string) bool { @@ -1533,7 +1540,7 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin var p int var err error switch operation { - case OpTypeBottomK, OpTypeTopK: + case OpTypeBottomK, OpTypeTopK, OpTypeApproxTopK: if params == nil { return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0)} } @@ -1544,6 +1551,9 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin if p <= 0 { return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("invalid parameter (must be greater than 0) %s(%s", operation, *params), 0, 0)} } + if operation == OpTypeApproxTopK && gr != nil { + return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("grouping not allowed for %s aggregation", operation), 0, 0)} + } default: if params != nil { @@ -1609,7 +1619,7 @@ func (e *VectorAggregationExpr) String() string { var params []string switch e.Operation { // bottomK and topk can have first parameter as 0 - case OpTypeBottomK, OpTypeTopK: + case OpTypeBottomK, OpTypeTopK, OpTypeApproxTopK: params = []string{fmt.Sprintf("%d", e.Params), e.Left.String()} default: if e.Params != 0 { @@ -2278,6 +2288,8 @@ var shardableOps = map[string]bool{ OpTypeMax: true, OpTypeMin: true, + OpTypeApproxTopK: true, + // range vector ops OpRangeTypeAvg: true, OpRangeTypeCount: true, diff --git a/pkg/logql/syntax/expr.y b/pkg/logql/syntax/expr.y index fb6aa7f344a6b..b32366d65e37e 100644 --- a/pkg/logql/syntax/expr.y +++ b/pkg/logql/syntax/expr.y @@ -135,7 +135,8 @@ import ( %token IDENTIFIER STRING NUMBER PARSER_FLAG %token DURATION RANGE %token MATCHERS LABELS EQ RE NRE NPA OPEN_BRACE CLOSE_BRACE OPEN_BRACKET CLOSE_BRACKET COMMA DOT PIPE_MATCH PIPE_EXACT PIPE_PATTERN - OPEN_PARENTHESIS CLOSE_PARENTHESIS BY WITHOUT COUNT_OVER_TIME RATE RATE_COUNTER SUM SORT SORT_DESC AVG MAX MIN COUNT STDDEV STDVAR BOTTOMK TOPK + OPEN_PARENTHESIS CLOSE_PARENTHESIS BY WITHOUT COUNT_OVER_TIME RATE RATE_COUNTER SUM SORT SORT_DESC AVG + MAX MIN COUNT STDDEV STDVAR BOTTOMK TOPK APPROX_TOPK BYTES_OVER_TIME BYTES_RATE BOOL JSON REGEXP LOGFMT PIPE LINE_FMT LABEL_FMT UNWRAP AVG_OVER_TIME SUM_OVER_TIME MIN_OVER_TIME MAX_OVER_TIME STDVAR_OVER_TIME STDDEV_OVER_TIME QUANTILE_OVER_TIME BYTES_CONV DURATION_CONV DURATION_SECONDS_CONV FIRST_OVER_TIME LAST_OVER_TIME ABSENT_OVER_TIME VECTOR LABEL_REPLACE UNPACK OFFSET PATTERN IP ON IGNORING GROUP_LEFT GROUP_RIGHT @@ -545,6 +546,7 @@ vectorOp: | TOPK { $$ = OpTypeTopK } | SORT { $$ = OpTypeSort } | SORT_DESC { $$ = OpTypeSortDesc } + | APPROX_TOPK { $$ = OpTypeApproxTopK } ; rangeOp: diff --git a/pkg/logql/syntax/expr.y.go b/pkg/logql/syntax/expr.y.go index 4918cf5cdc81a..85070b1c2b501 100644 --- a/pkg/logql/syntax/expr.y.go +++ b/pkg/logql/syntax/expr.y.go @@ -1,11 +1,9 @@ -// Code generated by goyacc -p expr -o expr.y.go expr.y. DO NOT EDIT. +// Code generated by goyacc -p expr -o pkg/logql/syntax/expr.y.go pkg/logql/syntax/expr.y. DO NOT EDIT. -//line expr.y:2 package syntax import __yyfmt__ "fmt" -//line expr.y:2 import ( "github.com/grafana/loki/v3/pkg/logql/log" @@ -13,7 +11,6 @@ import ( "time" ) -//line expr.y:12 type exprSymType struct { yys int Expr Expr @@ -119,57 +116,58 @@ const STDDEV = 57382 const STDVAR = 57383 const BOTTOMK = 57384 const TOPK = 57385 -const BYTES_OVER_TIME = 57386 -const BYTES_RATE = 57387 -const BOOL = 57388 -const JSON = 57389 -const REGEXP = 57390 -const LOGFMT = 57391 -const PIPE = 57392 -const LINE_FMT = 57393 -const LABEL_FMT = 57394 -const UNWRAP = 57395 -const AVG_OVER_TIME = 57396 -const SUM_OVER_TIME = 57397 -const MIN_OVER_TIME = 57398 -const MAX_OVER_TIME = 57399 -const STDVAR_OVER_TIME = 57400 -const STDDEV_OVER_TIME = 57401 -const QUANTILE_OVER_TIME = 57402 -const BYTES_CONV = 57403 -const DURATION_CONV = 57404 -const DURATION_SECONDS_CONV = 57405 -const FIRST_OVER_TIME = 57406 -const LAST_OVER_TIME = 57407 -const ABSENT_OVER_TIME = 57408 -const VECTOR = 57409 -const LABEL_REPLACE = 57410 -const UNPACK = 57411 -const OFFSET = 57412 -const PATTERN = 57413 -const IP = 57414 -const ON = 57415 -const IGNORING = 57416 -const GROUP_LEFT = 57417 -const GROUP_RIGHT = 57418 -const DECOLORIZE = 57419 -const DROP = 57420 -const KEEP = 57421 -const OR = 57422 -const AND = 57423 -const UNLESS = 57424 -const CMP_EQ = 57425 -const NEQ = 57426 -const LT = 57427 -const LTE = 57428 -const GT = 57429 -const GTE = 57430 -const ADD = 57431 -const SUB = 57432 -const MUL = 57433 -const DIV = 57434 -const MOD = 57435 -const POW = 57436 +const APPROX_TOPK = 57386 +const BYTES_OVER_TIME = 57387 +const BYTES_RATE = 57388 +const BOOL = 57389 +const JSON = 57390 +const REGEXP = 57391 +const LOGFMT = 57392 +const PIPE = 57393 +const LINE_FMT = 57394 +const LABEL_FMT = 57395 +const UNWRAP = 57396 +const AVG_OVER_TIME = 57397 +const SUM_OVER_TIME = 57398 +const MIN_OVER_TIME = 57399 +const MAX_OVER_TIME = 57400 +const STDVAR_OVER_TIME = 57401 +const STDDEV_OVER_TIME = 57402 +const QUANTILE_OVER_TIME = 57403 +const BYTES_CONV = 57404 +const DURATION_CONV = 57405 +const DURATION_SECONDS_CONV = 57406 +const FIRST_OVER_TIME = 57407 +const LAST_OVER_TIME = 57408 +const ABSENT_OVER_TIME = 57409 +const VECTOR = 57410 +const LABEL_REPLACE = 57411 +const UNPACK = 57412 +const OFFSET = 57413 +const PATTERN = 57414 +const IP = 57415 +const ON = 57416 +const IGNORING = 57417 +const GROUP_LEFT = 57418 +const GROUP_RIGHT = 57419 +const DECOLORIZE = 57420 +const DROP = 57421 +const KEEP = 57422 +const OR = 57423 +const AND = 57424 +const UNLESS = 57425 +const CMP_EQ = 57426 +const NEQ = 57427 +const LT = 57428 +const LTE = 57429 +const GT = 57430 +const GTE = 57431 +const ADD = 57432 +const SUB = 57433 +const MUL = 57434 +const DIV = 57435 +const MOD = 57436 +const POW = 57437 var exprToknames = [...]string{ "$end", @@ -215,6 +213,7 @@ var exprToknames = [...]string{ "STDVAR", "BOTTOMK", "TOPK", + "APPROX_TOPK", "BYTES_OVER_TIME", "BYTES_RATE", "BOOL", @@ -267,17 +266,14 @@ var exprToknames = [...]string{ "MOD", "POW", } - var exprStatenames = [...]string{} const exprEofCode = 1 const exprErrCode = 2 const exprInitialStackSize = 16 -//line expr.y:582 -//line yacctab:1 -var exprExca = [...]int8{ +var exprExca = [...]int{ -1, 1, 1, -1, -2, 0, @@ -285,127 +281,129 @@ var exprExca = [...]int8{ const exprPrivate = 57344 -const exprLast = 639 - -var exprAct = [...]int16{ - 289, 228, 84, 4, 214, 64, 182, 126, 204, 189, - 75, 200, 197, 63, 237, 5, 152, 187, 77, 2, - 56, 80, 48, 49, 50, 57, 58, 61, 62, 59, - 60, 51, 52, 53, 54, 55, 56, 283, 10, 49, - 50, 57, 58, 61, 62, 59, 60, 51, 52, 53, - 54, 55, 56, 57, 58, 61, 62, 59, 60, 51, - 52, 53, 54, 55, 56, 53, 54, 55, 56, 109, - 217, 139, 215, 115, 51, 52, 53, 54, 55, 56, - 266, 140, 221, 16, 216, 265, 292, 156, 136, 166, - 167, 297, 262, 161, 220, 16, 294, 261, 154, 148, - 150, 151, 366, 281, 184, 339, 16, 67, 280, 130, - 163, 207, 150, 151, 168, 169, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 278, 164, - 165, 16, 366, 277, 239, 94, 293, 194, 142, 386, - 191, 275, 202, 206, 16, 294, 274, 142, 264, 340, - 85, 86, 292, 141, 272, 219, 316, 16, 381, 271, - 260, 306, 235, 185, 183, 17, 18, 356, 229, 224, - 149, 231, 232, 110, 306, 240, 294, 17, 18, 136, - 355, 213, 208, 211, 212, 209, 210, 374, 17, 18, - 248, 249, 250, 269, 331, 184, 16, 306, 268, 373, - 130, 255, 306, 354, 252, 342, 343, 344, 353, 136, - 239, 72, 74, 17, 18, 371, 363, 339, 306, 69, - 70, 71, 285, 384, 308, 184, 17, 18, 287, 290, - 130, 296, 314, 299, 369, 109, 302, 115, 303, 17, - 18, 291, 154, 288, 359, 300, 263, 267, 270, 273, - 276, 279, 282, 349, 185, 183, 330, 294, 293, 310, - 312, 315, 317, 318, 72, 74, 202, 206, 325, 320, - 324, 295, 69, 70, 71, 346, 72, 74, 17, 18, - 73, 306, 239, 239, 69, 70, 71, 307, 328, 136, - 239, 332, 304, 334, 336, 224, 338, 109, 294, 230, - 243, 337, 348, 333, 313, 311, 109, 239, 295, 350, - 130, 230, 241, 72, 74, 83, 224, 85, 86, 292, - 301, 69, 70, 71, 233, 347, 144, 13, 136, 238, - 143, 327, 153, 73, 360, 361, 155, 326, 284, 109, - 362, 225, 13, 247, 184, 73, 364, 365, 230, 130, - 246, 155, 370, 245, 244, 218, 160, 159, 158, 227, - 16, 90, 89, 82, 72, 74, 376, 380, 377, 378, - 13, 352, 69, 70, 71, 253, 298, 305, 259, 6, - 382, 258, 73, 21, 22, 23, 36, 45, 46, 37, - 39, 40, 38, 41, 42, 43, 44, 24, 25, 230, - 256, 242, 234, 226, 183, 257, 254, 26, 27, 28, - 29, 30, 31, 32, 81, 379, 368, 33, 34, 35, - 47, 19, 236, 227, 367, 162, 345, 79, 72, 74, - 335, 190, 13, 73, 251, 88, 69, 70, 71, 322, - 323, 6, 17, 18, 87, 21, 22, 23, 36, 45, - 46, 37, 39, 40, 38, 41, 42, 43, 44, 24, - 25, 190, 375, 230, 188, 385, 383, 372, 358, 26, - 27, 28, 29, 30, 31, 32, 146, 357, 3, 33, - 34, 35, 47, 19, 157, 76, 329, 319, 309, 286, - 72, 74, 145, 321, 13, 147, 198, 73, 69, 70, - 71, 223, 222, 6, 17, 18, 136, 21, 22, 23, +const exprLast = 644 + +var exprAct = [...]int{ + + 290, 229, 85, 4, 215, 65, 183, 127, 205, 190, + 76, 201, 198, 64, 238, 5, 153, 188, 78, 2, + 57, 81, 49, 50, 51, 58, 59, 62, 63, 60, + 61, 52, 53, 54, 55, 56, 57, 284, 10, 50, + 51, 58, 59, 62, 63, 60, 61, 52, 53, 54, + 55, 56, 57, 58, 59, 62, 63, 60, 61, 52, + 53, 54, 55, 56, 57, 54, 55, 56, 57, 218, + 110, 137, 140, 216, 116, 52, 53, 54, 55, 56, + 57, 267, 293, 222, 16, 141, 266, 185, 157, 149, + 151, 152, 131, 263, 162, 221, 16, 68, 262, 155, + 73, 75, 167, 168, 165, 166, 298, 217, 70, 71, + 72, 164, 208, 151, 152, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 282, + 340, 367, 16, 294, 281, 295, 231, 137, 195, 367, + 143, 192, 95, 203, 207, 86, 87, 186, 184, 340, + 265, 293, 143, 185, 279, 387, 220, 16, 131, 278, + 382, 150, 261, 236, 111, 294, 370, 17, 18, 230, + 74, 295, 232, 233, 295, 307, 241, 142, 375, 17, + 18, 357, 347, 214, 209, 212, 213, 210, 211, 307, + 295, 249, 250, 251, 276, 356, 307, 16, 273, 275, + 341, 16, 355, 272, 270, 253, 295, 16, 84, 269, + 86, 87, 374, 296, 184, 17, 18, 364, 73, 75, + 225, 380, 328, 286, 372, 307, 70, 71, 72, 288, + 291, 354, 297, 240, 300, 360, 110, 303, 116, 304, + 17, 18, 292, 155, 289, 332, 301, 264, 268, 271, + 274, 277, 280, 283, 231, 317, 137, 343, 344, 345, + 311, 313, 316, 318, 319, 350, 137, 203, 207, 326, + 321, 325, 185, 331, 240, 305, 244, 131, 256, 240, + 17, 18, 185, 240, 17, 18, 225, 131, 74, 329, + 17, 18, 333, 307, 335, 337, 315, 339, 110, 309, + 307, 314, 338, 349, 334, 312, 308, 110, 296, 240, + 351, 302, 240, 73, 75, 234, 145, 225, 154, 144, + 13, 70, 71, 72, 137, 348, 327, 285, 13, 156, + 385, 242, 186, 184, 239, 361, 362, 156, 248, 247, + 110, 363, 226, 246, 245, 131, 219, 365, 366, 231, + 161, 160, 159, 371, 91, 90, 83, 381, 353, 254, + 228, 16, 306, 147, 260, 73, 75, 377, 259, 378, + 379, 13, 257, 70, 71, 72, 243, 299, 235, 146, + 6, 383, 148, 74, 21, 22, 23, 36, 45, 46, + 37, 39, 40, 38, 41, 42, 43, 44, 47, 24, + 25, 231, 227, 82, 258, 255, 369, 368, 336, 26, + 27, 28, 29, 30, 31, 32, 80, 346, 163, 33, + 34, 35, 48, 19, 237, 228, 191, 323, 324, 252, + 73, 75, 89, 191, 13, 74, 189, 88, 70, 71, + 72, 386, 384, 6, 17, 18, 373, 21, 22, 23, 36, 45, 46, 37, 39, 40, 38, 41, 42, 43, - 44, 24, 25, 221, 220, 230, 195, 130, 193, 192, - 351, 26, 27, 28, 29, 30, 31, 32, 205, 136, - 201, 33, 34, 35, 47, 19, 190, 81, 122, 123, - 121, 198, 131, 133, 297, 72, 74, 127, 128, 73, - 130, 113, 114, 69, 70, 71, 17, 18, 196, 118, - 124, 203, 125, 120, 199, 91, 119, 117, 132, 134, - 135, 122, 123, 121, 116, 131, 133, 186, 65, 137, - 66, 129, 138, 111, 112, 93, 92, 11, 9, 20, - 12, 15, 8, 124, 341, 125, 14, 7, 78, 68, - 1, 132, 134, 135, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 73, 95, 96, 97, 98, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, + 44, 47, 24, 25, 3, 359, 231, 358, 330, 320, + 310, 77, 26, 27, 28, 29, 30, 31, 32, 287, + 73, 75, 33, 34, 35, 48, 19, 158, 70, 71, + 72, 224, 322, 73, 75, 199, 376, 13, 223, 222, + 74, 70, 71, 72, 221, 196, 6, 17, 18, 137, + 21, 22, 23, 36, 45, 46, 37, 39, 40, 38, + 41, 42, 43, 44, 47, 24, 25, 194, 193, 67, + 131, 352, 206, 202, 191, 26, 27, 28, 29, 30, + 31, 32, 82, 137, 199, 33, 34, 35, 48, 19, + 74, 128, 123, 124, 122, 129, 132, 134, 298, 73, + 75, 114, 115, 74, 131, 197, 119, 70, 71, 72, + 17, 18, 204, 121, 125, 200, 126, 120, 118, 92, + 117, 187, 133, 135, 136, 66, 123, 124, 122, 138, + 132, 134, 130, 139, 112, 231, 113, 94, 93, 11, + 9, 20, 12, 15, 8, 342, 14, 7, 125, 79, + 126, 69, 1, 0, 0, 293, 133, 135, 136, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 74, + 96, 97, 98, 99, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, } +var exprPact = [...]int{ -var exprPact = [...]int16{ - 353, -1000, -58, -1000, -1000, 540, 353, -1000, -1000, -1000, - -1000, -1000, -1000, 409, 337, 289, -1000, 437, 428, 336, - 335, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 354, -1000, -59, -1000, -1000, 478, 354, -1000, -1000, -1000, + -1000, -1000, -1000, 398, 330, 182, -1000, 430, 425, 329, + 328, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 89, 89, - 89, 89, 89, 89, 89, 89, 89, 89, 89, 89, - 89, 89, 89, 540, -1000, 196, 534, -9, 75, -1000, - -1000, -1000, -1000, -1000, -1000, 303, 299, -58, 474, -1000, - -1000, 86, 325, 477, 332, 331, 330, -1000, -1000, 353, - 418, 353, 56, 14, -1000, 353, 353, 353, 353, 353, - 353, 353, 353, 353, 353, 353, 353, 353, 353, -1000, - -1000, -1000, -1000, -1000, -1000, 83, -1000, -1000, -1000, -1000, - -1000, 456, 541, 523, -1000, 522, -1000, -1000, -1000, -1000, - 284, 520, -1000, 546, 535, 533, 98, -1000, -1000, 66, - -10, 329, -1000, -1000, -1000, -1000, -1000, 542, 518, 517, - 496, 495, 314, 382, 413, 310, 297, 381, 415, 302, - 285, 380, 273, -42, 328, 327, 324, 317, -30, -30, - -26, -26, -74, -74, -74, -74, -15, -15, -15, -15, - -15, -15, 83, 284, 284, 284, 426, 354, -1000, -1000, - 393, 354, -1000, -1000, 174, -1000, 379, -1000, 392, 360, - -1000, 86, -1000, 357, -1000, 86, -1000, 88, 76, 189, - 150, 137, 124, 99, -1000, -43, 312, 66, 483, -1000, - -1000, -1000, -1000, -1000, -1000, 122, 310, 249, 126, 261, - 501, 349, 293, 122, 353, 265, 356, 260, -1000, -1000, - 197, -1000, 482, -1000, 278, 277, 205, 129, 204, 83, - 323, -1000, 354, 541, 481, -1000, 491, 434, 535, 533, - 311, -1000, -1000, -1000, 305, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 95, + 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, + 95, 95, 95, 95, 478, -1000, 465, 538, -9, 79, + -1000, -1000, -1000, -1000, -1000, -1000, 292, 289, -59, 361, + -1000, -1000, 76, 311, 480, 326, 325, 324, -1000, -1000, + 354, 411, 354, 30, 26, -1000, 354, 354, 354, 354, + 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, + -1000, -1000, -1000, -1000, -1000, -1000, 66, -1000, -1000, -1000, + -1000, -1000, 428, 529, 522, -1000, 521, -1000, -1000, -1000, + -1000, 319, 499, -1000, 539, 528, 527, 99, -1000, -1000, + 67, -12, 320, -1000, -1000, -1000, -1000, -1000, 537, 498, + 493, 492, 485, 315, 381, 415, 303, 288, 357, 417, + 307, 304, 355, 249, -43, 318, 317, 313, 312, -31, + -31, -27, -27, -75, -75, -75, -75, -15, -15, -15, + -15, -15, -15, 66, 319, 319, 319, 421, 338, -1000, + -1000, 392, 338, -1000, -1000, 251, -1000, 351, -1000, 391, + 347, -1000, 76, -1000, 343, -1000, 76, -1000, 89, 77, + 200, 194, 190, 150, 125, -1000, -44, 301, 67, 473, + -1000, -1000, -1000, -1000, -1000, -1000, 117, 303, 544, 123, + 203, 504, 350, 284, 117, 354, 248, 341, 279, -1000, + -1000, 272, -1000, 464, -1000, 278, 274, 269, 228, 261, + 66, 132, -1000, 338, 529, 463, -1000, 490, 422, 528, + 527, 300, -1000, -1000, -1000, 196, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 66, 480, -1000, 229, -1000, 167, 475, - 46, 475, 421, 16, 284, 16, 95, 144, 416, 248, - 298, -1000, -1000, 226, -1000, 353, 525, -1000, -1000, 350, - 181, -1000, 176, -1000, -1000, 153, -1000, 140, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 471, 462, -1000, 217, - -1000, 122, 46, 475, 46, -1000, -1000, 83, -1000, 16, - -1000, 190, -1000, -1000, -1000, 82, 414, 406, 207, 122, - 188, -1000, 461, -1000, -1000, -1000, -1000, 172, 160, -1000, - -1000, 46, -1000, 457, 52, 46, 38, 16, 16, 405, - -1000, -1000, 346, -1000, -1000, 131, 46, -1000, -1000, 16, - 460, -1000, -1000, 202, 459, 112, -1000, + -1000, -1000, -1000, -1000, 67, 462, -1000, 246, -1000, 218, + 85, 84, 85, 399, 11, 319, 11, 120, 195, 407, + 155, 298, -1000, -1000, 238, -1000, 354, 526, -1000, -1000, + 337, 204, -1000, 175, -1000, -1000, 168, -1000, 154, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 461, 459, -1000, + 208, -1000, 117, 84, 85, 84, -1000, -1000, 66, -1000, + 11, -1000, 191, -1000, -1000, -1000, 80, 397, 396, 139, + 117, 197, -1000, 440, -1000, -1000, -1000, -1000, 185, 151, + -1000, -1000, 84, -1000, 491, 88, 84, 52, 11, 11, + 211, -1000, -1000, 336, -1000, -1000, 133, 84, -1000, -1000, + 11, 436, -1000, -1000, 309, 435, 128, -1000, } - -var exprPgo = [...]int16{ - 0, 610, 18, 609, 2, 14, 478, 3, 16, 7, - 608, 607, 606, 604, 15, 602, 601, 600, 599, 84, - 598, 38, 597, 575, 596, 595, 594, 593, 13, 5, - 592, 591, 589, 6, 588, 107, 4, 587, 584, 577, - 576, 574, 11, 573, 571, 8, 569, 12, 568, 9, - 17, 562, 561, 1, 558, 557, 0, +var exprPgo = [...]int{ + + 0, 612, 18, 611, 2, 14, 464, 3, 16, 7, + 609, 607, 606, 605, 15, 604, 603, 602, 601, 107, + 600, 38, 599, 579, 598, 597, 596, 594, 13, 5, + 593, 592, 589, 6, 585, 97, 4, 581, 580, 578, + 577, 575, 11, 573, 572, 8, 566, 12, 565, 9, + 17, 562, 561, 1, 555, 551, 0, } +var exprR1 = [...]int{ -var exprR1 = [...]int8{ 0, 1, 2, 2, 7, 7, 7, 7, 7, 7, 7, 6, 6, 6, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, @@ -426,12 +424,12 @@ var exprR1 = [...]int8{ 20, 20, 20, 20, 20, 20, 20, 24, 24, 25, 25, 25, 25, 23, 23, 23, 23, 23, 23, 23, 23, 21, 21, 21, 17, 18, 16, 16, 16, 16, - 16, 16, 16, 16, 16, 16, 16, 12, 12, 12, + 16, 16, 16, 16, 16, 16, 16, 16, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, - 12, 12, 56, 5, 5, 4, 4, 4, 4, + 12, 12, 12, 56, 5, 5, 4, 4, 4, 4, } +var exprR2 = [...]int{ -var exprR2 = [...]int8{ 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 3, 1, 2, 3, 2, 3, 4, 5, 3, 4, 5, 6, 3, 4, 5, 6, 3, 4, 5, 6, @@ -454,98 +452,98 @@ var exprR2 = [...]int8{ 5, 1, 2, 2, 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 1, 3, 4, 4, 3, 3, + 1, 1, 1, 2, 1, 3, 4, 4, 3, 3, } +var exprChk = [...]int{ -var exprChk = [...]int16{ -1000, -1, -2, -6, -7, -14, 26, -11, -15, -20, - -21, -22, -17, 17, -12, -16, 7, 89, 90, 68, - -18, 30, 31, 32, 44, 45, 54, 55, 56, 57, - 58, 59, 60, 64, 65, 66, 33, 36, 39, 37, - 38, 40, 41, 42, 43, 34, 35, 67, 80, 81, - 82, 89, 90, 91, 92, 93, 94, 83, 84, 87, - 88, 85, 86, -28, -29, -34, 50, -35, -3, 23, - 24, 25, 15, 84, 16, -7, -6, -2, -10, 18, - -9, 5, 26, 26, -4, 28, 29, 7, 7, 26, - 26, -23, -24, -25, 46, -23, -23, -23, -23, -23, - -23, -23, -23, -23, -23, -23, -23, -23, -23, -29, - -35, -27, -26, -52, -51, -33, -38, -39, -46, -40, - -43, 49, 47, 48, 69, 71, -9, -55, -54, -31, - 26, 51, 77, 52, 78, 79, 5, -32, -30, 80, - 6, -19, 72, 27, 27, 18, 2, 21, 13, 84, - 14, 15, -8, 7, -14, 26, -7, 7, 26, 26, - 26, -7, 7, -2, 73, 74, 75, 76, -2, -2, + -21, -22, -17, 17, -12, -16, 7, 90, 91, 69, + -18, 30, 31, 32, 45, 46, 55, 56, 57, 58, + 59, 60, 61, 65, 66, 67, 33, 36, 39, 37, + 38, 40, 41, 42, 43, 34, 35, 44, 68, 81, + 82, 83, 90, 91, 92, 93, 94, 95, 84, 85, + 88, 89, 86, 87, -28, -29, -34, 51, -35, -3, + 23, 24, 25, 15, 85, 16, -7, -6, -2, -10, + 18, -9, 5, 26, 26, -4, 28, 29, 7, 7, + 26, 26, -23, -24, -25, 47, -23, -23, -23, -23, + -23, -23, -23, -23, -23, -23, -23, -23, -23, -23, + -29, -35, -27, -26, -52, -51, -33, -38, -39, -46, + -40, -43, 50, 48, 49, 70, 72, -9, -55, -54, + -31, 26, 52, 78, 53, 79, 80, 5, -32, -30, + 81, 6, -19, 73, 27, 27, 18, 2, 21, 13, + 85, 14, 15, -8, 7, -14, 26, -7, 7, 26, + 26, 26, -7, 7, -2, 74, 75, 76, 77, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - -2, -2, -33, 81, 21, 80, -37, -50, 8, -49, - 5, -50, 6, 6, -33, 6, -48, -47, 5, -41, - -42, 5, -9, -44, -45, 5, -9, 13, 84, 87, - 88, 85, 86, 83, -36, 6, -19, 80, 26, -9, - 6, 6, 6, 6, 2, 27, 21, 10, -53, -28, - 50, -14, -8, 27, 21, -7, 7, -5, 27, 5, - -5, 27, 21, 27, 26, 26, 26, 26, -33, -33, - -33, 8, -50, 21, 13, 27, 21, 13, 21, 21, - 72, 9, 4, -21, 72, 9, 4, -21, 9, 4, - -21, 9, 4, -21, 9, 4, -21, 9, 4, -21, - 9, 4, -21, 80, 26, -36, 6, -4, -8, -56, - -53, -28, 70, 10, 50, 10, -53, 53, 27, -53, - -28, 27, -4, -7, 27, 21, 21, 27, 27, 6, - -5, 27, -5, 27, 27, -5, 27, -5, -49, 6, - -47, 2, 5, 6, -42, -45, 26, 26, -36, 6, - 27, 27, -53, -28, -53, 9, -56, -33, -56, 10, - 5, -13, 61, 62, 63, 10, 27, 27, -53, 27, - -7, 5, 21, 27, 27, 27, 27, 6, 6, 27, - -4, -53, -56, 26, -56, -53, 50, 10, 10, 27, - -4, 27, 6, 27, 27, 5, -53, -56, -56, 10, - 21, 27, -56, 6, 21, 6, 27, + -2, -2, -2, -33, 82, 21, 81, -37, -50, 8, + -49, 5, -50, 6, 6, -33, 6, -48, -47, 5, + -41, -42, 5, -9, -44, -45, 5, -9, 13, 85, + 88, 89, 86, 87, 84, -36, 6, -19, 81, 26, + -9, 6, 6, 6, 6, 2, 27, 21, 10, -53, + -28, 51, -14, -8, 27, 21, -7, 7, -5, 27, + 5, -5, 27, 21, 27, 26, 26, 26, 26, -33, + -33, -33, 8, -50, 21, 13, 27, 21, 13, 21, + 21, 73, 9, 4, -21, 73, 9, 4, -21, 9, + 4, -21, 9, 4, -21, 9, 4, -21, 9, 4, + -21, 9, 4, -21, 81, 26, -36, 6, -4, -8, + -56, -53, -28, 71, 10, 51, 10, -53, 54, 27, + -53, -28, 27, -4, -7, 27, 21, 21, 27, 27, + 6, -5, 27, -5, 27, 27, -5, 27, -5, -49, + 6, -47, 2, 5, 6, -42, -45, 26, 26, -36, + 6, 27, 27, -53, -28, -53, 9, -56, -33, -56, + 10, 5, -13, 62, 63, 64, 10, 27, 27, -53, + 27, -7, 5, 21, 27, 27, 27, 27, 6, 6, + 27, -4, -53, -56, 26, -56, -53, 51, 10, 10, + 27, -4, 27, 6, 27, 27, 5, -53, -56, -56, + 10, 21, 27, -56, 6, 21, 6, 27, } +var exprDef = [...]int{ -var exprDef = [...]int16{ 0, -2, 1, 2, 3, 11, 0, 4, 5, 6, 7, 8, 9, 0, 0, 0, 191, 0, 0, 0, - 0, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 196, 197, 198, 199, - 200, 201, 202, 203, 204, 205, 206, 195, 177, 177, + 0, 208, 209, 210, 211, 212, 213, 214, 215, 216, + 217, 218, 219, 220, 221, 222, 196, 197, 198, 199, + 200, 201, 202, 203, 204, 205, 206, 207, 195, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, - 177, 177, 177, 12, 72, 74, 0, 92, 0, 57, - 58, 59, 60, 61, 62, 3, 2, 0, 0, 65, - 66, 0, 0, 0, 0, 0, 0, 192, 193, 0, - 0, 0, 183, 184, 178, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 73, - 94, 75, 76, 77, 78, 79, 80, 81, 82, 83, - 84, 97, 99, 0, 101, 0, 114, 115, 116, 117, - 0, 0, 107, 0, 0, 0, 0, 129, 130, 0, - 89, 0, 85, 10, 13, 63, 64, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3, 191, 0, 0, - 0, 3, 0, 162, 0, 0, 185, 188, 163, 164, - 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, - 175, 176, 119, 0, 0, 0, 98, 105, 95, 125, - 124, 103, 100, 102, 0, 106, 113, 110, 0, 156, - 154, 152, 153, 161, 159, 157, 158, 0, 0, 0, - 0, 0, 0, 0, 93, 86, 0, 0, 0, 67, - 68, 69, 70, 71, 39, 46, 0, 14, 0, 0, - 0, 0, 0, 50, 0, 3, 191, 0, 227, 223, - 0, 228, 0, 194, 0, 0, 0, 0, 120, 121, - 122, 96, 104, 0, 0, 118, 0, 0, 0, 0, - 0, 136, 143, 150, 0, 135, 142, 149, 131, 138, - 145, 132, 139, 146, 133, 140, 147, 134, 141, 148, - 137, 144, 151, 0, 0, 91, 0, 48, 0, 15, - 18, 34, 0, 22, 0, 26, 0, 0, 0, 0, - 0, 38, 52, 3, 51, 0, 0, 225, 226, 0, - 0, 180, 0, 182, 186, 0, 189, 0, 126, 123, - 111, 112, 108, 109, 155, 160, 0, 0, 88, 0, - 90, 47, 19, 35, 36, 222, 23, 42, 27, 30, - 40, 0, 43, 44, 45, 16, 0, 0, 0, 53, - 3, 224, 0, 179, 181, 187, 190, 0, 0, 87, - 49, 37, 31, 0, 17, 20, 0, 24, 28, 0, - 54, 55, 0, 127, 128, 0, 21, 25, 29, 32, - 0, 41, 33, 0, 0, 0, 56, + 177, 177, 177, 177, 12, 72, 74, 0, 92, 0, + 57, 58, 59, 60, 61, 62, 3, 2, 0, 0, + 65, 66, 0, 0, 0, 0, 0, 0, 192, 193, + 0, 0, 0, 183, 184, 178, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 73, 94, 75, 76, 77, 78, 79, 80, 81, 82, + 83, 84, 97, 99, 0, 101, 0, 114, 115, 116, + 117, 0, 0, 107, 0, 0, 0, 0, 129, 130, + 0, 89, 0, 85, 10, 13, 63, 64, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3, 191, 0, + 0, 0, 3, 0, 162, 0, 0, 185, 188, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, + 174, 175, 176, 119, 0, 0, 0, 98, 105, 95, + 125, 124, 103, 100, 102, 0, 106, 113, 110, 0, + 156, 154, 152, 153, 161, 159, 157, 158, 0, 0, + 0, 0, 0, 0, 0, 93, 86, 0, 0, 0, + 67, 68, 69, 70, 71, 39, 46, 0, 14, 0, + 0, 0, 0, 0, 50, 0, 3, 191, 0, 228, + 224, 0, 229, 0, 194, 0, 0, 0, 0, 120, + 121, 122, 96, 104, 0, 0, 118, 0, 0, 0, + 0, 0, 136, 143, 150, 0, 135, 142, 149, 131, + 138, 145, 132, 139, 146, 133, 140, 147, 134, 141, + 148, 137, 144, 151, 0, 0, 91, 0, 48, 0, + 15, 18, 34, 0, 22, 0, 26, 0, 0, 0, + 0, 0, 38, 52, 3, 51, 0, 0, 226, 227, + 0, 0, 180, 0, 182, 186, 0, 189, 0, 126, + 123, 111, 112, 108, 109, 155, 160, 0, 0, 88, + 0, 90, 47, 19, 35, 36, 223, 23, 42, 27, + 30, 40, 0, 43, 44, 45, 16, 0, 0, 0, + 53, 3, 225, 0, 179, 181, 187, 190, 0, 0, + 87, 49, 37, 31, 0, 17, 20, 0, 24, 28, + 0, 54, 55, 0, 127, 128, 0, 21, 25, 29, + 32, 0, 41, 33, 0, 0, 0, 56, } +var exprTok1 = [...]int{ -var exprTok1 = [...]int8{ 1, } +var exprTok2 = [...]int{ -var exprTok2 = [...]int8{ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, @@ -555,10 +553,9 @@ var exprTok2 = [...]int8{ 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, 93, 94, + 92, 93, 94, 95, } - -var exprTok3 = [...]int8{ +var exprTok3 = [...]int{ 0, } @@ -568,7 +565,6 @@ var exprErrorMessages = [...]struct { msg string }{} -//line yaccpar:1 /* parser for yacc output */ @@ -640,9 +636,9 @@ func exprErrorMessage(state, lookAhead int) string { expected := make([]int, 0, 4) // Look for shiftable tokens. - base := int(exprPact[state]) + base := exprPact[state] for tok := TOKSTART; tok-1 < len(exprToknames); tok++ { - if n := base + tok; n >= 0 && n < exprLast && int(exprChk[int(exprAct[n])]) == tok { + if n := base + tok; n >= 0 && n < exprLast && exprChk[exprAct[n]] == tok { if len(expected) == cap(expected) { return res } @@ -652,13 +648,13 @@ func exprErrorMessage(state, lookAhead int) string { if exprDef[state] == -2 { i := 0 - for exprExca[i] != -1 || int(exprExca[i+1]) != state { + for exprExca[i] != -1 || exprExca[i+1] != state { i += 2 } // Look for tokens that we accept or reduce. for i += 2; exprExca[i] >= 0; i += 2 { - tok := int(exprExca[i]) + tok := exprExca[i] if tok < TOKSTART || exprExca[i+1] == 0 { continue } @@ -689,30 +685,30 @@ func exprlex1(lex exprLexer, lval *exprSymType) (char, token int) { token = 0 char = lex.Lex(lval) if char <= 0 { - token = int(exprTok1[0]) + token = exprTok1[0] goto out } if char < len(exprTok1) { - token = int(exprTok1[char]) + token = exprTok1[char] goto out } if char >= exprPrivate { if char < exprPrivate+len(exprTok2) { - token = int(exprTok2[char-exprPrivate]) + token = exprTok2[char-exprPrivate] goto out } } for i := 0; i < len(exprTok3); i += 2 { - token = int(exprTok3[i+0]) + token = exprTok3[i+0] if token == char { - token = int(exprTok3[i+1]) + token = exprTok3[i+1] goto out } } out: if token == 0 { - token = int(exprTok2[1]) /* unknown char */ + token = exprTok2[1] /* unknown char */ } if exprDebug >= 3 { __yyfmt__.Printf("lex %s(%d)\n", exprTokname(token), uint(char)) @@ -767,7 +763,7 @@ exprstack: exprS[exprp].yys = exprstate exprnewstate: - exprn = int(exprPact[exprstate]) + exprn = exprPact[exprstate] if exprn <= exprFlag { goto exprdefault /* simple state */ } @@ -778,8 +774,8 @@ exprnewstate: if exprn < 0 || exprn >= exprLast { goto exprdefault } - exprn = int(exprAct[exprn]) - if int(exprChk[exprn]) == exprtoken { /* valid shift */ + exprn = exprAct[exprn] + if exprChk[exprn] == exprtoken { /* valid shift */ exprrcvr.char = -1 exprtoken = -1 exprVAL = exprrcvr.lval @@ -792,7 +788,7 @@ exprnewstate: exprdefault: /* default state action */ - exprn = int(exprDef[exprstate]) + exprn = exprDef[exprstate] if exprn == -2 { if exprrcvr.char < 0 { exprrcvr.char, exprtoken = exprlex1(exprlex, &exprrcvr.lval) @@ -801,18 +797,18 @@ exprdefault: /* look through exception table */ xi := 0 for { - if exprExca[xi+0] == -1 && int(exprExca[xi+1]) == exprstate { + if exprExca[xi+0] == -1 && exprExca[xi+1] == exprstate { break } xi += 2 } for xi += 2; ; xi += 2 { - exprn = int(exprExca[xi+0]) + exprn = exprExca[xi+0] if exprn < 0 || exprn == exprtoken { break } } - exprn = int(exprExca[xi+1]) + exprn = exprExca[xi+1] if exprn < 0 { goto ret0 } @@ -834,10 +830,10 @@ exprdefault: /* find a state where "error" is a legal shift action */ for exprp >= 0 { - exprn = int(exprPact[exprS[exprp].yys]) + exprErrCode + exprn = exprPact[exprS[exprp].yys] + exprErrCode if exprn >= 0 && exprn < exprLast { - exprstate = int(exprAct[exprn]) /* simulate a shift of "error" */ - if int(exprChk[exprstate]) == exprErrCode { + exprstate = exprAct[exprn] /* simulate a shift of "error" */ + if exprChk[exprstate] == exprErrCode { goto exprstack } } @@ -873,7 +869,7 @@ exprdefault: exprpt := exprp _ = exprpt // guard against "declared and not used" - exprp -= int(exprR2[exprn]) + exprp -= exprR2[exprn] // exprp is now the index of $0. Perform the default action. Iff the // reduced production is ε, $1 is possibly out of range. if exprp+1 >= len(exprS) { @@ -884,16 +880,16 @@ exprdefault: exprVAL = exprS[exprp+1] /* consult goto table to find next state */ - exprn = int(exprR1[exprn]) - exprg := int(exprPgo[exprn]) + exprn = exprR1[exprn] + exprg := exprPgo[exprn] exprj := exprg + exprS[exprp].yys + 1 if exprj >= exprLast { - exprstate = int(exprAct[exprg]) + exprstate = exprAct[exprg] } else { - exprstate = int(exprAct[exprj]) - if int(exprChk[exprstate]) != -exprn { - exprstate = int(exprAct[exprg]) + exprstate = exprAct[exprj] + if exprChk[exprstate] != -exprn { + exprstate = exprAct[exprg] } } // dummy call; replaced with literal code @@ -901,1062 +897,885 @@ exprdefault: case 1: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:154 { exprlex.(*parser).expr = exprDollar[1].Expr } case 2: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:157 { exprVAL.Expr = exprDollar[1].LogExpr } case 3: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:158 { exprVAL.Expr = exprDollar[1].MetricExpr } case 4: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:162 { exprVAL.MetricExpr = exprDollar[1].RangeAggregationExpr } case 5: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:163 { exprVAL.MetricExpr = exprDollar[1].VectorAggregationExpr } case 6: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:164 { exprVAL.MetricExpr = exprDollar[1].BinOpExpr } case 7: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:165 { exprVAL.MetricExpr = exprDollar[1].LiteralExpr } case 8: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:166 { exprVAL.MetricExpr = exprDollar[1].LabelReplaceExpr } case 9: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:167 { exprVAL.MetricExpr = exprDollar[1].VectorExpr } case 10: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:168 { exprVAL.MetricExpr = exprDollar[2].MetricExpr } case 11: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:172 { exprVAL.LogExpr = newMatcherExpr(exprDollar[1].Selector) } case 12: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:173 { exprVAL.LogExpr = newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].PipelineExpr) } case 13: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:174 { exprVAL.LogExpr = exprDollar[2].LogExpr } case 14: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:178 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].duration, nil, nil) } case 15: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:179 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].duration, nil, exprDollar[3].OffsetExpr) } case 16: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:180 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[4].duration, nil, nil) } case 17: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:181 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[4].duration, nil, exprDollar[5].OffsetExpr) } case 18: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:182 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].duration, exprDollar[3].UnwrapExpr, nil) } case 19: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:183 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].duration, exprDollar[4].UnwrapExpr, exprDollar[3].OffsetExpr) } case 20: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:184 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[4].duration, exprDollar[5].UnwrapExpr, nil) } case 21: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:185 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[4].duration, exprDollar[6].UnwrapExpr, exprDollar[5].OffsetExpr) } case 22: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:186 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[3].duration, exprDollar[2].UnwrapExpr, nil) } case 23: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:187 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[1].Selector), exprDollar[3].duration, exprDollar[2].UnwrapExpr, exprDollar[4].OffsetExpr) } case 24: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:188 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[5].duration, exprDollar[3].UnwrapExpr, nil) } case 25: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:189 { exprVAL.LogRangeExpr = newLogRange(newMatcherExpr(exprDollar[2].Selector), exprDollar[5].duration, exprDollar[3].UnwrapExpr, exprDollar[6].OffsetExpr) } case 26: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:190 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].PipelineExpr), exprDollar[3].duration, nil, nil) } case 27: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:191 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].PipelineExpr), exprDollar[3].duration, nil, exprDollar[4].OffsetExpr) } case 28: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:192 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[2].Selector), exprDollar[3].PipelineExpr), exprDollar[5].duration, nil, nil) } case 29: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:193 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[2].Selector), exprDollar[3].PipelineExpr), exprDollar[5].duration, nil, exprDollar[6].OffsetExpr) } case 30: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:194 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].PipelineExpr), exprDollar[4].duration, exprDollar[3].UnwrapExpr, nil) } case 31: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:195 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[2].PipelineExpr), exprDollar[4].duration, exprDollar[3].UnwrapExpr, exprDollar[5].OffsetExpr) } case 32: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:196 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[2].Selector), exprDollar[3].PipelineExpr), exprDollar[6].duration, exprDollar[4].UnwrapExpr, nil) } case 33: exprDollar = exprS[exprpt-7 : exprpt+1] -//line expr.y:197 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[2].Selector), exprDollar[3].PipelineExpr), exprDollar[6].duration, exprDollar[4].UnwrapExpr, exprDollar[7].OffsetExpr) } case 34: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:198 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[3].PipelineExpr), exprDollar[2].duration, nil, nil) } case 35: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:199 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[4].PipelineExpr), exprDollar[2].duration, nil, exprDollar[3].OffsetExpr) } case 36: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:200 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[3].PipelineExpr), exprDollar[2].duration, exprDollar[4].UnwrapExpr, nil) } case 37: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:201 { exprVAL.LogRangeExpr = newLogRange(newPipelineExpr(newMatcherExpr(exprDollar[1].Selector), exprDollar[4].PipelineExpr), exprDollar[2].duration, exprDollar[5].UnwrapExpr, exprDollar[3].OffsetExpr) } case 38: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:202 { exprVAL.LogRangeExpr = exprDollar[2].LogRangeExpr } case 40: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:207 { exprVAL.UnwrapExpr = newUnwrapExpr(exprDollar[3].str, "") } case 41: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:208 { exprVAL.UnwrapExpr = newUnwrapExpr(exprDollar[5].str, exprDollar[3].ConvOp) } case 42: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:209 { exprVAL.UnwrapExpr = exprDollar[1].UnwrapExpr.addPostFilter(exprDollar[3].LabelFilter) } case 43: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:213 { exprVAL.ConvOp = OpConvBytes } case 44: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:214 { exprVAL.ConvOp = OpConvDuration } case 45: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:215 { exprVAL.ConvOp = OpConvDurationSeconds } case 46: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:219 { exprVAL.RangeAggregationExpr = newRangeAggregationExpr(exprDollar[3].LogRangeExpr, exprDollar[1].RangeOp, nil, nil) } case 47: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:220 { exprVAL.RangeAggregationExpr = newRangeAggregationExpr(exprDollar[5].LogRangeExpr, exprDollar[1].RangeOp, nil, &exprDollar[3].str) } case 48: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:221 { exprVAL.RangeAggregationExpr = newRangeAggregationExpr(exprDollar[3].LogRangeExpr, exprDollar[1].RangeOp, exprDollar[5].Grouping, nil) } case 49: exprDollar = exprS[exprpt-7 : exprpt+1] -//line expr.y:222 { exprVAL.RangeAggregationExpr = newRangeAggregationExpr(exprDollar[5].LogRangeExpr, exprDollar[1].RangeOp, exprDollar[7].Grouping, &exprDollar[3].str) } case 50: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:227 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[3].MetricExpr, exprDollar[1].VectorOp, nil, nil) } case 51: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:228 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[4].MetricExpr, exprDollar[1].VectorOp, exprDollar[2].Grouping, nil) } case 52: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:229 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[3].MetricExpr, exprDollar[1].VectorOp, exprDollar[5].Grouping, nil) } case 53: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:231 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[5].MetricExpr, exprDollar[1].VectorOp, nil, &exprDollar[3].str) } case 54: exprDollar = exprS[exprpt-7 : exprpt+1] -//line expr.y:232 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[5].MetricExpr, exprDollar[1].VectorOp, exprDollar[7].Grouping, &exprDollar[3].str) } case 55: exprDollar = exprS[exprpt-7 : exprpt+1] -//line expr.y:233 { exprVAL.VectorAggregationExpr = mustNewVectorAggregationExpr(exprDollar[6].MetricExpr, exprDollar[1].VectorOp, exprDollar[2].Grouping, &exprDollar[4].str) } case 56: exprDollar = exprS[exprpt-12 : exprpt+1] -//line expr.y:238 { exprVAL.LabelReplaceExpr = mustNewLabelReplaceExpr(exprDollar[3].MetricExpr, exprDollar[5].str, exprDollar[7].str, exprDollar[9].str, exprDollar[11].str) } case 57: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:242 { exprVAL.Filter = log.LineMatchRegexp } case 58: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:243 { exprVAL.Filter = log.LineMatchEqual } case 59: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:244 { exprVAL.Filter = log.LineMatchPattern } case 60: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:245 { exprVAL.Filter = log.LineMatchNotRegexp } case 61: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:246 { exprVAL.Filter = log.LineMatchNotEqual } case 62: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:247 { exprVAL.Filter = log.LineMatchNotPattern } case 63: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:251 { exprVAL.Selector = exprDollar[2].Matchers } case 64: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:252 { exprVAL.Selector = exprDollar[2].Matchers } case 65: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:253 { } case 66: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:257 { exprVAL.Matchers = []*labels.Matcher{exprDollar[1].Matcher} } case 67: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:258 { exprVAL.Matchers = append(exprDollar[1].Matchers, exprDollar[3].Matcher) } case 68: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:262 { exprVAL.Matcher = mustNewMatcher(labels.MatchEqual, exprDollar[1].str, exprDollar[3].str) } case 69: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:263 { exprVAL.Matcher = mustNewMatcher(labels.MatchNotEqual, exprDollar[1].str, exprDollar[3].str) } case 70: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:264 { exprVAL.Matcher = mustNewMatcher(labels.MatchRegexp, exprDollar[1].str, exprDollar[3].str) } case 71: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:265 { exprVAL.Matcher = mustNewMatcher(labels.MatchNotRegexp, exprDollar[1].str, exprDollar[3].str) } case 72: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:269 { exprVAL.PipelineExpr = MultiStageExpr{exprDollar[1].PipelineStage} } case 73: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:270 { exprVAL.PipelineExpr = append(exprDollar[1].PipelineExpr, exprDollar[2].PipelineStage) } case 74: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:274 { exprVAL.PipelineStage = exprDollar[1].LineFilters } case 75: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:275 { exprVAL.PipelineStage = exprDollar[2].LogfmtParser } case 76: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:276 { exprVAL.PipelineStage = exprDollar[2].LabelParser } case 77: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:277 { exprVAL.PipelineStage = exprDollar[2].JSONExpressionParser } case 78: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:278 { exprVAL.PipelineStage = exprDollar[2].LogfmtExpressionParser } case 79: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:279 { exprVAL.PipelineStage = &LabelFilterExpr{LabelFilterer: exprDollar[2].LabelFilter} } case 80: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:280 { exprVAL.PipelineStage = exprDollar[2].LineFormatExpr } case 81: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:281 { exprVAL.PipelineStage = exprDollar[2].DecolorizeExpr } case 82: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:282 { exprVAL.PipelineStage = exprDollar[2].LabelFormatExpr } case 83: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:283 { exprVAL.PipelineStage = exprDollar[2].DropLabelsExpr } case 84: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:284 { exprVAL.PipelineStage = exprDollar[2].KeepLabelsExpr } case 85: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:288 { exprVAL.FilterOp = OpFilterIP } case 86: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:292 { exprVAL.OrFilter = newLineFilterExpr(log.LineMatchEqual, "", exprDollar[1].str) } case 87: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:293 { exprVAL.OrFilter = newLineFilterExpr(log.LineMatchEqual, exprDollar[1].FilterOp, exprDollar[3].str) } case 88: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:294 { exprVAL.OrFilter = newOrLineFilter(newLineFilterExpr(log.LineMatchEqual, "", exprDollar[1].str), exprDollar[3].OrFilter) } case 89: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:298 { exprVAL.LineFilter = newLineFilterExpr(exprDollar[1].Filter, "", exprDollar[2].str) } case 90: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:299 { exprVAL.LineFilter = newLineFilterExpr(exprDollar[1].Filter, exprDollar[2].FilterOp, exprDollar[4].str) } case 91: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:300 { exprVAL.LineFilter = newOrLineFilter(newLineFilterExpr(exprDollar[1].Filter, "", exprDollar[2].str), exprDollar[4].OrFilter) } case 92: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:304 { exprVAL.LineFilters = exprDollar[1].LineFilter } case 93: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:305 { exprVAL.LineFilters = newOrLineFilter(exprDollar[1].LineFilter, exprDollar[3].OrFilter) } case 94: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:306 { exprVAL.LineFilters = newNestedLineFilterExpr(exprDollar[1].LineFilters, exprDollar[2].LineFilter) } case 95: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:310 { exprVAL.ParserFlags = []string{exprDollar[1].str} } case 96: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:311 { exprVAL.ParserFlags = append(exprDollar[1].ParserFlags, exprDollar[2].str) } case 97: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:315 { exprVAL.LogfmtParser = newLogfmtParserExpr(nil) } case 98: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:316 { exprVAL.LogfmtParser = newLogfmtParserExpr(exprDollar[2].ParserFlags) } case 99: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:320 { exprVAL.LabelParser = newLabelParserExpr(OpParserTypeJSON, "") } case 100: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:321 { exprVAL.LabelParser = newLabelParserExpr(OpParserTypeRegexp, exprDollar[2].str) } case 101: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:322 { exprVAL.LabelParser = newLabelParserExpr(OpParserTypeUnpack, "") } case 102: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:323 { exprVAL.LabelParser = newLabelParserExpr(OpParserTypePattern, exprDollar[2].str) } case 103: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:327 { exprVAL.JSONExpressionParser = newJSONExpressionParser(exprDollar[2].LabelExtractionExpressionList) } case 104: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:330 { exprVAL.LogfmtExpressionParser = newLogfmtExpressionParser(exprDollar[3].LabelExtractionExpressionList, exprDollar[2].ParserFlags) } case 105: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:331 { exprVAL.LogfmtExpressionParser = newLogfmtExpressionParser(exprDollar[2].LabelExtractionExpressionList, nil) } case 106: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:334 { exprVAL.LineFormatExpr = newLineFmtExpr(exprDollar[2].str) } case 107: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:336 { exprVAL.DecolorizeExpr = newDecolorizeExpr() } case 108: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:339 { exprVAL.LabelFormat = log.NewRenameLabelFmt(exprDollar[1].str, exprDollar[3].str) } case 109: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:340 { exprVAL.LabelFormat = log.NewTemplateLabelFmt(exprDollar[1].str, exprDollar[3].str) } case 110: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:344 { exprVAL.LabelsFormat = []log.LabelFmt{exprDollar[1].LabelFormat} } case 111: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:345 { exprVAL.LabelsFormat = append(exprDollar[1].LabelsFormat, exprDollar[3].LabelFormat) } case 113: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:350 { exprVAL.LabelFormatExpr = newLabelFmtExpr(exprDollar[2].LabelsFormat) } case 114: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:353 { exprVAL.LabelFilter = log.NewStringLabelFilter(exprDollar[1].Matcher) } case 115: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:354 { exprVAL.LabelFilter = exprDollar[1].IPLabelFilter } case 116: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:355 { exprVAL.LabelFilter = exprDollar[1].UnitFilter } case 117: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:356 { exprVAL.LabelFilter = exprDollar[1].NumberFilter } case 118: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:357 { exprVAL.LabelFilter = exprDollar[2].LabelFilter } case 119: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:358 { exprVAL.LabelFilter = log.NewAndLabelFilter(exprDollar[1].LabelFilter, exprDollar[2].LabelFilter) } case 120: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:359 { exprVAL.LabelFilter = log.NewAndLabelFilter(exprDollar[1].LabelFilter, exprDollar[3].LabelFilter) } case 121: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:360 { exprVAL.LabelFilter = log.NewAndLabelFilter(exprDollar[1].LabelFilter, exprDollar[3].LabelFilter) } case 122: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:361 { exprVAL.LabelFilter = log.NewOrLabelFilter(exprDollar[1].LabelFilter, exprDollar[3].LabelFilter) } case 123: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:365 { exprVAL.LabelExtractionExpression = log.NewLabelExtractionExpr(exprDollar[1].str, exprDollar[3].str) } case 124: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:366 { exprVAL.LabelExtractionExpression = log.NewLabelExtractionExpr(exprDollar[1].str, exprDollar[1].str) } case 125: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:369 { exprVAL.LabelExtractionExpressionList = []log.LabelExtractionExpr{exprDollar[1].LabelExtractionExpression} } case 126: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:370 { exprVAL.LabelExtractionExpressionList = append(exprDollar[1].LabelExtractionExpressionList, exprDollar[3].LabelExtractionExpression) } case 127: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:374 { exprVAL.IPLabelFilter = log.NewIPLabelFilter(exprDollar[5].str, exprDollar[1].str, log.LabelFilterEqual) } case 128: exprDollar = exprS[exprpt-6 : exprpt+1] -//line expr.y:375 { exprVAL.IPLabelFilter = log.NewIPLabelFilter(exprDollar[5].str, exprDollar[1].str, log.LabelFilterNotEqual) } case 129: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:379 { exprVAL.UnitFilter = exprDollar[1].DurationFilter } case 130: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:380 { exprVAL.UnitFilter = exprDollar[1].BytesFilter } case 131: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:383 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterGreaterThan, exprDollar[1].str, exprDollar[3].duration) } case 132: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:384 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterGreaterThanOrEqual, exprDollar[1].str, exprDollar[3].duration) } case 133: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:385 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterLesserThan, exprDollar[1].str, exprDollar[3].duration) } case 134: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:386 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterLesserThanOrEqual, exprDollar[1].str, exprDollar[3].duration) } case 135: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:387 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterNotEqual, exprDollar[1].str, exprDollar[3].duration) } case 136: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:388 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].duration) } case 137: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:389 { exprVAL.DurationFilter = log.NewDurationLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].duration) } case 138: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:393 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterGreaterThan, exprDollar[1].str, exprDollar[3].bytes) } case 139: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:394 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterGreaterThanOrEqual, exprDollar[1].str, exprDollar[3].bytes) } case 140: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:395 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterLesserThan, exprDollar[1].str, exprDollar[3].bytes) } case 141: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:396 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterLesserThanOrEqual, exprDollar[1].str, exprDollar[3].bytes) } case 142: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:397 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterNotEqual, exprDollar[1].str, exprDollar[3].bytes) } case 143: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:398 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].bytes) } case 144: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:399 { exprVAL.BytesFilter = log.NewBytesLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].bytes) } case 145: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:403 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterGreaterThan, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 146: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:404 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterGreaterThanOrEqual, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 147: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:405 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterLesserThan, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 148: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:406 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterLesserThanOrEqual, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 149: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:407 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterNotEqual, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 150: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:408 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 151: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:409 { exprVAL.NumberFilter = log.NewNumericLabelFilter(log.LabelFilterEqual, exprDollar[1].str, exprDollar[3].LiteralExpr.Val) } case 152: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:413 { exprVAL.DropLabel = log.NewDropLabel(nil, exprDollar[1].str) } case 153: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:414 { exprVAL.DropLabel = log.NewDropLabel(exprDollar[1].Matcher, "") } case 154: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:417 { exprVAL.DropLabels = []log.DropLabel{exprDollar[1].DropLabel} } case 155: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:418 { exprVAL.DropLabels = append(exprDollar[1].DropLabels, exprDollar[3].DropLabel) } case 156: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:421 { exprVAL.DropLabelsExpr = newDropLabelsExpr(exprDollar[2].DropLabels) } case 157: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:424 { exprVAL.KeepLabel = log.NewKeepLabel(nil, exprDollar[1].str) } case 158: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:425 { exprVAL.KeepLabel = log.NewKeepLabel(exprDollar[1].Matcher, "") } case 159: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:428 { exprVAL.KeepLabels = []log.KeepLabel{exprDollar[1].KeepLabel} } case 160: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:429 { exprVAL.KeepLabels = append(exprDollar[1].KeepLabels, exprDollar[3].KeepLabel) } case 161: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:432 { exprVAL.KeepLabelsExpr = newKeepLabelsExpr(exprDollar[2].KeepLabels) } case 162: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:436 { exprVAL.BinOpExpr = mustNewBinOpExpr("or", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 163: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:437 { exprVAL.BinOpExpr = mustNewBinOpExpr("and", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 164: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:438 { exprVAL.BinOpExpr = mustNewBinOpExpr("unless", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 165: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:439 { exprVAL.BinOpExpr = mustNewBinOpExpr("+", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 166: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:440 { exprVAL.BinOpExpr = mustNewBinOpExpr("-", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 167: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:441 { exprVAL.BinOpExpr = mustNewBinOpExpr("*", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 168: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:442 { exprVAL.BinOpExpr = mustNewBinOpExpr("/", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 169: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:443 { exprVAL.BinOpExpr = mustNewBinOpExpr("%", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 170: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:444 { exprVAL.BinOpExpr = mustNewBinOpExpr("^", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 171: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:445 { exprVAL.BinOpExpr = mustNewBinOpExpr("==", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 172: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:446 { exprVAL.BinOpExpr = mustNewBinOpExpr("!=", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 173: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:447 { exprVAL.BinOpExpr = mustNewBinOpExpr(">", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 174: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:448 { exprVAL.BinOpExpr = mustNewBinOpExpr(">=", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 175: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:449 { exprVAL.BinOpExpr = mustNewBinOpExpr("<", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 176: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:450 { exprVAL.BinOpExpr = mustNewBinOpExpr("<=", exprDollar[3].BinOpModifier, exprDollar[1].Expr, exprDollar[4].Expr) } case 177: exprDollar = exprS[exprpt-0 : exprpt+1] -//line expr.y:454 { exprVAL.BoolModifier = &BinOpOptions{VectorMatching: &VectorMatching{Card: CardOneToOne}} } case 178: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:458 { exprVAL.BoolModifier = &BinOpOptions{VectorMatching: &VectorMatching{Card: CardOneToOne}, ReturnBool: true} } case 179: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:465 { exprVAL.OnOrIgnoringModifier = exprDollar[1].BoolModifier exprVAL.OnOrIgnoringModifier.VectorMatching.On = true @@ -1964,53 +1783,45 @@ exprdefault: } case 180: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:471 { exprVAL.OnOrIgnoringModifier = exprDollar[1].BoolModifier exprVAL.OnOrIgnoringModifier.VectorMatching.On = true } case 181: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:476 { exprVAL.OnOrIgnoringModifier = exprDollar[1].BoolModifier exprVAL.OnOrIgnoringModifier.VectorMatching.MatchingLabels = exprDollar[4].Labels } case 182: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:481 { exprVAL.OnOrIgnoringModifier = exprDollar[1].BoolModifier } case 183: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:487 { exprVAL.BinOpModifier = exprDollar[1].BoolModifier } case 184: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:488 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier } case 185: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:490 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardManyToOne } case 186: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:495 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardManyToOne } case 187: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:500 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardManyToOne @@ -2018,21 +1829,18 @@ exprdefault: } case 188: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:506 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardOneToMany } case 189: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:511 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardOneToMany } case 190: exprDollar = exprS[exprpt-5 : exprpt+1] -//line expr.y:516 { exprVAL.BinOpModifier = exprDollar[1].OnOrIgnoringModifier exprVAL.BinOpModifier.VectorMatching.Card = CardOneToMany @@ -2040,229 +1848,196 @@ exprdefault: } case 191: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:524 { exprVAL.LiteralExpr = mustNewLiteralExpr(exprDollar[1].str, false) } case 192: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:525 { exprVAL.LiteralExpr = mustNewLiteralExpr(exprDollar[2].str, false) } case 193: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:526 { exprVAL.LiteralExpr = mustNewLiteralExpr(exprDollar[2].str, true) } case 194: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:530 { exprVAL.VectorExpr = NewVectorExpr(exprDollar[3].str) } case 195: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:533 { exprVAL.Vector = OpTypeVector } case 196: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:537 { exprVAL.VectorOp = OpTypeSum } case 197: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:538 { exprVAL.VectorOp = OpTypeAvg } case 198: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:539 { exprVAL.VectorOp = OpTypeCount } case 199: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:540 { exprVAL.VectorOp = OpTypeMax } case 200: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:541 { exprVAL.VectorOp = OpTypeMin } case 201: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:542 { exprVAL.VectorOp = OpTypeStddev } case 202: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:543 { exprVAL.VectorOp = OpTypeStdvar } case 203: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:544 { exprVAL.VectorOp = OpTypeBottomK } case 204: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:545 { exprVAL.VectorOp = OpTypeTopK } case 205: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:546 { exprVAL.VectorOp = OpTypeSort } case 206: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:547 { exprVAL.VectorOp = OpTypeSortDesc } case 207: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:551 { - exprVAL.RangeOp = OpRangeTypeCount + exprVAL.VectorOp = OpTypeApproxTopK } case 208: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:552 { - exprVAL.RangeOp = OpRangeTypeRate + exprVAL.RangeOp = OpRangeTypeCount } case 209: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:553 { - exprVAL.RangeOp = OpRangeTypeRateCounter + exprVAL.RangeOp = OpRangeTypeRate } case 210: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:554 { - exprVAL.RangeOp = OpRangeTypeBytes + exprVAL.RangeOp = OpRangeTypeRateCounter } case 211: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:555 { - exprVAL.RangeOp = OpRangeTypeBytesRate + exprVAL.RangeOp = OpRangeTypeBytes } case 212: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:556 { - exprVAL.RangeOp = OpRangeTypeAvg + exprVAL.RangeOp = OpRangeTypeBytesRate } case 213: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:557 { - exprVAL.RangeOp = OpRangeTypeSum + exprVAL.RangeOp = OpRangeTypeAvg } case 214: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:558 { - exprVAL.RangeOp = OpRangeTypeMin + exprVAL.RangeOp = OpRangeTypeSum } case 215: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:559 { - exprVAL.RangeOp = OpRangeTypeMax + exprVAL.RangeOp = OpRangeTypeMin } case 216: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:560 { - exprVAL.RangeOp = OpRangeTypeStdvar + exprVAL.RangeOp = OpRangeTypeMax } case 217: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:561 { - exprVAL.RangeOp = OpRangeTypeStddev + exprVAL.RangeOp = OpRangeTypeStdvar } case 218: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:562 { - exprVAL.RangeOp = OpRangeTypeQuantile + exprVAL.RangeOp = OpRangeTypeStddev } case 219: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:563 { - exprVAL.RangeOp = OpRangeTypeFirst + exprVAL.RangeOp = OpRangeTypeQuantile } case 220: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:564 { - exprVAL.RangeOp = OpRangeTypeLast + exprVAL.RangeOp = OpRangeTypeFirst } case 221: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:565 { - exprVAL.RangeOp = OpRangeTypeAbsent + exprVAL.RangeOp = OpRangeTypeLast } case 222: + exprDollar = exprS[exprpt-1 : exprpt+1] + { + exprVAL.RangeOp = OpRangeTypeAbsent + } + case 223: exprDollar = exprS[exprpt-2 : exprpt+1] -//line expr.y:569 { exprVAL.OffsetExpr = newOffsetExpr(exprDollar[2].duration) } - case 223: + case 224: exprDollar = exprS[exprpt-1 : exprpt+1] -//line expr.y:572 { exprVAL.Labels = []string{exprDollar[1].str} } - case 224: + case 225: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:573 { exprVAL.Labels = append(exprDollar[1].Labels, exprDollar[3].str) } - case 225: + case 226: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:577 { exprVAL.Grouping = &Grouping{Without: false, Groups: exprDollar[3].Labels} } - case 226: + case 227: exprDollar = exprS[exprpt-4 : exprpt+1] -//line expr.y:578 { exprVAL.Grouping = &Grouping{Without: true, Groups: exprDollar[3].Labels} } - case 227: + case 228: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:579 { exprVAL.Grouping = &Grouping{Without: false, Groups: nil} } - case 228: + case 229: exprDollar = exprS[exprpt-3 : exprpt+1] -//line expr.y:580 { exprVAL.Grouping = &Grouping{Without: true, Groups: nil} } diff --git a/pkg/logql/syntax/lex.go b/pkg/logql/syntax/lex.go index dffb05ab0189b..4175682efa5cf 100644 --- a/pkg/logql/syntax/lex.go +++ b/pkg/logql/syntax/lex.go @@ -121,6 +121,8 @@ var functionTokens = map[string]int{ OpTypeSortDesc: SORT_DESC, OpLabelReplace: LABEL_REPLACE, + OpTypeApproxTopK: APPROX_TOPK, + // conversion Op OpConvBytes: BYTES_CONV, OpConvDuration: DURATION_CONV, diff --git a/pkg/logql/syntax/parser_test.go b/pkg/logql/syntax/parser_test.go index f6c919317f5d3..7f257cd3c25d1 100644 --- a/pkg/logql/syntax/parser_test.go +++ b/pkg/logql/syntax/parser_test.go @@ -364,6 +364,10 @@ var ParseTestCases = []struct { in: `absent_over_time({ foo = "bar" }[5h]) by (foo)`, err: logqlmodel.NewParseError("grouping not allowed for absent_over_time aggregation", 0, 0), }, + { + in: `approx_topk(2, count_over_time({ foo = "bar" }[5h])) by (foo)`, + err: logqlmodel.NewParseError("grouping not allowed for approx_topk aggregation", 0, 0), + }, { in: `rate({ foo = "bar" }[5minutes])`, err: logqlmodel.NewParseError(`unknown unit "minutes" in duration "5minutes"`, 0, 21), diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 61a60e44f80fa..e7f003327ea01 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -7,10 +7,12 @@ import ( "math/rand" "sort" "strings" + "sync" "time" "unicode" "github.com/cespare/xxhash/v2" + "github.com/grafana/dskit/concurrency" "github.com/prometheus/prometheus/model/labels" promql_parser "github.com/prometheus/prometheus/promql/parser" @@ -21,6 +23,8 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) +const ConCurrency = 100 + func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier { return MockQuerier{ shards: shards, @@ -45,7 +49,6 @@ func (q MockQuerier) extractOldShard(xs []string) (*index.ShardAnnotation, error } return parsed[0].PowerOfTwo, nil - } func (q MockQuerier) SelectLogs(_ context.Context, req SelectLogParams) (iter.EntryIterator, error) { @@ -113,8 +116,8 @@ func processStream(in []logproto.Stream, pipeline log.Pipeline) []logproto.Strea resByStream := map[string]*logproto.Stream{} for _, stream := range in { + sp := pipeline.ForStream(mustParseLabels(stream.Labels)) for _, e := range stream.Entries { - sp := pipeline.ForStream(mustParseLabels(stream.Labels)) if l, out, matches := sp.Process(e.Timestamp.UnixNano(), []byte(e.Line)); matches { var s *logproto.Stream var found bool @@ -137,12 +140,12 @@ func processStream(in []logproto.Stream, pipeline log.Pipeline) []logproto.Strea return streams } -func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Series { +func processSeries(in []logproto.Stream, ex log.SampleExtractor) ([]logproto.Series, error) { resBySeries := map[string]*logproto.Series{} for _, stream := range in { + exs := ex.ForStream(mustParseLabels(stream.Labels)) for _, e := range stream.Entries { - exs := ex.ForStream(mustParseLabels(stream.Labels)) if f, lbs, ok := exs.Process(e.Timestamp.UnixNano(), []byte(e.Line)); ok { var s *logproto.Series var found bool @@ -151,6 +154,7 @@ func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Seri s = &logproto.Series{Labels: lbs.String(), StreamHash: exs.BaseLabels().Hash()} resBySeries[lbs.String()] = s } + s.Samples = append(s.Samples, logproto.Sample{ Timestamp: e.Timestamp.UnixNano(), Value: f, @@ -159,12 +163,13 @@ func processSeries(in []logproto.Stream, ex log.SampleExtractor) []logproto.Seri } } } + series := []logproto.Series{} for _, s := range resBySeries { sort.Sort(s) series = append(series, *s) } - return series + return series, nil } func (q MockQuerier) SelectSamples(_ context.Context, req SelectSampleParams) (iter.SampleIterator, error) { @@ -212,7 +217,10 @@ outer: matched = append(matched, stream) } - filtered := processSeries(matched, extractor) + filtered, err := processSeries(matched, extractor) + if err != nil { + return nil, err + } return iter.NewTimeRangedSampleIterator( iter.NewMultiSeriesIterator(filtered), @@ -227,26 +235,23 @@ type MockDownstreamer struct { func (m MockDownstreamer) Downstreamer(_ context.Context) Downstreamer { return m } -func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery, _ Accumulator) ([]logqlmodel.Result, error) { - results := make([]logqlmodel.Result, 0, len(queries)) - for _, query := range queries { - res, err := m.Query(query.Params).Exec(ctx) +func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery, acc Accumulator) ([]logqlmodel.Result, error) { + mu := sync.Mutex{} + err := concurrency.ForEachJob(ctx, len(queries), ConCurrency, func(ctx context.Context, idx int) error { + res, err := m.Query(queries[idx].Params).Exec(ctx) if err != nil { - return nil, err + return err } - results = append(results, res) + mu.Lock() + defer mu.Unlock() + err = acc.Accumulate(ctx, res, idx) + return err + }) + if err != nil { + return nil, err } - if matrix, ok := results[0].Data.(ProbabilisticQuantileMatrix); ok { - if len(results) == 1 { - return results, nil - } - for _, m := range results[1:] { - matrix, _ = matrix.Merge(m.Data.(ProbabilisticQuantileMatrix)) - } - return []logqlmodel.Result{{Data: matrix}}, nil - } - return results, nil + return acc.Result(), nil } // create nStreams of nEntries with labelNames each where each label value diff --git a/pkg/querier/queryrange/extensions.go b/pkg/querier/queryrange/extensions.go index ec5fa25ae308e..de2763505a63c 100644 --- a/pkg/querier/queryrange/extensions.go +++ b/pkg/querier/queryrange/extensions.go @@ -221,6 +221,23 @@ func (m *QuantileSketchResponse) WithHeaders(h []queryrangebase.PrometheusRespon return m } +// GetHeaders returns the HTTP headers in the response. +func (m *CountMinSketchResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader { + if m != nil { + return convertPrometheusResponseHeadersToPointers(m.Headers) + } + return nil +} + +func (m *CountMinSketchResponse) SetHeader(name, value string) { + m.Headers = setHeader(m.Headers, name, value) +} + +func (m *CountMinSketchResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { + m.Headers = h + return m +} + func (m *ShardsResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader { if m != nil { return convertPrometheusResponseHeadersToPointers(m.Headers) diff --git a/pkg/querier/queryrange/limits/definitions.go b/pkg/querier/queryrange/limits/definitions.go index be366fdc10a44..e881db94766dc 100644 --- a/pkg/querier/queryrange/limits/definitions.go +++ b/pkg/querier/queryrange/limits/definitions.go @@ -36,4 +36,6 @@ type Limits interface { MaxStatsCacheFreshness(context.Context, string) time.Duration MaxMetadataCacheFreshness(context.Context, string) time.Duration VolumeEnabled(string) bool + + ShardAggregations(string) []string } diff --git a/pkg/querier/queryrange/marshal.go b/pkg/querier/queryrange/marshal.go index 7aa9f807bb3bf..7ac435a7eceea 100644 --- a/pkg/querier/queryrange/marshal.go +++ b/pkg/querier/queryrange/marshal.go @@ -136,6 +136,13 @@ func ResultToResponse(result logqlmodel.Result, params logql.Params) (queryrange Warnings: result.Warnings, Statistics: result.Statistics, }, nil + case logql.CountMinSketchVector: + r, err := data.ToProto() + return &CountMinSketchResponse{ + Response: r, + Warnings: result.Warnings, + Statistics: result.Statistics, + }, err } return nil, fmt.Errorf("unsupported data type: %T", result.Data) @@ -202,6 +209,17 @@ func ResponseToResult(resp queryrangebase.Response) (logqlmodel.Result, error) { Warnings: r.Warnings, Statistics: r.Statistics, }, nil + case *CountMinSketchResponse: + cms, err := logql.CountMinSketchVectorFromProto(r.Response) + if err != nil { + return logqlmodel.Result{}, fmt.Errorf("cannot decode count min sketch vector: %w", err) + } + return logqlmodel.Result{ + Data: cms, + Headers: resp.GetHeaders(), + Warnings: r.Warnings, + Statistics: r.Statistics, + }, nil default: return logqlmodel.Result{}, fmt.Errorf("cannot decode (%T)", resp) } @@ -237,6 +255,8 @@ func QueryResponseUnwrap(res *QueryResponse) (queryrangebase.Response, error) { return concrete.DetectedLabels, nil case *QueryResponse_DetectedFields: return concrete.DetectedFields, nil + case *QueryResponse_CountMinSketches: + return concrete.CountMinSketches, nil default: return nil, fmt.Errorf("unsupported QueryResponse response type, got (%T)", res.Response) } @@ -278,6 +298,8 @@ func QueryResponseWrap(res queryrangebase.Response) (*QueryResponse, error) { p.Response = &QueryResponse_DetectedLabels{response} case *DetectedFieldsResponse: p.Response = &QueryResponse_DetectedFields{response} + case *CountMinSketchResponse: + p.Response = &QueryResponse_CountMinSketches{response} default: return nil, fmt.Errorf("invalid response format, got (%T)", res) } diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index 82e929e683034..43dc2f50cd306 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -908,6 +908,59 @@ func (m *QuantileSketchResponse) GetStatistics() stats.Result { return stats.Result{} } +type CountMinSketchResponse struct { + Response *github_com_grafana_loki_v3_pkg_logproto.CountMinSketchVector `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/v3/pkg/logproto.CountMinSketchVector" json:"response,omitempty"` + Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"` + Warnings []string `protobuf:"bytes,3,rep,name=warnings,proto3" json:"warnings,omitempty"` + Statistics stats.Result `protobuf:"bytes,4,opt,name=statistics,proto3" json:"statistics"` +} + +func (m *CountMinSketchResponse) Reset() { *m = CountMinSketchResponse{} } +func (*CountMinSketchResponse) ProtoMessage() {} +func (*CountMinSketchResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_51b9d53b40d11902, []int{13} +} +func (m *CountMinSketchResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CountMinSketchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CountMinSketchResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *CountMinSketchResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CountMinSketchResponse.Merge(m, src) +} +func (m *CountMinSketchResponse) XXX_Size() int { + return m.Size() +} +func (m *CountMinSketchResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CountMinSketchResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CountMinSketchResponse proto.InternalMessageInfo + +func (m *CountMinSketchResponse) GetWarnings() []string { + if m != nil { + return m.Warnings + } + return nil +} + +func (m *CountMinSketchResponse) GetStatistics() stats.Result { + if m != nil { + return m.Statistics + } + return stats.Result{} +} + type ShardsResponse struct { Response *github_com_grafana_loki_v3_pkg_logproto.ShardsResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/v3/pkg/logproto.ShardsResponse" json:"response,omitempty"` Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"` @@ -916,7 +969,7 @@ type ShardsResponse struct { func (m *ShardsResponse) Reset() { *m = ShardsResponse{} } func (*ShardsResponse) ProtoMessage() {} func (*ShardsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{13} + return fileDescriptor_51b9d53b40d11902, []int{14} } func (m *ShardsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,7 +1006,7 @@ type DetectedFieldsResponse struct { func (m *DetectedFieldsResponse) Reset() { *m = DetectedFieldsResponse{} } func (*DetectedFieldsResponse) ProtoMessage() {} func (*DetectedFieldsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{14} + return fileDescriptor_51b9d53b40d11902, []int{15} } func (m *DetectedFieldsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -990,7 +1043,7 @@ type QueryPatternsResponse struct { func (m *QueryPatternsResponse) Reset() { *m = QueryPatternsResponse{} } func (*QueryPatternsResponse) ProtoMessage() {} func (*QueryPatternsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{15} + return fileDescriptor_51b9d53b40d11902, []int{16} } func (m *QueryPatternsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1027,7 +1080,7 @@ type DetectedLabelsResponse struct { func (m *DetectedLabelsResponse) Reset() { *m = DetectedLabelsResponse{} } func (*DetectedLabelsResponse) ProtoMessage() {} func (*DetectedLabelsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{16} + return fileDescriptor_51b9d53b40d11902, []int{17} } func (m *DetectedLabelsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1071,13 +1124,14 @@ type QueryResponse struct { // *QueryResponse_DetectedFields // *QueryResponse_PatternsResponse // *QueryResponse_DetectedLabels + // *QueryResponse_CountMinSketches Response isQueryResponse_Response `protobuf_oneof:"response"` } func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{17} + return fileDescriptor_51b9d53b40d11902, []int{18} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1149,6 +1203,9 @@ type QueryResponse_PatternsResponse struct { type QueryResponse_DetectedLabels struct { DetectedLabels *DetectedLabelsResponse `protobuf:"bytes,13,opt,name=detectedLabels,proto3,oneof"` } +type QueryResponse_CountMinSketches struct { + CountMinSketches *CountMinSketchResponse `protobuf:"bytes,14,opt,name=countMinSketches,proto3,oneof"` +} func (*QueryResponse_Series) isQueryResponse_Response() {} func (*QueryResponse_Labels) isQueryResponse_Response() {} @@ -1162,6 +1219,7 @@ func (*QueryResponse_ShardsResponse) isQueryResponse_Response() {} func (*QueryResponse_DetectedFields) isQueryResponse_Response() {} func (*QueryResponse_PatternsResponse) isQueryResponse_Response() {} func (*QueryResponse_DetectedLabels) isQueryResponse_Response() {} +func (*QueryResponse_CountMinSketches) isQueryResponse_Response() {} func (m *QueryResponse) GetResponse() isQueryResponse_Response { if m != nil { @@ -1261,6 +1319,13 @@ func (m *QueryResponse) GetDetectedLabels() *DetectedLabelsResponse { return nil } +func (m *QueryResponse) GetCountMinSketches() *CountMinSketchResponse { + if x, ok := m.GetResponse().(*QueryResponse_CountMinSketches); ok { + return x.CountMinSketches + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*QueryResponse) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -1276,6 +1341,7 @@ func (*QueryResponse) XXX_OneofWrappers() []interface{} { (*QueryResponse_DetectedFields)(nil), (*QueryResponse_PatternsResponse)(nil), (*QueryResponse_DetectedLabels)(nil), + (*QueryResponse_CountMinSketches)(nil), } } @@ -1298,7 +1364,7 @@ type QueryRequest struct { func (m *QueryRequest) Reset() { *m = QueryRequest{} } func (*QueryRequest) ProtoMessage() {} func (*QueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{18} + return fileDescriptor_51b9d53b40d11902, []int{19} } func (m *QueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,6 +1556,7 @@ func init() { proto.RegisterType((*VolumeResponse)(nil), "queryrange.VolumeResponse") proto.RegisterType((*TopKSketchesResponse)(nil), "queryrange.TopKSketchesResponse") proto.RegisterType((*QuantileSketchResponse)(nil), "queryrange.QuantileSketchResponse") + proto.RegisterType((*CountMinSketchResponse)(nil), "queryrange.CountMinSketchResponse") proto.RegisterType((*ShardsResponse)(nil), "queryrange.ShardsResponse") proto.RegisterType((*DetectedFieldsResponse)(nil), "queryrange.DetectedFieldsResponse") proto.RegisterType((*QueryPatternsResponse)(nil), "queryrange.QueryPatternsResponse") @@ -1504,129 +1571,132 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1944 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcb, 0x6f, 0x1b, 0xc7, - 0x19, 0xe7, 0xf2, 0x29, 0x7e, 0x7a, 0x58, 0x1d, 0xab, 0xca, 0x56, 0x71, 0xb8, 0x2c, 0x81, 0x26, - 0x6a, 0xd1, 0x92, 0x31, 0x95, 0xb8, 0x89, 0xea, 0x1a, 0xf1, 0x5a, 0x76, 0x65, 0xd7, 0x69, 0x9c, - 0x95, 0xd0, 0x43, 0x2f, 0xc5, 0x88, 0x1c, 0x91, 0x5b, 0x91, 0xbb, 0xeb, 0xdd, 0xa1, 0x6c, 0x01, - 0x45, 0x91, 0x7f, 0x20, 0x68, 0xee, 0xbd, 0x17, 0xbd, 0x15, 0x05, 0x7a, 0xea, 0xa9, 0xc7, 0xe4, - 0x50, 0xc0, 0xc7, 0x80, 0x40, 0xb7, 0x35, 0xdd, 0x43, 0xa1, 0x53, 0x80, 0xfe, 0x03, 0xc5, 0x3c, - 0x76, 0x39, 0x43, 0x52, 0x35, 0xa5, 0x16, 0x05, 0x14, 0xe4, 0x22, 0xcd, 0xe3, 0xfb, 0xcd, 0xce, - 0xfc, 0xbe, 0xdf, 0xf7, 0xcd, 0x83, 0xf0, 0x46, 0x70, 0xd4, 0x69, 0x3c, 0x1e, 0x90, 0xd0, 0x25, - 0x21, 0xff, 0x7f, 0x12, 0x62, 0xaf, 0x43, 0x94, 0x62, 0x3d, 0x08, 0x7d, 0xea, 0x23, 0x18, 0xb7, - 0x6c, 0x34, 0x3b, 0x2e, 0xed, 0x0e, 0x0e, 0xea, 0x2d, 0xbf, 0xdf, 0xe8, 0xf8, 0x1d, 0xbf, 0xd1, - 0xf1, 0xfd, 0x4e, 0x8f, 0xe0, 0xc0, 0x8d, 0x64, 0xb1, 0x11, 0x06, 0xad, 0x46, 0x44, 0x31, 0x1d, - 0x44, 0x02, 0xbf, 0xb1, 0xc6, 0x0c, 0x79, 0x91, 0x43, 0x64, 0xab, 0x25, 0xcd, 0x79, 0xed, 0x60, - 0x70, 0xd8, 0xa0, 0x6e, 0x9f, 0x44, 0x14, 0xf7, 0x83, 0xc4, 0x80, 0xcd, 0xaf, 0xe7, 0x77, 0x04, - 0xd2, 0xf5, 0xda, 0xe4, 0x69, 0x07, 0x53, 0xf2, 0x04, 0x9f, 0x48, 0x83, 0x57, 0x35, 0x83, 0xa4, - 0x20, 0x3b, 0x37, 0xb4, 0xce, 0x00, 0x53, 0x4a, 0x42, 0x4f, 0xf6, 0x7d, 0x43, 0xeb, 0x8b, 0x8e, - 0x08, 0x6d, 0x75, 0x65, 0x57, 0x55, 0x76, 0x3d, 0xee, 0xf5, 0xfd, 0x36, 0xe9, 0xf1, 0x85, 0x44, - 0xe2, 0xaf, 0xb4, 0xb8, 0xca, 0x2c, 0x82, 0x41, 0xd4, 0xe5, 0x7f, 0x64, 0xe3, 0x9d, 0x97, 0x72, - 0x79, 0x80, 0x23, 0xd2, 0x68, 0x93, 0x43, 0xd7, 0x73, 0xa9, 0xeb, 0x7b, 0x91, 0x5a, 0x96, 0x83, - 0xdc, 0x98, 0x6f, 0x90, 0x49, 0xff, 0x6c, 0xbc, 0xc9, 0x70, 0x11, 0xf5, 0x43, 0xdc, 0x21, 0x8d, - 0x56, 0x77, 0xe0, 0x1d, 0x35, 0x5a, 0xb8, 0xd5, 0x25, 0x8d, 0x90, 0x44, 0x83, 0x1e, 0x8d, 0x44, - 0x85, 0x9e, 0x04, 0x44, 0x7e, 0xa9, 0xf6, 0x59, 0x1e, 0x16, 0x1f, 0xfa, 0x47, 0xae, 0x43, 0x1e, - 0x0f, 0x48, 0x44, 0xd1, 0x1a, 0x14, 0xf8, 0xa8, 0xa6, 0x51, 0x35, 0x36, 0xcb, 0x8e, 0xa8, 0xb0, - 0xd6, 0x9e, 0xdb, 0x77, 0xa9, 0x99, 0xad, 0x1a, 0x9b, 0xcb, 0x8e, 0xa8, 0x20, 0x04, 0xf9, 0x88, - 0x92, 0xc0, 0xcc, 0x55, 0x8d, 0xcd, 0x9c, 0xc3, 0xcb, 0x68, 0x03, 0x16, 0x5c, 0x8f, 0x92, 0xf0, - 0x18, 0xf7, 0xcc, 0x32, 0x6f, 0x4f, 0xeb, 0xe8, 0x16, 0x94, 0x22, 0x8a, 0x43, 0xba, 0x1f, 0x99, - 0xf9, 0xaa, 0xb1, 0xb9, 0xd8, 0xdc, 0xa8, 0x0b, 0xcf, 0xd7, 0x13, 0xcf, 0xd7, 0xf7, 0x13, 0xcf, - 0xdb, 0x0b, 0x9f, 0xc6, 0x56, 0xe6, 0x93, 0xbf, 0x59, 0x86, 0x93, 0x80, 0xd0, 0x36, 0x14, 0x88, - 0xd7, 0xde, 0x8f, 0xcc, 0xc2, 0x39, 0xd0, 0x02, 0x82, 0xae, 0x43, 0xb9, 0xed, 0x86, 0xa4, 0xc5, - 0x58, 0x36, 0x8b, 0x55, 0x63, 0x73, 0xa5, 0x79, 0xb5, 0x9e, 0x0a, 0x65, 0x27, 0xe9, 0x72, 0xc6, - 0x56, 0x6c, 0x79, 0x01, 0xa6, 0x5d, 0xb3, 0xc4, 0x99, 0xe0, 0x65, 0x54, 0x83, 0x62, 0xd4, 0xc5, - 0x61, 0x3b, 0x32, 0x17, 0xaa, 0xb9, 0xcd, 0xb2, 0x0d, 0xa7, 0xb1, 0x25, 0x5b, 0x1c, 0xf9, 0x1f, - 0xfd, 0x1c, 0xf2, 0x41, 0x0f, 0x7b, 0x26, 0xf0, 0x59, 0xae, 0xd6, 0x15, 0x2f, 0x3d, 0xea, 0x61, - 0xcf, 0x7e, 0x77, 0x18, 0x5b, 0x6f, 0xab, 0xc1, 0x13, 0xe2, 0x43, 0xec, 0xe1, 0x46, 0xcf, 0x3f, - 0x72, 0x1b, 0xc7, 0x5b, 0x0d, 0xd5, 0xf7, 0x6c, 0xa0, 0xfa, 0x87, 0x6c, 0x00, 0x06, 0x75, 0xf8, - 0xc0, 0xe8, 0x01, 0x2c, 0x32, 0x1f, 0x93, 0x3b, 0xcc, 0xc1, 0x91, 0xb9, 0xc8, 0xbf, 0xf3, 0xca, - 0x78, 0x35, 0xbc, 0xdd, 0x21, 0x87, 0x3f, 0x0a, 0xfd, 0x41, 0x60, 0x5f, 0x39, 0x8d, 0x2d, 0xd5, - 0xde, 0x51, 0x2b, 0xe8, 0x01, 0xac, 0x30, 0x51, 0xb8, 0x5e, 0xe7, 0x83, 0x80, 0x2b, 0xd0, 0x5c, - 0xe2, 0xc3, 0x5d, 0xab, 0xab, 0x92, 0xa9, 0xdf, 0xd1, 0x6c, 0xec, 0x3c, 0xa3, 0xd7, 0x99, 0x40, - 0xd6, 0x46, 0x39, 0x40, 0x4c, 0x4b, 0xf7, 0xbd, 0x88, 0x62, 0x8f, 0x5e, 0x44, 0x52, 0x37, 0xa1, - 0xc8, 0x82, 0x7f, 0x3f, 0xe2, 0xa2, 0x9a, 0xd7, 0xc7, 0x12, 0xa3, 0x3b, 0x39, 0x7f, 0x2e, 0x27, - 0x17, 0x66, 0x3a, 0xb9, 0xf8, 0x52, 0x27, 0x97, 0xfe, 0x4f, 0x4e, 0x5e, 0xf8, 0xdf, 0x3a, 0xb9, - 0x7c, 0x61, 0x27, 0x9b, 0x90, 0x67, 0xb3, 0x44, 0xab, 0x90, 0x0b, 0xf1, 0x13, 0xee, 0xd3, 0x25, - 0x87, 0x15, 0x6b, 0xa3, 0x3c, 0x2c, 0x89, 0x54, 0x12, 0x05, 0xbe, 0x17, 0x11, 0xc6, 0xe3, 0x1e, - 0xcf, 0xfe, 0xc2, 0xf3, 0x92, 0x47, 0xde, 0xe2, 0xc8, 0x1e, 0xf4, 0x1e, 0xe4, 0x77, 0x30, 0xc5, - 0x5c, 0x05, 0x8b, 0xcd, 0x35, 0x95, 0x47, 0x36, 0x16, 0xeb, 0xb3, 0xd7, 0xd9, 0x44, 0x4e, 0x63, - 0x6b, 0xa5, 0x8d, 0x29, 0xfe, 0xae, 0xdf, 0x77, 0x29, 0xe9, 0x07, 0xf4, 0xc4, 0xe1, 0x48, 0xf4, - 0x36, 0x94, 0xef, 0x86, 0xa1, 0x1f, 0xee, 0x9f, 0x04, 0x84, 0xab, 0xa6, 0x6c, 0xbf, 0x72, 0x1a, - 0x5b, 0x57, 0x49, 0xd2, 0xa8, 0x20, 0xc6, 0x96, 0xe8, 0xdb, 0x50, 0xe0, 0x15, 0xae, 0x93, 0xb2, - 0x7d, 0xf5, 0x34, 0xb6, 0xae, 0x70, 0x88, 0x62, 0x2e, 0x2c, 0x74, 0x59, 0x15, 0xe6, 0x92, 0x55, - 0xaa, 0xee, 0xa2, 0xaa, 0x6e, 0x13, 0x4a, 0xc7, 0x24, 0x8c, 0xd8, 0x30, 0x25, 0xde, 0x9e, 0x54, - 0xd1, 0x6d, 0x00, 0x46, 0x8c, 0x1b, 0x51, 0xb7, 0x95, 0x38, 0x7b, 0xb9, 0x2e, 0x36, 0x1b, 0x87, - 0xfb, 0xc8, 0x46, 0x92, 0x05, 0xc5, 0xd0, 0x51, 0xca, 0xe8, 0xf7, 0x06, 0x94, 0x76, 0x09, 0x6e, - 0x93, 0x90, 0xb9, 0x37, 0xb7, 0xb9, 0xd8, 0xfc, 0x56, 0x5d, 0xdd, 0x59, 0x1e, 0x85, 0x7e, 0x9f, - 0xd0, 0x2e, 0x19, 0x44, 0x89, 0x83, 0x84, 0xb5, 0xed, 0x0d, 0x63, 0x8b, 0xcc, 0x29, 0xd5, 0xb9, - 0x36, 0xb4, 0x33, 0x3f, 0x75, 0x1a, 0x5b, 0xc6, 0xf7, 0x9c, 0x64, 0x96, 0xa8, 0x09, 0x0b, 0x4f, - 0x70, 0xe8, 0xb9, 0x5e, 0x27, 0x32, 0x81, 0x47, 0xda, 0xfa, 0x69, 0x6c, 0xa1, 0xa4, 0x4d, 0x71, - 0x44, 0x6a, 0x57, 0xfb, 0xab, 0x01, 0x5f, 0x63, 0xc2, 0xd8, 0x63, 0xf3, 0x89, 0x94, 0x14, 0xd3, - 0xc7, 0xb4, 0xd5, 0x35, 0x0d, 0x36, 0x8c, 0x23, 0x2a, 0xea, 0x7e, 0x93, 0xfd, 0xaf, 0xf6, 0x9b, - 0xdc, 0xf9, 0xf7, 0x9b, 0x24, 0xaf, 0xe4, 0x67, 0xe6, 0x95, 0xc2, 0x59, 0x79, 0xa5, 0xf6, 0x6b, - 0x99, 0x43, 0x93, 0xf5, 0x9d, 0x23, 0x94, 0xee, 0xa5, 0xa1, 0x94, 0xe3, 0xb3, 0x4d, 0x15, 0x2a, - 0xc6, 0xba, 0xdf, 0x26, 0x1e, 0x75, 0x0f, 0x5d, 0x12, 0xbe, 0x24, 0xa0, 0x14, 0x95, 0xe6, 0x74, - 0x95, 0xaa, 0x12, 0xcb, 0x5f, 0x0a, 0x89, 0xe9, 0x71, 0x55, 0xb8, 0x40, 0x5c, 0xd5, 0xfe, 0x95, - 0x85, 0x75, 0xe6, 0x91, 0x87, 0xf8, 0x80, 0xf4, 0x7e, 0x82, 0xfb, 0xe7, 0xf4, 0xca, 0xeb, 0x8a, - 0x57, 0xca, 0x36, 0xfa, 0x8a, 0xf5, 0xf9, 0x58, 0xff, 0xad, 0x01, 0x0b, 0xc9, 0x06, 0x80, 0xea, - 0x00, 0x02, 0xc6, 0x73, 0xbc, 0xe0, 0x7a, 0x85, 0x81, 0xc3, 0xb4, 0xd5, 0x51, 0x2c, 0xd0, 0x2f, - 0xa0, 0x28, 0x6a, 0x32, 0x16, 0x94, 0x6d, 0x73, 0x8f, 0x86, 0x04, 0xf7, 0x6f, 0xb7, 0x71, 0x40, - 0x49, 0x68, 0xbf, 0xcb, 0x66, 0x31, 0x8c, 0xad, 0x37, 0xce, 0x62, 0x29, 0x39, 0xe1, 0x4b, 0x1c, - 0xf3, 0xaf, 0xf8, 0xa6, 0x23, 0xbf, 0x50, 0xfb, 0xd8, 0x80, 0x55, 0x36, 0x51, 0x46, 0x4d, 0x2a, - 0x8c, 0x1d, 0x58, 0x08, 0x65, 0x99, 0x4f, 0x77, 0xb1, 0x59, 0xab, 0xeb, 0xb4, 0xce, 0xa0, 0x92, - 0x6f, 0xb8, 0x86, 0x93, 0x22, 0xd1, 0x96, 0x46, 0x63, 0x76, 0x16, 0x8d, 0x62, 0x8f, 0x56, 0x89, - 0xfb, 0x73, 0x16, 0xd0, 0x7d, 0x76, 0x43, 0x62, 0xfa, 0x1b, 0x4b, 0xf5, 0xe9, 0xd4, 0x8c, 0xae, - 0x8d, 0x49, 0x99, 0xb6, 0xb7, 0x6f, 0x0d, 0x63, 0x6b, 0xfb, 0x25, 0xda, 0xf9, 0x0f, 0x78, 0x65, - 0x15, 0xaa, 0x7c, 0xb3, 0x97, 0x41, 0xbe, 0xb5, 0x3f, 0x66, 0x61, 0xe5, 0xa7, 0x7e, 0x6f, 0xd0, - 0x27, 0x29, 0x7d, 0xc1, 0x14, 0x7d, 0xe6, 0x98, 0x3e, 0xdd, 0xd6, 0xde, 0x1e, 0xc6, 0xd6, 0x8d, - 0x79, 0xa9, 0xd3, 0xb1, 0x97, 0x9a, 0xb6, 0xdf, 0xe4, 0x60, 0x6d, 0xdf, 0x0f, 0x7e, 0xbc, 0xc7, - 0x6f, 0xd1, 0x4a, 0x9a, 0xec, 0x4e, 0x91, 0xb7, 0x36, 0x26, 0x8f, 0x21, 0xde, 0xc7, 0x34, 0x74, - 0x9f, 0xda, 0x37, 0x86, 0xb1, 0xd5, 0x9c, 0x97, 0xb8, 0x31, 0xee, 0x32, 0x93, 0xa6, 0x9d, 0x81, - 0x72, 0xf3, 0x9d, 0x81, 0x26, 0xf2, 0x42, 0x7e, 0xbe, 0xbc, 0xf0, 0x87, 0x1c, 0xac, 0x7f, 0x38, - 0xc0, 0x1e, 0x75, 0x7b, 0x44, 0x78, 0x28, 0xf5, 0xcf, 0x2f, 0xa7, 0xfc, 0x53, 0x19, 0xfb, 0x47, - 0xc7, 0x48, 0x4f, 0xbd, 0x37, 0x8c, 0xad, 0x9b, 0xf3, 0x7a, 0x6a, 0xd6, 0x08, 0x5f, 0xf9, 0x6c, - 0x1e, 0x9f, 0xfd, 0x29, 0x0b, 0x2b, 0x7b, 0xe2, 0x7c, 0x98, 0xb0, 0x75, 0x3c, 0xc3, 0x57, 0xea, - 0x83, 0x58, 0x70, 0x50, 0xd7, 0x11, 0xe7, 0x4b, 0x47, 0x3a, 0xf6, 0x52, 0xa7, 0xa3, 0xbf, 0x64, - 0x61, 0x7d, 0x87, 0x50, 0xd2, 0xa2, 0xa4, 0x7d, 0xcf, 0x25, 0x3d, 0x85, 0xc4, 0x8f, 0x8c, 0x29, - 0x16, 0xab, 0xca, 0x85, 0x6e, 0x26, 0xc8, 0xb6, 0x87, 0xb1, 0x75, 0x6b, 0x5e, 0x1e, 0x67, 0x8f, - 0x71, 0xa9, 0xf9, 0xfc, 0x2c, 0x0b, 0x5f, 0x17, 0x8f, 0x14, 0xe2, 0x05, 0x75, 0x4c, 0xe7, 0xaf, - 0xa6, 0xd8, 0xb4, 0xd4, 0xfc, 0x31, 0x03, 0x62, 0xdf, 0x1e, 0xc6, 0xd6, 0x0f, 0xe7, 0x4f, 0x20, - 0x33, 0x86, 0xf8, 0xd2, 0x68, 0x93, 0xdf, 0x2b, 0xce, 0xab, 0x4d, 0x1d, 0x74, 0x31, 0x6d, 0xea, - 0x63, 0x5c, 0x6a, 0x3e, 0xff, 0x51, 0x84, 0x65, 0xae, 0x92, 0x94, 0xc6, 0xef, 0x80, 0xbc, 0x88, - 0x49, 0x0e, 0x51, 0x72, 0x79, 0x0f, 0x83, 0x56, 0x7d, 0x4f, 0x5e, 0xd1, 0x84, 0x05, 0x7a, 0x07, - 0x8a, 0x11, 0xbf, 0x22, 0xcb, 0x33, 0x76, 0x65, 0xf2, 0x15, 0x4a, 0xbf, 0x8c, 0xef, 0x66, 0x1c, - 0x69, 0x8f, 0x6e, 0x42, 0xb1, 0xc7, 0x59, 0x94, 0x4f, 0x04, 0xb5, 0x49, 0xe4, 0xf4, 0xa5, 0x91, - 0xa1, 0x05, 0x06, 0xdd, 0x80, 0x02, 0xdf, 0x00, 0xe4, 0x76, 0xa0, 0x7d, 0x76, 0xfa, 0x48, 0xbd, - 0x9b, 0x71, 0x84, 0x39, 0x6a, 0x42, 0x3e, 0x08, 0xfd, 0xbe, 0xbc, 0x58, 0x5d, 0x9b, 0xfc, 0xa6, - 0x7a, 0x13, 0xd9, 0xcd, 0x38, 0xdc, 0x16, 0xbd, 0x05, 0xa5, 0x88, 0x5f, 0x61, 0x22, 0xfe, 0x24, - 0xc5, 0xce, 0xaf, 0x13, 0x30, 0x05, 0x92, 0x98, 0xa2, 0xb7, 0xa0, 0x78, 0xcc, 0x0f, 0xa8, 0xf2, - 0x9d, 0x73, 0x43, 0x05, 0xe9, 0x47, 0x57, 0xb6, 0x2e, 0x61, 0x8b, 0xee, 0xc1, 0x12, 0xf5, 0x83, - 0xa3, 0xe4, 0x1c, 0x28, 0x9f, 0xb3, 0xaa, 0x2a, 0x76, 0xd6, 0x39, 0x71, 0x37, 0xe3, 0x68, 0x38, - 0xf4, 0x08, 0x56, 0x1f, 0x6b, 0x67, 0x07, 0x92, 0x3c, 0x5c, 0x6a, 0x3c, 0xcf, 0x3e, 0xd5, 0xec, - 0x66, 0x9c, 0x29, 0x34, 0xda, 0x81, 0x95, 0x48, 0xdb, 0xe1, 0xe4, 0x23, 0xbd, 0xb6, 0x2e, 0x7d, - 0x0f, 0xdc, 0xcd, 0x38, 0x13, 0x18, 0xf4, 0x10, 0x56, 0xda, 0x5a, 0x7e, 0x97, 0x4f, 0xf0, 0xda, - 0xac, 0x66, 0xef, 0x00, 0x6c, 0x34, 0x1d, 0x8b, 0x3e, 0x80, 0xd5, 0x60, 0x22, 0xb7, 0xc9, 0x37, - 0xf8, 0x6f, 0xea, 0xab, 0x9c, 0x91, 0x04, 0xd9, 0x22, 0x27, 0xc1, 0xea, 0xf4, 0x44, 0x88, 0x9b, - 0xcb, 0x67, 0x4f, 0x4f, 0x4f, 0x02, 0xea, 0xf4, 0x44, 0x8f, 0x0d, 0xe3, 0x74, 0x54, 0xfb, 0xb8, - 0x08, 0x4b, 0x32, 0xcc, 0xc4, 0xbb, 0xdb, 0xf7, 0xd3, 0xc8, 0x11, 0x51, 0xf6, 0xda, 0x59, 0x91, - 0xc3, 0xcd, 0x95, 0xc0, 0x79, 0x33, 0x0d, 0x1c, 0x11, 0x72, 0xeb, 0xe3, 0x14, 0xc7, 0xbf, 0xab, - 0x20, 0x64, 0xb0, 0x6c, 0x25, 0xc1, 0x22, 0x22, 0xed, 0xd5, 0xd9, 0xb7, 0xd7, 0x04, 0x25, 0x23, - 0x65, 0x1b, 0x4a, 0xae, 0xf8, 0x31, 0x62, 0x56, 0x8c, 0x4d, 0xff, 0x56, 0xc1, 0xb4, 0x2f, 0x01, - 0x68, 0x6b, 0x1c, 0x31, 0x05, 0xf9, 0xf8, 0x3e, 0x15, 0x31, 0x29, 0x28, 0x09, 0x98, 0xeb, 0x69, - 0xc0, 0x14, 0x27, 0x1f, 0xec, 0x93, 0x70, 0x49, 0x17, 0x26, 0xa3, 0xe5, 0x2e, 0x2c, 0x27, 0xfa, - 0xe2, 0x5d, 0x32, 0x5c, 0x5e, 0x3b, 0xeb, 0x58, 0x97, 0xe0, 0x75, 0x14, 0xba, 0x3f, 0x25, 0xca, - 0xf2, 0xe4, 0x56, 0x3c, 0x29, 0xc9, 0x64, 0xa4, 0x49, 0x45, 0x3e, 0x80, 0x2b, 0x63, 0x51, 0x89, - 0x39, 0xc1, 0xf4, 0xb5, 0x40, 0x93, 0x63, 0x32, 0xd4, 0x24, 0x50, 0x9d, 0x96, 0x14, 0xe3, 0xe2, - 0x59, 0xd3, 0x4a, 0xa4, 0x38, 0x35, 0x2d, 0xd1, 0x81, 0x76, 0x61, 0xa1, 0x4f, 0x28, 0x6e, 0x63, - 0x8a, 0xcd, 0x12, 0xdf, 0x96, 0x5e, 0x9f, 0x0a, 0x10, 0x89, 0xae, 0xbf, 0x2f, 0x0d, 0xef, 0x7a, - 0x34, 0x3c, 0x91, 0x27, 0xeb, 0x14, 0xbd, 0xf1, 0x03, 0x58, 0xd6, 0x0c, 0xd0, 0x2a, 0xe4, 0x8e, - 0x48, 0xf2, 0x03, 0x15, 0x2b, 0xa2, 0x35, 0x28, 0x1c, 0xe3, 0xde, 0x80, 0x70, 0x7d, 0x96, 0x1d, - 0x51, 0xd9, 0xce, 0xbe, 0x63, 0xd8, 0x65, 0x28, 0x85, 0xe2, 0x2b, 0x76, 0xe7, 0xd9, 0xf3, 0x4a, - 0xe6, 0xf3, 0xe7, 0x95, 0xcc, 0x17, 0xcf, 0x2b, 0xc6, 0x47, 0xa3, 0x8a, 0xf1, 0xbb, 0x51, 0xc5, - 0xf8, 0x74, 0x54, 0x31, 0x9e, 0x8d, 0x2a, 0xc6, 0xdf, 0x47, 0x15, 0xe3, 0x9f, 0xa3, 0x4a, 0xe6, - 0x8b, 0x51, 0xc5, 0xf8, 0xe4, 0x45, 0x25, 0xf3, 0xec, 0x45, 0x25, 0xf3, 0xf9, 0x8b, 0x4a, 0xe6, - 0x67, 0xd7, 0xcf, 0xbd, 0x43, 0x1e, 0x14, 0x39, 0x53, 0x5b, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, - 0x6b, 0xd4, 0xf8, 0x77, 0xa9, 0x1f, 0x00, 0x00, + // 1991 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x6f, 0x1b, 0xc7, + 0x15, 0xe6, 0xf2, 0xa7, 0x38, 0x92, 0x68, 0x75, 0xac, 0x2a, 0x5b, 0xc5, 0xe1, 0xb2, 0x04, 0x9a, + 0xa8, 0x45, 0x4b, 0xc6, 0x54, 0xe2, 0x26, 0xaa, 0x6b, 0xc4, 0x6b, 0xd9, 0x95, 0x5d, 0xbb, 0x71, + 0x56, 0x42, 0x0e, 0xbd, 0x14, 0x23, 0x72, 0x44, 0x6e, 0x45, 0xee, 0xae, 0x77, 0x86, 0xb2, 0x05, + 0x14, 0x45, 0xfe, 0x81, 0xa0, 0xb9, 0xf7, 0x5e, 0xf4, 0x56, 0x14, 0x28, 0x7a, 0xe8, 0xa9, 0xc7, + 0xe4, 0x50, 0xc0, 0xc7, 0x80, 0x40, 0xd9, 0x9a, 0xbe, 0x14, 0x3a, 0x05, 0xe8, 0x3f, 0x50, 0xcc, + 0x8f, 0x5d, 0xce, 0x70, 0x57, 0x35, 0xe9, 0x16, 0x05, 0x54, 0xf8, 0x42, 0xee, 0xcc, 0xbc, 0x6f, + 0x76, 0xe6, 0x7b, 0xdf, 0x9b, 0x37, 0x33, 0x0b, 0xde, 0x0a, 0x8e, 0xbb, 0xcd, 0x47, 0x43, 0x1c, + 0xba, 0x38, 0xe4, 0xff, 0xa7, 0x21, 0xf2, 0xba, 0x58, 0x79, 0x6c, 0x04, 0xa1, 0x4f, 0x7d, 0x08, + 0xa6, 0x35, 0x9b, 0xad, 0xae, 0x4b, 0x7b, 0xc3, 0xc3, 0x46, 0xdb, 0x1f, 0x34, 0xbb, 0x7e, 0xd7, + 0x6f, 0x76, 0x7d, 0xbf, 0xdb, 0xc7, 0x28, 0x70, 0x89, 0x7c, 0x6c, 0x86, 0x41, 0xbb, 0x49, 0x28, + 0xa2, 0x43, 0x22, 0xf0, 0x9b, 0xeb, 0xcc, 0x90, 0x3f, 0x72, 0x88, 0xac, 0xb5, 0xa4, 0x39, 0x2f, + 0x1d, 0x0e, 0x8f, 0x9a, 0xd4, 0x1d, 0x60, 0x42, 0xd1, 0x20, 0x88, 0x0c, 0xd8, 0xf8, 0xfa, 0x7e, + 0x57, 0x20, 0x5d, 0xaf, 0x83, 0x9f, 0x74, 0x11, 0xc5, 0x8f, 0xd1, 0xa9, 0x34, 0x78, 0x5d, 0x33, + 0x88, 0x1e, 0x64, 0xe3, 0xa6, 0xd6, 0x18, 0x20, 0x4a, 0x71, 0xe8, 0xc9, 0xb6, 0x6f, 0x68, 0x6d, + 0xe4, 0x18, 0xd3, 0x76, 0x4f, 0x36, 0xd5, 0x64, 0xd3, 0xa3, 0xfe, 0xc0, 0xef, 0xe0, 0x3e, 0x9f, + 0x08, 0x11, 0xbf, 0xd2, 0xe2, 0x32, 0xb3, 0x08, 0x86, 0xa4, 0xc7, 0x7f, 0x64, 0xe5, 0xad, 0x17, + 0x72, 0x79, 0x88, 0x08, 0x6e, 0x76, 0xf0, 0x91, 0xeb, 0xb9, 0xd4, 0xf5, 0x3d, 0xa2, 0x3e, 0xcb, + 0x4e, 0xae, 0xcd, 0xd7, 0xc9, 0xac, 0x7f, 0x36, 0xdf, 0x66, 0x38, 0x42, 0xfd, 0x10, 0x75, 0x71, + 0xb3, 0xdd, 0x1b, 0x7a, 0xc7, 0xcd, 0x36, 0x6a, 0xf7, 0x70, 0x33, 0xc4, 0x64, 0xd8, 0xa7, 0x44, + 0x14, 0xe8, 0x69, 0x80, 0xe5, 0x9b, 0xea, 0x5f, 0xe4, 0xc1, 0xf2, 0x7d, 0xff, 0xd8, 0x75, 0xf0, + 0xa3, 0x21, 0x26, 0x14, 0xae, 0x83, 0x02, 0xef, 0xd5, 0x34, 0x6a, 0xc6, 0x56, 0xd9, 0x11, 0x05, + 0x56, 0xdb, 0x77, 0x07, 0x2e, 0x35, 0xb3, 0x35, 0x63, 0x6b, 0xd5, 0x11, 0x05, 0x08, 0x41, 0x9e, + 0x50, 0x1c, 0x98, 0xb9, 0x9a, 0xb1, 0x95, 0x73, 0xf8, 0x33, 0xdc, 0x04, 0x4b, 0xae, 0x47, 0x71, + 0x78, 0x82, 0xfa, 0x66, 0x99, 0xd7, 0xc7, 0x65, 0x78, 0x03, 0x94, 0x08, 0x45, 0x21, 0x3d, 0x20, + 0x66, 0xbe, 0x66, 0x6c, 0x2d, 0xb7, 0x36, 0x1b, 0xc2, 0xf3, 0x8d, 0xc8, 0xf3, 0x8d, 0x83, 0xc8, + 0xf3, 0xf6, 0xd2, 0xe7, 0x63, 0x2b, 0xf3, 0xd9, 0xdf, 0x2c, 0xc3, 0x89, 0x40, 0x70, 0x07, 0x14, + 0xb0, 0xd7, 0x39, 0x20, 0x66, 0x61, 0x01, 0xb4, 0x80, 0xc0, 0xab, 0xa0, 0xdc, 0x71, 0x43, 0xdc, + 0x66, 0x2c, 0x9b, 0xc5, 0x9a, 0xb1, 0x55, 0x69, 0x5d, 0x6e, 0xc4, 0x42, 0xd9, 0x8d, 0x9a, 0x9c, + 0xa9, 0x15, 0x9b, 0x5e, 0x80, 0x68, 0xcf, 0x2c, 0x71, 0x26, 0xf8, 0x33, 0xac, 0x83, 0x22, 0xe9, + 0xa1, 0xb0, 0x43, 0xcc, 0xa5, 0x5a, 0x6e, 0xab, 0x6c, 0x83, 0xb3, 0xb1, 0x25, 0x6b, 0x1c, 0xf9, + 0x0f, 0x7f, 0x06, 0xf2, 0x41, 0x1f, 0x79, 0x26, 0xe0, 0xa3, 0x5c, 0x6b, 0x28, 0x5e, 0x7a, 0xd8, + 0x47, 0x9e, 0xfd, 0xfe, 0x68, 0x6c, 0xbd, 0xab, 0x06, 0x4f, 0x88, 0x8e, 0x90, 0x87, 0x9a, 0x7d, + 0xff, 0xd8, 0x6d, 0x9e, 0x6c, 0x37, 0x55, 0xdf, 0xb3, 0x8e, 0x1a, 0x1f, 0xb1, 0x0e, 0x18, 0xd4, + 0xe1, 0x1d, 0xc3, 0x7b, 0x60, 0x99, 0xf9, 0x18, 0xdf, 0x62, 0x0e, 0x26, 0xe6, 0x32, 0x7f, 0xcf, + 0x6b, 0xd3, 0xd9, 0xf0, 0x7a, 0x07, 0x1f, 0xfd, 0x28, 0xf4, 0x87, 0x81, 0x7d, 0xe9, 0x6c, 0x6c, + 0xa9, 0xf6, 0x8e, 0x5a, 0x80, 0xf7, 0x40, 0x85, 0x89, 0xc2, 0xf5, 0xba, 0x1f, 0x06, 0x5c, 0x81, + 0xe6, 0x0a, 0xef, 0xee, 0x4a, 0x43, 0x95, 0x4c, 0xe3, 0x96, 0x66, 0x63, 0xe7, 0x19, 0xbd, 0xce, + 0x0c, 0xb2, 0x3e, 0xc9, 0x01, 0xc8, 0xb4, 0x74, 0xd7, 0x23, 0x14, 0x79, 0xf4, 0x65, 0x24, 0x75, + 0x1d, 0x14, 0x59, 0xf0, 0x1f, 0x10, 0x2e, 0xaa, 0x79, 0x7d, 0x2c, 0x31, 0xba, 0x93, 0xf3, 0x0b, + 0x39, 0xb9, 0x90, 0xea, 0xe4, 0xe2, 0x0b, 0x9d, 0x5c, 0xfa, 0x1f, 0x39, 0x79, 0xe9, 0xbf, 0xeb, + 0xe4, 0xf2, 0x4b, 0x3b, 0xd9, 0x04, 0x79, 0x36, 0x4a, 0xb8, 0x06, 0x72, 0x21, 0x7a, 0xcc, 0x7d, + 0xba, 0xe2, 0xb0, 0xc7, 0xfa, 0x24, 0x0f, 0x56, 0xc4, 0x52, 0x42, 0x02, 0xdf, 0x23, 0x98, 0xf1, + 0xb8, 0xcf, 0x57, 0x7f, 0xe1, 0x79, 0xc9, 0x23, 0xaf, 0x71, 0x64, 0x0b, 0xfc, 0x00, 0xe4, 0x77, + 0x11, 0x45, 0x5c, 0x05, 0xcb, 0xad, 0x75, 0x95, 0x47, 0xd6, 0x17, 0x6b, 0xb3, 0x37, 0xd8, 0x40, + 0xce, 0xc6, 0x56, 0xa5, 0x83, 0x28, 0xfa, 0xae, 0x3f, 0x70, 0x29, 0x1e, 0x04, 0xf4, 0xd4, 0xe1, + 0x48, 0xf8, 0x2e, 0x28, 0xdf, 0x0e, 0x43, 0x3f, 0x3c, 0x38, 0x0d, 0x30, 0x57, 0x4d, 0xd9, 0x7e, + 0xed, 0x6c, 0x6c, 0x5d, 0xc6, 0x51, 0xa5, 0x82, 0x98, 0x5a, 0xc2, 0x6f, 0x83, 0x02, 0x2f, 0x70, + 0x9d, 0x94, 0xed, 0xcb, 0x67, 0x63, 0xeb, 0x12, 0x87, 0x28, 0xe6, 0xc2, 0x42, 0x97, 0x55, 0x61, + 0x2e, 0x59, 0xc5, 0xea, 0x2e, 0xaa, 0xea, 0x36, 0x41, 0xe9, 0x04, 0x87, 0x84, 0x75, 0x53, 0xe2, + 0xf5, 0x51, 0x11, 0xde, 0x04, 0x80, 0x11, 0xe3, 0x12, 0xea, 0xb6, 0x23, 0x67, 0xaf, 0x36, 0x44, + 0xb2, 0x71, 0xb8, 0x8f, 0x6c, 0x28, 0x59, 0x50, 0x0c, 0x1d, 0xe5, 0x19, 0xfe, 0xce, 0x00, 0xa5, + 0x3d, 0x8c, 0x3a, 0x38, 0x64, 0xee, 0xcd, 0x6d, 0x2d, 0xb7, 0xbe, 0xd5, 0x50, 0x33, 0xcb, 0xc3, + 0xd0, 0x1f, 0x60, 0xda, 0xc3, 0x43, 0x12, 0x39, 0x48, 0x58, 0xdb, 0xde, 0x68, 0x6c, 0xe1, 0x39, + 0xa5, 0x3a, 0x57, 0x42, 0x3b, 0xf7, 0x55, 0x67, 0x63, 0xcb, 0xf8, 0x9e, 0x13, 0x8d, 0x12, 0xb6, + 0xc0, 0xd2, 0x63, 0x14, 0x7a, 0xae, 0xd7, 0x25, 0x26, 0xe0, 0x91, 0xb6, 0x71, 0x36, 0xb6, 0x60, + 0x54, 0xa7, 0x38, 0x22, 0xb6, 0xab, 0xff, 0xd5, 0x00, 0x5f, 0x63, 0xc2, 0xd8, 0x67, 0xe3, 0x21, + 0xca, 0x12, 0x33, 0x40, 0xb4, 0xdd, 0x33, 0x0d, 0xd6, 0x8d, 0x23, 0x0a, 0x6a, 0xbe, 0xc9, 0xfe, + 0x47, 0xf9, 0x26, 0xb7, 0x78, 0xbe, 0x89, 0xd6, 0x95, 0x7c, 0xea, 0xba, 0x52, 0x38, 0x6f, 0x5d, + 0xa9, 0xff, 0x4a, 0xae, 0xa1, 0xd1, 0xfc, 0x16, 0x08, 0xa5, 0x3b, 0x71, 0x28, 0xe5, 0xf8, 0x68, + 0x63, 0x85, 0x8a, 0xbe, 0xee, 0x76, 0xb0, 0x47, 0xdd, 0x23, 0x17, 0x87, 0x2f, 0x08, 0x28, 0x45, + 0xa5, 0x39, 0x5d, 0xa5, 0xaa, 0xc4, 0xf2, 0x17, 0x42, 0x62, 0x7a, 0x5c, 0x15, 0x5e, 0x22, 0xae, + 0xea, 0xff, 0xcc, 0x82, 0x0d, 0xe6, 0x91, 0xfb, 0xe8, 0x10, 0xf7, 0x7f, 0x82, 0x06, 0x0b, 0x7a, + 0xe5, 0x4d, 0xc5, 0x2b, 0x65, 0x1b, 0xbe, 0x62, 0x7d, 0x3e, 0xd6, 0x7f, 0x63, 0x80, 0xa5, 0x28, + 0x01, 0xc0, 0x06, 0x00, 0x02, 0xc6, 0xd7, 0x78, 0xc1, 0x75, 0x85, 0x81, 0xc3, 0xb8, 0xd6, 0x51, + 0x2c, 0xe0, 0xcf, 0x41, 0x51, 0x94, 0x64, 0x2c, 0x28, 0x69, 0x73, 0x9f, 0x86, 0x18, 0x0d, 0x6e, + 0x76, 0x50, 0x40, 0x71, 0x68, 0xbf, 0xcf, 0x46, 0x31, 0x1a, 0x5b, 0x6f, 0x9d, 0xc7, 0x52, 0xb4, + 0xc3, 0x97, 0x38, 0xe6, 0x5f, 0xf1, 0x4e, 0x47, 0xbe, 0xa1, 0xfe, 0xa9, 0x01, 0xd6, 0xd8, 0x40, + 0x19, 0x35, 0xb1, 0x30, 0x76, 0xc1, 0x52, 0x28, 0x9f, 0xf9, 0x70, 0x97, 0x5b, 0xf5, 0x86, 0x4e, + 0x6b, 0x0a, 0x95, 0x3c, 0xe1, 0x1a, 0x4e, 0x8c, 0x84, 0xdb, 0x1a, 0x8d, 0xd9, 0x34, 0x1a, 0x45, + 0x8e, 0x56, 0x89, 0xfb, 0x73, 0x16, 0xc0, 0xbb, 0xec, 0x84, 0xc4, 0xf4, 0x37, 0x95, 0xea, 0x93, + 0xc4, 0x88, 0xae, 0x4c, 0x49, 0x49, 0xda, 0xdb, 0x37, 0x46, 0x63, 0x6b, 0xe7, 0x05, 0xda, 0xf9, + 0x37, 0x78, 0x65, 0x16, 0xaa, 0x7c, 0xb3, 0x17, 0x41, 0xbe, 0xf5, 0x3f, 0x64, 0x41, 0xe5, 0x63, + 0xbf, 0x3f, 0x1c, 0xe0, 0x98, 0xbe, 0x20, 0x41, 0x9f, 0x39, 0xa5, 0x4f, 0xb7, 0xb5, 0x77, 0x46, + 0x63, 0xeb, 0xda, 0xbc, 0xd4, 0xe9, 0xd8, 0x0b, 0x4d, 0xdb, 0xaf, 0x73, 0x60, 0xfd, 0xc0, 0x0f, + 0x7e, 0xbc, 0xcf, 0x4f, 0xd1, 0xca, 0x32, 0xd9, 0x4b, 0x90, 0xb7, 0x3e, 0x25, 0x8f, 0x21, 0x1e, + 0x20, 0x1a, 0xba, 0x4f, 0xec, 0x6b, 0xa3, 0xb1, 0xd5, 0x9a, 0x97, 0xb8, 0x29, 0xee, 0x22, 0x93, + 0xa6, 0xed, 0x81, 0x72, 0xf3, 0xed, 0x81, 0x66, 0xd6, 0x85, 0xfc, 0x7c, 0xeb, 0xc2, 0xef, 0x73, + 0x60, 0xe3, 0xa3, 0x21, 0xf2, 0xa8, 0xdb, 0xc7, 0xc2, 0x43, 0xb1, 0x7f, 0x7e, 0x91, 0xf0, 0x4f, + 0x75, 0xea, 0x1f, 0x1d, 0x23, 0x3d, 0xf5, 0xc1, 0x68, 0x6c, 0x5d, 0x9f, 0xd7, 0x53, 0x69, 0x3d, + 0xbc, 0xf2, 0xd9, 0xbc, 0x3e, 0xbb, 0xe5, 0x0f, 0x3d, 0xfa, 0xc0, 0xf5, 0x16, 0xf1, 0x99, 0x8e, + 0xf9, 0x18, 0xb7, 0xa9, 0x1f, 0x2e, 0xe6, 0xb3, 0xb4, 0x1e, 0x5e, 0xf9, 0x6c, 0x1e, 0x9f, 0xfd, + 0x29, 0x0b, 0x2a, 0xfb, 0x62, 0x4f, 0x1f, 0xb1, 0x75, 0x92, 0xe2, 0x2b, 0xf5, 0x12, 0x33, 0x38, + 0x6c, 0xe8, 0x88, 0xc5, 0x52, 0x88, 0x8e, 0xbd, 0xd0, 0x29, 0xe4, 0x2f, 0x59, 0xb0, 0xb1, 0x8b, + 0x29, 0x6e, 0x53, 0xdc, 0xb9, 0xe3, 0xe2, 0xbe, 0x42, 0xe2, 0x27, 0x46, 0x82, 0xc5, 0x9a, 0x72, + 0x08, 0x4f, 0x05, 0xd9, 0xf6, 0x68, 0x6c, 0xdd, 0x98, 0x97, 0xc7, 0xf4, 0x3e, 0x2e, 0x34, 0x9f, + 0x5f, 0x64, 0xc1, 0xd7, 0xc5, 0xc5, 0x92, 0xb8, 0xf5, 0x9e, 0xd2, 0xf9, 0xcb, 0x04, 0x9b, 0x96, + 0xba, 0xe6, 0xa7, 0x40, 0xec, 0x9b, 0xa3, 0xb1, 0xf5, 0xc3, 0xf9, 0x17, 0xfd, 0x94, 0x2e, 0xfe, + 0x6f, 0xb4, 0xc9, 0xcf, 0x82, 0x8b, 0x6a, 0x53, 0x07, 0xbd, 0x9c, 0x36, 0xf5, 0x3e, 0x2e, 0x34, + 0x9f, 0x7f, 0x2c, 0x81, 0x55, 0xae, 0x92, 0x98, 0xc6, 0xef, 0x00, 0x79, 0x78, 0x96, 0x1c, 0xc2, + 0xe8, 0xc2, 0x25, 0x0c, 0xda, 0x8d, 0x7d, 0x79, 0xac, 0x16, 0x16, 0xf0, 0x3d, 0x50, 0x24, 0xfc, + 0x5a, 0x43, 0x9e, 0x8b, 0xaa, 0xb3, 0x37, 0x87, 0xfa, 0x05, 0xca, 0x5e, 0xc6, 0x91, 0xf6, 0xf0, + 0x3a, 0x28, 0xf6, 0x39, 0x8b, 0xf2, 0x5a, 0xa7, 0x3e, 0x8b, 0x4c, 0x1e, 0xf4, 0x19, 0x5a, 0x60, + 0xe0, 0x35, 0x50, 0xe0, 0x09, 0x40, 0xa6, 0x03, 0xed, 0xb5, 0xc9, 0x63, 0xd0, 0x5e, 0xc6, 0x11, + 0xe6, 0xb0, 0x05, 0xf2, 0x41, 0xe8, 0x0f, 0xe4, 0x61, 0xf8, 0xca, 0xec, 0x3b, 0xd5, 0xd3, 0xe3, + 0x5e, 0xc6, 0xe1, 0xb6, 0xf0, 0x1d, 0x50, 0x22, 0xfc, 0xd8, 0x49, 0xf8, 0x35, 0x22, 0x3b, 0x73, + 0xcc, 0xc0, 0x14, 0x48, 0x64, 0x0a, 0xdf, 0x01, 0xc5, 0x13, 0x7e, 0xa8, 0x90, 0x77, 0xd3, 0x9b, + 0x2a, 0x48, 0x3f, 0x6e, 0xb0, 0x79, 0x09, 0x5b, 0x78, 0x07, 0xac, 0x50, 0x3f, 0x38, 0x8e, 0xf6, + 0xee, 0xf2, 0x0a, 0xb2, 0xa6, 0x62, 0xd3, 0xf6, 0xf6, 0x7b, 0x19, 0x47, 0xc3, 0xc1, 0x87, 0x60, + 0xed, 0x91, 0xb6, 0xdf, 0xc3, 0xd1, 0x65, 0xb3, 0xc6, 0x73, 0xfa, 0x4e, 0x74, 0x2f, 0xe3, 0x24, + 0xd0, 0x70, 0x17, 0x54, 0x88, 0x96, 0xe1, 0xe4, 0x87, 0x15, 0x6d, 0x5e, 0x7a, 0x0e, 0xdc, 0xcb, + 0x38, 0x33, 0x18, 0x78, 0x1f, 0x54, 0x3a, 0xda, 0xfa, 0x2e, 0x3f, 0x9b, 0x68, 0xa3, 0x4a, 0xcf, + 0x00, 0xac, 0x37, 0x1d, 0x0b, 0x3f, 0x04, 0x6b, 0xc1, 0xcc, 0xda, 0x26, 0xbf, 0x9b, 0x7c, 0x53, + 0x9f, 0x65, 0xca, 0x22, 0xc8, 0x26, 0x39, 0x0b, 0x56, 0x87, 0x27, 0x42, 0xdc, 0x5c, 0x3d, 0x7f, + 0x78, 0xfa, 0x22, 0xa0, 0x0e, 0x4f, 0xb4, 0x30, 0x27, 0xb4, 0xb5, 0x0d, 0x1c, 0x26, 0x66, 0x25, + 0xd9, 0x5f, 0xfa, 0xd6, 0x92, 0x8d, 0x6f, 0x16, 0x6d, 0x83, 0xe9, 0x02, 0x57, 0xff, 0xb4, 0x08, + 0x56, 0x64, 0xe0, 0x8a, 0xdb, 0xd7, 0xef, 0xc7, 0xb1, 0x28, 0xe2, 0xf6, 0x8d, 0xf3, 0x62, 0x91, + 0x9b, 0x2b, 0xa1, 0xf8, 0x76, 0x1c, 0x8a, 0x22, 0x88, 0x37, 0xa6, 0x8b, 0x26, 0x9f, 0x89, 0x82, + 0x90, 0xe1, 0xb7, 0x1d, 0x85, 0x9f, 0x88, 0xdd, 0xd7, 0xd3, 0xef, 0x30, 0x22, 0x94, 0x8c, 0xbd, + 0x1d, 0x50, 0x72, 0xc5, 0x27, 0xa9, 0xb4, 0xa8, 0x4d, 0x7e, 0xb1, 0x62, 0xd1, 0x24, 0x01, 0x70, + 0x7b, 0x1a, 0x83, 0x05, 0xf9, 0x09, 0x26, 0x11, 0x83, 0x31, 0x28, 0x0a, 0xc1, 0xab, 0x71, 0x08, + 0x16, 0x67, 0x3f, 0xdb, 0x44, 0x01, 0x18, 0x4f, 0x4c, 0xc6, 0xdf, 0x6d, 0xb0, 0x1a, 0x29, 0x96, + 0x37, 0xc9, 0x00, 0x7c, 0xe3, 0xbc, 0x8d, 0x62, 0x84, 0xd7, 0x51, 0xf0, 0x6e, 0x42, 0xe6, 0xe5, + 0xd9, 0xe4, 0x3e, 0x2b, 0xf2, 0xa8, 0xa7, 0x59, 0x8d, 0xdf, 0x03, 0x97, 0xa6, 0x32, 0x15, 0x63, + 0x02, 0xc9, 0xc3, 0xa1, 0x26, 0xf0, 0xa8, 0xab, 0x59, 0xa0, 0x3a, 0x2c, 0x29, 0xef, 0xe5, 0xf3, + 0x86, 0x15, 0x89, 0x3b, 0x31, 0x2c, 0xa9, 0xed, 0x3d, 0xb0, 0x34, 0xc0, 0x14, 0x75, 0x10, 0x45, + 0x66, 0x89, 0x27, 0xba, 0x37, 0x13, 0x21, 0x27, 0xd1, 0x8d, 0x07, 0xd2, 0xf0, 0xb6, 0x47, 0xc3, + 0x53, 0xb9, 0x57, 0x8f, 0xd1, 0x9b, 0x3f, 0x00, 0xab, 0x9a, 0x01, 0x5c, 0x03, 0xb9, 0x63, 0x1c, + 0x7d, 0xa6, 0x64, 0x8f, 0x70, 0x1d, 0x14, 0x4e, 0x50, 0x7f, 0x88, 0xb9, 0x3e, 0xcb, 0x8e, 0x28, + 0xec, 0x64, 0xdf, 0x33, 0xec, 0x32, 0x28, 0x85, 0xe2, 0x2d, 0x76, 0xf7, 0xe9, 0xb3, 0x6a, 0xe6, + 0xcb, 0x67, 0xd5, 0xcc, 0x57, 0xcf, 0xaa, 0xc6, 0x27, 0x93, 0xaa, 0xf1, 0xdb, 0x49, 0xd5, 0xf8, + 0x7c, 0x52, 0x35, 0x9e, 0x4e, 0xaa, 0xc6, 0xdf, 0x27, 0x55, 0xe3, 0x1f, 0x93, 0x6a, 0xe6, 0xab, + 0x49, 0xd5, 0xf8, 0xec, 0x79, 0x35, 0xf3, 0xf4, 0x79, 0x35, 0xf3, 0xe5, 0xf3, 0x6a, 0xe6, 0xa7, + 0x57, 0x17, 0xce, 0xb9, 0x87, 0x45, 0xce, 0xd4, 0xf6, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xaa, + 0x37, 0x05, 0xb8, 0xaf, 0x21, 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -2200,6 +2270,53 @@ func (this *QuantileSketchResponse) Equal(that interface{}) bool { } return true } +func (this *CountMinSketchResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CountMinSketchResponse) + if !ok { + that2, ok := that.(CountMinSketchResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if that1.Response == nil { + if this.Response != nil { + return false + } + } else if !this.Response.Equal(*that1.Response) { + return false + } + if len(this.Headers) != len(that1.Headers) { + return false + } + for i := range this.Headers { + if !this.Headers[i].Equal(that1.Headers[i]) { + return false + } + } + if len(this.Warnings) != len(that1.Warnings) { + return false + } + for i := range this.Warnings { + if this.Warnings[i] != that1.Warnings[i] { + return false + } + } + if !this.Statistics.Equal(&that1.Statistics) { + return false + } + return true +} func (this *ShardsResponse) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2665,6 +2782,30 @@ func (this *QueryResponse_DetectedLabels) Equal(that interface{}) bool { } return true } +func (this *QueryResponse_CountMinSketches) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryResponse_CountMinSketches) + if !ok { + that2, ok := that.(QueryResponse_CountMinSketches) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.CountMinSketches.Equal(that1.CountMinSketches) { + return false + } + return true +} func (this *QueryRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -3135,6 +3276,19 @@ func (this *QuantileSketchResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *CountMinSketchResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&queryrange.CountMinSketchResponse{") + s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n") + s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") + s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") + s = append(s, "Statistics: "+strings.Replace(this.Statistics.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *ShardsResponse) GoString() string { if this == nil { return "nil" @@ -3183,7 +3337,7 @@ func (this *QueryResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 17) + s := make([]string, 0, 18) s = append(s, "&queryrange.QueryResponse{") if this.Status != nil { s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") @@ -3290,6 +3444,14 @@ func (this *QueryResponse_DetectedLabels) GoString() string { `DetectedLabels:` + fmt.Sprintf("%#v", this.DetectedLabels) + `}`}, ", ") return s } +func (this *QueryResponse_CountMinSketches) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&queryrange.QueryResponse_CountMinSketches{` + + `CountMinSketches:` + fmt.Sprintf("%#v", this.CountMinSketches) + `}`}, ", ") + return s +} func (this *QueryRequest) GoString() string { if this == nil { return "nil" @@ -4277,6 +4439,74 @@ func (m *QuantileSketchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *CountMinSketchResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CountMinSketchResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CountMinSketchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + if len(m.Warnings) > 0 { + for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Warnings[iNdEx]) + copy(dAtA[i:], m.Warnings[iNdEx]) + i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Warnings[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Headers) > 0 { + for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Headers[iNdEx].Size() + i -= size + if _, err := m.Headers[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.Response != nil { + { + size := m.Response.Size() + i -= size + if _, err := m.Response.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *ShardsResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4757,6 +4987,26 @@ func (m *QueryResponse_DetectedLabels) MarshalToSizedBuffer(dAtA []byte) (int, e } return len(dAtA) - i, nil } +func (m *QueryResponse_CountMinSketches) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *QueryResponse_CountMinSketches) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.CountMinSketches != nil { + { + size, err := m.CountMinSketches.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x72 + } + return len(dAtA) - i, nil +} func (m *QueryRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -5383,6 +5633,33 @@ func (m *QuantileSketchResponse) Size() (n int) { return n } +func (m *CountMinSketchResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Response != nil { + l = m.Response.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + if len(m.Headers) > 0 { + for _, e := range m.Headers { + l = e.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + } + if len(m.Warnings) > 0 { + for _, s := range m.Warnings { + l = len(s) + n += 1 + l + sovQueryrange(uint64(l)) + } + } + l = m.Statistics.Size() + n += 1 + l + sovQueryrange(uint64(l)) + return n +} + func (m *ShardsResponse) Size() (n int) { if m == nil { return 0 @@ -5619,6 +5896,18 @@ func (m *QueryResponse_DetectedLabels) Size() (n int) { } return n } +func (m *QueryResponse_CountMinSketches) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CountMinSketches != nil { + l = m.CountMinSketches.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} func (m *QueryRequest) Size() (n int) { if m == nil { return 0 @@ -5951,6 +6240,19 @@ func (this *QuantileSketchResponse) String() string { }, "") return s } +func (this *CountMinSketchResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&CountMinSketchResponse{`, + `Response:` + fmt.Sprintf("%v", this.Response) + `,`, + `Headers:` + fmt.Sprintf("%v", this.Headers) + `,`, + `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, + `Statistics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statistics), "Result", "stats.Result", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} func (this *ShardsResponse) String() string { if this == nil { return "nil" @@ -6126,6 +6428,16 @@ func (this *QueryResponse_DetectedLabels) String() string { }, "") return s } +func (this *QueryResponse_CountMinSketches) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryResponse_CountMinSketches{`, + `CountMinSketches:` + strings.Replace(fmt.Sprintf("%v", this.CountMinSketches), "CountMinSketchResponse", "CountMinSketchResponse", 1) + `,`, + `}`, + }, "") + return s +} func (this *QueryRequest) String() string { if this == nil { return "nil" @@ -8887,6 +9199,194 @@ func (m *QuantileSketchResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *CountMinSketchResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CountMinSketchResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CountMinSketchResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Response == nil { + m.Response = &github_com_grafana_loki_v3_pkg_logproto.CountMinSketchVector{} + } + if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Headers = append(m.Headers, github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader{}) + if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Statistics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQueryrange(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ShardsResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -9864,6 +10364,41 @@ func (m *QueryResponse) Unmarshal(dAtA []byte) error { } m.Response = &QueryResponse_DetectedLabels{v} iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CountMinSketches", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &CountMinSketchResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Response = &QueryResponse_CountMinSketches{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) diff --git a/pkg/querier/queryrange/queryrange.proto b/pkg/querier/queryrange/queryrange.proto index 47bec327369f4..fe234939465d3 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -182,6 +182,16 @@ message QuantileSketchResponse { stats.Result statistics = 4 [(gogoproto.nullable) = false]; } +message CountMinSketchResponse { + logproto.CountMinSketchVector response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/logproto.CountMinSketchVector"]; + repeated definitions.PrometheusResponseHeader Headers = 2 [ + (gogoproto.jsontag) = "-", + (gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" + ]; + repeated string warnings = 3 [(gogoproto.jsontag) = "warnings,omitempty"]; + stats.Result statistics = 4 [(gogoproto.nullable) = false]; +} + message ShardsResponse { indexgatewaypb.ShardsResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/logproto.ShardsResponse"]; repeated definitions.PrometheusResponseHeader Headers = 2 [ @@ -228,6 +238,7 @@ message QueryResponse { DetectedFieldsResponse detectedFields = 11; QueryPatternsResponse patternsResponse = 12; DetectedLabelsResponse detectedLabels = 13; + CountMinSketchResponse countMinSketches = 14; } } diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 9fe578fad665a..4a5f7558e63c1 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "net/http" + "slices" "time" "github.com/dustin/go-humanize" @@ -216,7 +217,13 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que } strategy := version.Strategy(resolver, uint64(ast.limits.TSDBMaxBytesPerShard(tenants[0]))) - mapper := logql.NewShardMapper(strategy, ast.metrics, ast.shardAggregation) + // Merge global shard aggregations and tenant overrides. + limitShardAggregation := validation.IntersectionPerTenant(tenants, func(tenant string) []string { + return ast.limits.ShardAggregations(tenant) + }) + mergedShardAggregation := slices.Compact(append(limitShardAggregation, ast.shardAggregation...)) + + mapper := logql.NewShardMapper(strategy, ast.metrics, mergedShardAggregation) noop, bytesPerShard, parsed, err := mapper.Parse(params.GetExpression()) if err != nil { diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index a899d64c21be0..1a17c00861646 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -1536,6 +1536,10 @@ func (f fakeLimits) TSDBShardingStrategy(string) string { return logql.PowerOfTwoVersion.String() } +func (f fakeLimits) ShardAggregations(string) []string { + return nil +} + type ingesterQueryOpts struct { queryStoreOnly bool queryIngestersWithin time.Duration diff --git a/pkg/querier/queryrange/shard_resolver.go b/pkg/querier/queryrange/shard_resolver.go index 4fe444c3bc59b..a8d6fa4cbff3f 100644 --- a/pkg/querier/queryrange/shard_resolver.go +++ b/pkg/querier/queryrange/shard_resolver.go @@ -201,7 +201,7 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) { maxBytesPerShard := validation.SmallestPositiveIntPerTenant(tenantIDs, r.limits.TSDBMaxBytesPerShard) factor := sharding.GuessShardFactor(combined.Bytes, uint64(maxBytesPerShard), r.maxShards) - var bytesPerShard = combined.Bytes + bytesPerShard := combined.Bytes if factor > 0 { bytesPerShard = combined.Bytes / uint64(factor) } @@ -275,7 +275,6 @@ func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerSh Query: expr.String(), TargetBytesPerShard: targetBytesPerShard, }) - if err != nil { return nil, nil, errors.Wrapf(err, "failed to get shards for expression, got %T: %+v", err, err) } diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index e5e80177c64d0..523b5c9e5fb06 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -1,6 +1,7 @@ package validation import ( + "slices" "time" ) @@ -87,3 +88,31 @@ func MaxDurationOrZeroPerTenant(tenantIDs []string, f func(string) time.Duration } return *result } + +// IntersectionPerTenant is returning the intersection of feature flags. This is useful to determine the minimal +// feature set supported. +func IntersectionPerTenant(tenantIDs []string, f func(string) []string) []string { + var result []string + for _, tenantID := range tenantIDs { + v := f(tenantID) + slices.Sort(v) + if result == nil { + result = v + continue + } + var updatedResult []string + for i, j := 0, 0; i < len(result) && j < len(v); { + if result[i] == v[j] { + updatedResult = append(updatedResult, result[i]) + i++ + j++ + } else if result[i] < v[j] { + i++ + } else { + j++ + } + } + result = updatedResult + } + return result +} diff --git a/pkg/util/validation/limits_test.go b/pkg/util/validation/limits_test.go index 896b11ee7d3a4..9d352b0885f2f 100644 --- a/pkg/util/validation/limits_test.go +++ b/pkg/util/validation/limits_test.go @@ -3,6 +3,8 @@ package validation import ( "testing" "time" + + "github.com/stretchr/testify/require" ) // nolint:goconst @@ -220,3 +222,98 @@ func TestMaxDurationOrZeroPerTenant(t *testing.T) { }) } } + +func TestIntersectionPerTenant(t *testing.T) { + tests := []struct { + name string + tenantIDs []string + f func(string) []string + expected []string + }{ + { + name: "no tenants", + tenantIDs: []string{}, + f: func(_ string) []string { + return nil + }, + expected: []string{}, + }, + { + name: "single tenant with features", + tenantIDs: []string{"tenant1"}, + f: func(tenantID string) []string { + if tenantID == "tenant1" { + return []string{"featureA", "featureB", "featureC"} + } + return nil + }, + expected: []string{"featureA", "featureB", "featureC"}, + }, + { + name: "multiple tenants with common features", + tenantIDs: []string{"tenant1", "tenant2"}, + f: func(tenantID string) []string { + if tenantID == "tenant1" { + return []string{"featureA", "featureB", "featureC"} + } + if tenantID == "tenant2" { + return []string{"featureB", "featureC", "featureD"} + } + return nil + }, + expected: []string{"featureB", "featureC"}, + }, + { + name: "multiple tenants with no common features", + tenantIDs: []string{"tenant1", "tenant2"}, + f: func(tenantID string) []string { + if tenantID == "tenant1" { + return []string{"featureA"} + } + if tenantID == "tenant2" { + return []string{"featureB"} + } + return nil + }, + expected: []string{}, + }, + { + name: "multiple tenants with overlapping features", + tenantIDs: []string{"tenant1", "tenant2", "tenant3"}, + f: func(tenantID string) []string { + if tenantID == "tenant1" { + return []string{"featureA", "featureB"} + } + if tenantID == "tenant2" { + return []string{"featureB", "featureC"} + } + if tenantID == "tenant3" { + return []string{"featureB", "featureD"} + } + return nil + }, + expected: []string{"featureB"}, + }, + { + name: "tenant with empty feature set", + tenantIDs: []string{"tenant1", "tenant2"}, + f: func(tenantID string) []string { + if tenantID == "tenant1" { + return []string{"featureA", "featureB"} + } + if tenantID == "tenant2" { + return []string{} + } + return nil + }, + expected: []string{}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + actual := IntersectionPerTenant(tt.tenantIDs, tt.f) + require.ElementsMatch(t, actual, tt.expected) + }) + } +} diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 5d819c8d88b50..353a02a6adfd6 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -228,6 +228,8 @@ type Limits struct { IngestionPartitionsTenantShardSize int `yaml:"ingestion_partitions_tenant_shard_size" json:"ingestion_partitions_tenant_shard_size" category:"experimental"` + ShardAggregations []string `yaml:"shard_aggregations,omitempty" json:"shard_aggregations,omitempty" doc:"description=List of LogQL vector and range aggregations that should be sharded."` + PatternIngesterTokenizableJSONFieldsDefault dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_default" json:"pattern_ingester_tokenizable_json_fields_default" doc:"hidden"` PatternIngesterTokenizableJSONFieldsAppend dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_append" json:"pattern_ingester_tokenizable_json_fields_append" doc:"hidden"` PatternIngesterTokenizableJSONFieldsDelete dskit_flagext.StringSliceCSV `yaml:"pattern_ingester_tokenizable_json_fields_delete" json:"pattern_ingester_tokenizable_json_fields_delete" doc:"hidden"` @@ -1086,6 +1088,10 @@ func (o *Overrides) BlockIngestionStatusCode(userID string) int { return o.getOverridesForUser(userID).BlockIngestionStatusCode } +func (o *Overrides) ShardAggregations(userID string) []string { + return o.getOverridesForUser(userID).ShardAggregations +} + func (o *Overrides) PatternIngesterTokenizableJSONFields(userID string) []string { defaultFields := o.getOverridesForUser(userID).PatternIngesterTokenizableJSONFieldsDefault appendFields := o.getOverridesForUser(userID).PatternIngesterTokenizableJSONFieldsAppend diff --git a/tools/dev/loki-tsdb-storage-s3/config/loki.yaml b/tools/dev/loki-tsdb-storage-s3/config/loki.yaml index 4ab4056956d3b..57c5f41adf581 100644 --- a/tools/dev/loki-tsdb-storage-s3/config/loki.yaml +++ b/tools/dev/loki-tsdb-storage-s3/config/loki.yaml @@ -85,6 +85,8 @@ limits_config: reject_old_samples: true reject_old_samples_max_age: 168h split_queries_by_interval: 15m + shard_aggregations: + - approx_topk querier: query_ingesters_within: 2h multi_tenant_queries_enabled: true diff --git a/tools/dev/loki-tsdb-storage-s3/dev.dockerfile b/tools/dev/loki-tsdb-storage-s3/dev.dockerfile index d8526a3d93856..0b8ed52f413b6 100644 --- a/tools/dev/loki-tsdb-storage-s3/dev.dockerfile +++ b/tools/dev/loki-tsdb-storage-s3/dev.dockerfile @@ -1,6 +1,6 @@ FROM golang:1.23 ENV CGO_ENABLED=0 -RUN go install github.com/go-delve/delve/cmd/dlv@v1.22.1 +RUN go install github.com/go-delve/delve/cmd/dlv@v1.23.1 FROM alpine:3.20.3