From deb3970a74d4bf97126519e25270dbf41c6d2fb9 Mon Sep 17 00:00:00 2001 From: Periklis Tsirakidis Date: Thu, 6 Feb 2025 16:03:21 +0100 Subject: [PATCH] feat(ingest-limits): Request stream usage for owned partitions only --- pkg/distributor/distributor.go | 2 +- pkg/kafka/encoding.go | 8 +- pkg/kafka/encoding_test.go | 14 +- pkg/limits/frontend/service.go | 155 +- pkg/limits/frontend/service_test.go | 136 +- pkg/limits/ingest_limits.go | 443 ++++-- pkg/limits/ingest_limits_test.go | 369 ++++- pkg/logproto/logproto.pb.go | 1823 +++++++++++++++-------- pkg/logproto/logproto.proto | 16 +- pkg/loki/loki.go | 2 +- pkg/loki/modules.go | 1 + tools/stream-metadata-generator/main.go | 2 +- 12 files changed, 2085 insertions(+), 886 deletions(-) diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 7838cdcf50055..6545d678391c8 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -1159,7 +1159,7 @@ func (d *Distributor) sendStreamToKafka(ctx context.Context, stream KeyedStream, } // Add metadata record - metadataRecord := kafka.EncodeStreamMetadata(partitionID, d.cfg.KafkaConfig.Topic, tenant, stream.HashNoShard) + metadataRecord := kafka.EncodeStreamMetadata(partitionID, d.cfg.KafkaConfig.Topic, tenant, stream.HashNoShard, stream.RingToken) records = append(records, metadataRecord) d.kafkaRecordsPerRequest.Observe(float64(len(records))) diff --git a/pkg/kafka/encoding.go b/pkg/kafka/encoding.go index e89a37caee680..604a0fdd54606 100644 --- a/pkg/kafka/encoding.go +++ b/pkg/kafka/encoding.go @@ -203,19 +203,23 @@ func sovPush(x uint64) (n int) { // EncodeStreamMetadata encodes the stream metadata into a Kafka record // using the tenantID as the key and partition as the target partition -func EncodeStreamMetadata(partition int32, topic string, tenantID string, streamHash uint64) *kgo.Record { +func EncodeStreamMetadata(partition int32, topic string, tenantID string, streamHash uint64, ringToken uint32) *kgo.Record { // Validate stream hash if streamHash == 0 { return nil } + if ringToken == 0 { + return nil + } + // Get metadata from pool metadata := metadataPool.Get().(*logproto.StreamMetadata) defer metadataPool.Put(metadata) // Set stream hash metadata.StreamHash = streamHash - + metadata.RingToken = ringToken // Encode the metadata into a byte slice value, err := metadata.Marshal() if err != nil { diff --git a/pkg/kafka/encoding_test.go b/pkg/kafka/encoding_test.go index 64ae2dfaf9a35..a9455d2b1ab7e 100644 --- a/pkg/kafka/encoding_test.go +++ b/pkg/kafka/encoding_test.go @@ -155,6 +155,7 @@ func TestEncodeDecodeStreamMetadata(t *testing.T) { tests := []struct { name string hash uint64 + ringToken uint32 partition int32 topic string tenantID string @@ -163,6 +164,7 @@ func TestEncodeDecodeStreamMetadata(t *testing.T) { { name: "Valid metadata", hash: 12345, + ringToken: 1, partition: 1, topic: "logs", tenantID: "tenant-1", @@ -171,17 +173,27 @@ func TestEncodeDecodeStreamMetadata(t *testing.T) { { name: "Zero hash - should error", hash: 0, + ringToken: 1, partition: 3, topic: "traces", tenantID: "tenant-3", expectErr: true, }, + { + name: "Zero ring token - should error", + hash: 12345, + ringToken: 0, + partition: 1, + topic: "logs", + tenantID: "tenant-1", + expectErr: true, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { // Encode metadata - record := EncodeStreamMetadata(tt.partition, tt.topic, tt.tenantID, tt.hash) + record := EncodeStreamMetadata(tt.partition, tt.topic, tt.tenantID, tt.hash, tt.ringToken) if tt.expectErr { require.Nil(t, record) return diff --git a/pkg/limits/frontend/service.go b/pkg/limits/frontend/service.go index 4c5a98c148979..5236c2c31fe2c 100644 --- a/pkg/limits/frontend/service.go +++ b/pkg/limits/frontend/service.go @@ -2,6 +2,9 @@ package frontend import ( "context" + "fmt" + "sort" + "strings" "github.com/go-kit/log" "github.com/grafana/dskit/ring" @@ -42,10 +45,9 @@ var ( ) type metrics struct { - tenantExceedsLimits *prometheus.CounterVec - tenantActiveStreams *prometheus.GaugeVec - tenantDuplicateStreamsFound *prometheus.CounterVec - tenantRejectedStreams *prometheus.CounterVec + tenantExceedsLimits *prometheus.CounterVec + tenantActiveStreams *prometheus.GaugeVec + tenantRejectedStreams *prometheus.CounterVec } func newMetrics(reg prometheus.Registerer) *metrics { @@ -60,11 +62,6 @@ func newMetrics(reg prometheus.Registerer) *metrics { Name: "ingest_limits_frontend_streams_active", Help: "The current number of active streams (seen within the window) per tenant.", }, []string{"tenant"}), - tenantDuplicateStreamsFound: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "ingest_limits_frontend_streams_duplicate_total", - Help: "The total number of duplicate streams found per tenant.", - }, []string{"tenant"}), tenantRejectedStreams: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Namespace: constants.Loki, Name: "ingest_limits_frontend_streams_rejected_total", @@ -73,7 +70,7 @@ func newMetrics(reg prometheus.Registerer) *metrics { } } -type ringFunc func(context.Context, logproto.IngestLimitsClient) (*logproto.GetStreamUsageResponse, error) +type ringGetUsageFunc func(context.Context, logproto.IngestLimitsClient, []int32) (*logproto.GetStreamUsageResponse, error) // RingIngestLimitsService is an IngestLimitsService that uses the ring to read the responses // from all limits backends. @@ -99,7 +96,7 @@ func NewRingIngestLimitsService(ring ring.ReadRing, pool *ring_client.Pool, limi } } -func (s *RingIngestLimitsService) forAllBackends(ctx context.Context, f ringFunc) ([]Response, error) { +func (s *RingIngestLimitsService) forAllBackends(ctx context.Context, f ringGetUsageFunc) ([]GetStreamUsageResponse, error) { replicaSet, err := s.ring.GetAllHealthy(LimitsRead) if err != nil { return nil, err @@ -107,33 +104,107 @@ func (s *RingIngestLimitsService) forAllBackends(ctx context.Context, f ringFunc return s.forGivenReplicaSet(ctx, replicaSet, f) } -func (s *RingIngestLimitsService) forGivenReplicaSet(ctx context.Context, replicaSet ring.ReplicationSet, f ringFunc) ([]Response, error) { +func (s *RingIngestLimitsService) forGivenReplicaSet(ctx context.Context, replicaSet ring.ReplicationSet, f ringGetUsageFunc) ([]GetStreamUsageResponse, error) { + partitions, err := s.perReplicaSetPartitions(ctx, replicaSet) + if err != nil { + return nil, err + } + g, ctx := errgroup.WithContext(ctx) - responses := make([]Response, len(replicaSet.Instances)) + responses := make([]GetStreamUsageResponse, len(replicaSet.Instances)) + for i, instance := range replicaSet.Instances { g.Go(func() error { client, err := s.pool.GetClientFor(instance.Addr) if err != nil { return err } - resp, err := f(ctx, client.(logproto.IngestLimitsClient)) + + var partitionStr strings.Builder + for _, partition := range partitions[instance.Addr] { + partitionStr.WriteString(fmt.Sprintf("%d,", partition)) + } + + resp, err := f(ctx, client.(logproto.IngestLimitsClient), partitions[instance.Addr]) if err != nil { return err } - responses[i] = Response{Addr: instance.Addr, Response: resp} + responses[i] = GetStreamUsageResponse{Addr: instance.Addr, Response: resp} return nil }) } + if err := g.Wait(); err != nil { return nil, err } + return responses, nil } +func (s *RingIngestLimitsService) perReplicaSetPartitions(ctx context.Context, replicaSet ring.ReplicationSet) (map[string][]int32, error) { + g, ctx := errgroup.WithContext(ctx) + responses := make([]GetAssignedPartitionsResponse, len(replicaSet.Instances)) + for i, instance := range replicaSet.Instances { + g.Go(func() error { + client, err := s.pool.GetClientFor(instance.Addr) + if err != nil { + return err + } + resp, err := client.(logproto.IngestLimitsClient).GetAssignedPartitions(ctx, &logproto.GetAssignedPartitionsRequest{}) + if err != nil { + return err + } + responses[i] = GetAssignedPartitionsResponse{Addr: instance.Addr, Response: resp} + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, err + } + + partitions := make(map[string][]int32) + // Track highest value seen for each partition + highestValues := make(map[int32]int64) + // Track which addr has the highest value for each partition + highestAddr := make(map[int32]string) + + // First pass - find highest values for each partition + for _, resp := range responses { + for partition, value := range resp.Response.AssignedPartitions { + if currentHighest, exists := highestValues[partition]; !exists || value > currentHighest { + highestValues[partition] = value + highestAddr[partition] = resp.Addr + } + } + } + + // Second pass - assign partitions to addrs that have the highest values + for partition, addr := range highestAddr { + partitions[addr] = append(partitions[addr], partition) + } + + // Sort partition IDs for each address for consistent ordering + for addr := range partitions { + sort.Slice(partitions[addr], func(i, j int) bool { + return partitions[addr][i] < partitions[addr][j] + }) + } + + return partitions, nil +} + func (s *RingIngestLimitsService) ExceedsLimits(ctx context.Context, req *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error) { - resps, err := s.forAllBackends(ctx, func(_ context.Context, client logproto.IngestLimitsClient) (*logproto.GetStreamUsageResponse, error) { + reqStreams := make([]uint64, 0, len(req.Streams)) + for _, stream := range req.Streams { + reqStreams = append(reqStreams, stream.StreamHash) + } + + resps, err := s.forAllBackends(ctx, func(_ context.Context, client logproto.IngestLimitsClient, partitions []int32) (*logproto.GetStreamUsageResponse, error) { return client.GetStreamUsage(ctx, &logproto.GetStreamUsageRequest{ - Tenant: req.Tenant, + Tenant: req.Tenant, + Partitions: partitions, + StreamHashes: reqStreams, }) }) if err != nil { @@ -142,28 +213,9 @@ func (s *RingIngestLimitsService) ExceedsLimits(ctx context.Context, req *logpro maxGlobalStreams := s.limits.MaxGlobalStreamsPerUser(req.Tenant) - var ( - activeStreamsTotal uint64 - uniqueStreamHashes = make(map[uint64]bool) - ) + var activeStreamsTotal uint64 for _, resp := range resps { - var duplicates uint64 - // Record the unique stream hashes - // and count duplicates active streams - // to be subtracted from the total - for _, stream := range resp.Response.RecordedStreams { - if uniqueStreamHashes[stream.StreamHash] { - duplicates++ - continue - } - uniqueStreamHashes[stream.StreamHash] = true - } - - activeStreamsTotal += resp.Response.ActiveStreams - duplicates - - if duplicates > 0 { - s.metrics.tenantDuplicateStreamsFound.WithLabelValues(req.Tenant).Inc() - } + activeStreamsTotal += resp.Response.ActiveStreams } s.metrics.tenantActiveStreams.WithLabelValues(req.Tenant).Set(float64(activeStreamsTotal)) @@ -174,13 +226,19 @@ func (s *RingIngestLimitsService) ExceedsLimits(ctx context.Context, req *logpro }, nil } - var rejectedStreams []*logproto.RejectedStream - for _, stream := range req.Streams { - if !uniqueStreamHashes[stream.StreamHash] { - rejectedStreams = append(rejectedStreams, &logproto.RejectedStream{ - StreamHash: stream.StreamHash, - Reason: RejectedStreamReasonExceedsGlobalLimit, - }) + var ( + rejectedStreams []*logproto.RejectedStream + uniqueStreamHashes = make(map[uint64]bool) + ) + for _, resp := range resps { + for _, unknownStream := range resp.Response.UnknownStreams { + if !uniqueStreamHashes[unknownStream] { + uniqueStreamHashes[unknownStream] = true + rejectedStreams = append(rejectedStreams, &logproto.RejectedStream{ + StreamHash: unknownStream, + Reason: RejectedStreamReasonExceedsGlobalLimit, + }) + } } } @@ -195,7 +253,12 @@ func (s *RingIngestLimitsService) ExceedsLimits(ctx context.Context, req *logpro }, nil } -type Response struct { +type GetStreamUsageResponse struct { Addr string Response *logproto.GetStreamUsageResponse } + +type GetAssignedPartitionsResponse struct { + Addr string + Response *logproto.GetAssignedPartitionsResponse +} diff --git a/pkg/limits/frontend/service_test.go b/pkg/limits/frontend/service_test.go index 31f8d0f3e58ef..17e7b496f07ba 100644 --- a/pkg/limits/frontend/service_test.go +++ b/pkg/limits/frontend/service_test.go @@ -45,7 +45,12 @@ func (f *mockFactory) FromInstance(_ ring.InstanceDesc) (ring_client.PoolClient, type mockIngestLimitsClient struct { logproto.IngestLimitsClient - getStreamUsageResponse *logproto.GetStreamUsageResponse + getStreamUsageResponse *logproto.GetStreamUsageResponse + getAssignedPartitionsResponse *logproto.GetAssignedPartitionsResponse +} + +func (m *mockIngestLimitsClient) GetAssignedPartitions(_ context.Context, _ *logproto.GetAssignedPartitionsRequest, _ ...grpc.CallOption) (*logproto.GetAssignedPartitionsResponse, error) { + return m.getAssignedPartitionsResponse, nil } func (m *mockIngestLimitsClient) GetStreamUsage(_ context.Context, _ *logproto.GetStreamUsageRequest, _ ...grpc.CallOption) (*logproto.GetStreamUsageResponse, error) { @@ -68,24 +73,33 @@ func (m *mockIngestLimitsClient) Watch(_ context.Context, _ *grpc_health_v1.Heal func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { tests := []struct { - name string - tenant string - maxGlobalStreams int - streams []*logproto.StreamMetadataWithSize - backendResponses []*logproto.GetStreamUsageResponse - expectedRejections []*logproto.RejectedStream + name string + tenant string + maxGlobalStreams int + streams []*logproto.StreamMetadataWithSize + getStreamUsageResps []*logproto.GetStreamUsageResponse + getAssignedPartitionsResps []*logproto.GetAssignedPartitionsResponse + expectedRejections []*logproto.RejectedStream }{ { name: "no streams", tenant: "test", maxGlobalStreams: 10, streams: []*logproto.StreamMetadataWithSize{}, - backendResponses: []*logproto.GetStreamUsageResponse{ + getStreamUsageResps: []*logproto.GetStreamUsageResponse{ { Tenant: "test", ActiveStreams: 0, }, }, + getAssignedPartitionsResps: []*logproto.GetAssignedPartitionsResponse{ + { + AssignedPartitions: map[int32]int64{ + 0: 1, + 1: 1, + }, + }, + }, expectedRejections: nil, }, { @@ -96,13 +110,17 @@ func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { {StreamHash: 1}, {StreamHash: 2}, }, - backendResponses: []*logproto.GetStreamUsageResponse{ + getStreamUsageResps: []*logproto.GetStreamUsageResponse{ { Tenant: "test", ActiveStreams: 2, - RecordedStreams: []*logproto.RecordedStreams{ - {StreamHash: 1}, - {StreamHash: 2}, + }, + }, + getAssignedPartitionsResps: []*logproto.GetAssignedPartitionsResponse{ + { + AssignedPartitions: map[int32]int64{ + 0: 1, + 1: 1, }, }, }, @@ -113,19 +131,21 @@ func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { tenant: "test", maxGlobalStreams: 5, streams: []*logproto.StreamMetadataWithSize{ - {StreamHash: 6}, - {StreamHash: 7}, + {StreamHash: 6}, // Exceeds limit + {StreamHash: 7}, // Exceeds limit }, - backendResponses: []*logproto.GetStreamUsageResponse{ + getStreamUsageResps: []*logproto.GetStreamUsageResponse{ { - Tenant: "test", - ActiveStreams: 5, - RecordedStreams: []*logproto.RecordedStreams{ - {StreamHash: 1}, - {StreamHash: 2}, - {StreamHash: 3}, - {StreamHash: 4}, - {StreamHash: 5}, + Tenant: "test", + ActiveStreams: 5, + UnknownStreams: []uint64{6, 7}, + }, + }, + getAssignedPartitionsResps: []*logproto.GetAssignedPartitionsResponse{ + { + AssignedPartitions: map[int32]int64{ + 0: 1, + 1: 1, }, }, }, @@ -144,55 +164,27 @@ func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { {StreamHash: 3}, {StreamHash: 4}, {StreamHash: 5}, - {StreamHash: 6}, - {StreamHash: 7}, + {StreamHash: 6}, // Exceeds limit + {StreamHash: 7}, // Exceeds limit }, - backendResponses: []*logproto.GetStreamUsageResponse{ + getStreamUsageResps: []*logproto.GetStreamUsageResponse{ { - Tenant: "test", - ActiveStreams: 5, - RecordedStreams: []*logproto.RecordedStreams{ - {StreamHash: 1}, - {StreamHash: 2}, - {StreamHash: 3}, - {StreamHash: 4}, - {StreamHash: 5}, - }, + Tenant: "test", + ActiveStreams: 5, + UnknownStreams: []uint64{6, 7}, }, }, - expectedRejections: []*logproto.RejectedStream{ - {StreamHash: 6, Reason: RejectedStreamReasonExceedsGlobalLimit}, - {StreamHash: 7, Reason: RejectedStreamReasonExceedsGlobalLimit}, - }, - }, - { - name: "multiple backends with duplicates", - tenant: "test", - maxGlobalStreams: 3, - streams: []*logproto.StreamMetadataWithSize{ - {StreamHash: 1}, - {StreamHash: 2}, - }, - backendResponses: []*logproto.GetStreamUsageResponse{ - { - Tenant: "test", - ActiveStreams: 3, - RecordedStreams: []*logproto.RecordedStreams{ - {StreamHash: 1}, - {StreamHash: 3}, - }, - }, + getAssignedPartitionsResps: []*logproto.GetAssignedPartitionsResponse{ { - Tenant: "test", - ActiveStreams: 3, - RecordedStreams: []*logproto.RecordedStreams{ - {StreamHash: 1}, // Duplicate - {StreamHash: 4}, + AssignedPartitions: map[int32]int64{ + 0: 1, + 1: 1, }, }, }, expectedRejections: []*logproto.RejectedStream{ - {StreamHash: 2, Reason: RejectedStreamReasonExceedsGlobalLimit}, + {StreamHash: 6, Reason: RejectedStreamReasonExceedsGlobalLimit}, + {StreamHash: 7, Reason: RejectedStreamReasonExceedsGlobalLimit}, }, }, { @@ -202,9 +194,16 @@ func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { streams: []*logproto.StreamMetadataWithSize{ {StreamHash: 1}, }, - backendResponses: []*logproto.GetStreamUsageResponse{ + getStreamUsageResps: []*logproto.GetStreamUsageResponse{ {}, }, + getAssignedPartitionsResps: []*logproto.GetAssignedPartitionsResponse{ + { + AssignedPartitions: map[int32]int64{ + 0: 1, + }, + }, + }, expectedRejections: nil, // No rejections because activeStreamsTotal is 0 }, } @@ -212,12 +211,13 @@ func TestRingIngestLimitsService_ExceedsLimits(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { // Create mock clients that return the test responses - mockClients := make([]logproto.IngestLimitsClient, len(tt.backendResponses)) - mockInstances := make([]ring.InstanceDesc, len(tt.backendResponses)) + mockClients := make([]logproto.IngestLimitsClient, len(tt.getStreamUsageResps)) + mockInstances := make([]ring.InstanceDesc, len(tt.getStreamUsageResps)) - for i, resp := range tt.backendResponses { + for i, resp := range tt.getStreamUsageResps { mockClients[i] = &mockIngestLimitsClient{ - getStreamUsageResponse: resp, + getStreamUsageResponse: resp, + getAssignedPartitionsResponse: tt.getAssignedPartitionsResps[i], } mockInstances[i] = ring.InstanceDesc{ Addr: "mock-instance", diff --git a/pkg/limits/ingest_limits.go b/pkg/limits/ingest_limits.go index 825889cc331dc..fafcc6f2aa896 100644 --- a/pkg/limits/ingest_limits.go +++ b/pkg/limits/ingest_limits.go @@ -5,6 +5,8 @@ import ( "errors" "fmt" "net/http" + "strconv" + "strings" "sync" "time" @@ -18,21 +20,46 @@ import ( "github.com/twmb/franz-go/plugin/kprom" "github.com/grafana/loki/v3/pkg/kafka" - "github.com/grafana/loki/v3/pkg/kafka/client" + "github.com/grafana/loki/v3/pkg/kafka/partitionring/consumer" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/constants" ) const ( + // Ring RingKey = "ingest-limits" RingName = "ingest-limits" + + // Kafka + consumerGroup = "ingest-limits" +) + +var ( + tenantPartitionDesc = prometheus.NewDesc( + constants.Loki+"_ingest_limits_partitions", + "The current number of partitions per tenant.", + []string{"tenant"}, + nil, + ) + + tenantRecordedStreamsDesc = prometheus.NewDesc( + constants.Loki+"_ingest_limits_recorded_streams", + "The current number of recorded streams per tenant. This is not a global total, as tenants can be sharded over multiple pods.", + []string{"tenant"}, + nil, + ) + + tenantActiveStreamsDesc = prometheus.NewDesc( + constants.Loki+"_ingest_limits_active_streams", + "The current number of active streams (seen within the window) per tenant. This is not a global total, as tenants can be sharded over multiple pods.", + []string{"tenant"}, + nil, + ) ) type metrics struct { - tenantCurrentRecordedStreams *prometheus.GaugeVec - tenantStreamEvictionsTotal *prometheus.CounterVec - tenantActiveStreams *prometheus.GaugeVec + tenantStreamEvictionsTotal *prometheus.CounterVec kafkaReadLatency prometheus.Histogram kafkaReadBytesTotal prometheus.Counter @@ -40,21 +67,11 @@ type metrics struct { func newMetrics(reg prometheus.Registerer) *metrics { return &metrics{ - tenantCurrentRecordedStreams: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: constants.Loki, - Name: "ingest_limits_recorded_streams", - Help: "The current number of recorded streams per tenant. This is not a global total, as tenants can be sharded over multiple pods.", - }, []string{"tenant"}), tenantStreamEvictionsTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Namespace: constants.Loki, Name: "ingest_limits_stream_evictions_total", Help: "The total number of streams evicted due to age per tenant. This is not a global total, as tenants can be sharded over multiple pods.", }, []string{"tenant"}), - tenantActiveStreams: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: constants.Loki, - Name: "ingest_limits_active_streams", - Help: "The current number of active streams (seen within the window) per tenant. This is not a global total, as tenants can be sharded over multiple pods.", - }, []string{"tenant"}), kafkaReadLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Namespace: constants.Loki, Name: "ingest_limits_kafka_read_latency_seconds", @@ -72,14 +89,20 @@ func newMetrics(reg prometheus.Registerer) *metrics { } } +type streamMetadata struct { + hash uint64 + lastSeenAt int64 +} + // IngestLimits is a service that manages stream metadata limits. type IngestLimits struct { services.Service cfg Config logger log.Logger - client *kgo.Client + client *consumer.Client + partitionRing ring.PartitionRingReader lifecycler *ring.Lifecycler lifecyclerWatcher *services.FailureWatcher @@ -88,7 +111,11 @@ type IngestLimits struct { // Track stream metadata mtx sync.RWMutex - metadata map[string]map[uint64]int64 // tenant -> streamHash -> lastSeenAt + metadata map[string]map[int32][]streamMetadata // tenant -> partitionID -> streamMetadata + + // Track partition assignments + mtxAssingedPartitions sync.RWMutex + assingedPartitions map[int32]int64 // partitionID -> lastAssignedAt } // Flush implements ring.FlushTransferer. It transfers state to another ingest limits instance. @@ -101,13 +128,19 @@ func (s *IngestLimits) TransferOut(_ context.Context) error { // NewIngestLimits creates a new IngestLimits service. It initializes the metadata map and sets up a Kafka client // The client is configured to consume stream metadata from a dedicated topic with the metadata suffix. -func NewIngestLimits(cfg Config, logger log.Logger, reg prometheus.Registerer) (*IngestLimits, error) { +func NewIngestLimits(cfg Config, partitionRing ring.PartitionRingReader, logger log.Logger, reg prometheus.Registerer) (*IngestLimits, error) { var err error s := &IngestLimits{ - cfg: cfg, - logger: logger, - metadata: make(map[string]map[uint64]int64), - metrics: newMetrics(reg), + cfg: cfg, + logger: logger, + partitionRing: partitionRing, + metadata: make(map[string]map[int32][]streamMetadata), + metrics: newMetrics(reg), + } + + // Initialize internal metadata metrics + if err := reg.Register(s); err != nil { + return nil, fmt.Errorf("failed to register ingest limits internal metadata metrics: %w", err) } // Initialize lifecycler @@ -122,27 +155,23 @@ func NewIngestLimits(cfg Config, logger log.Logger, reg prometheus.Registerer) ( metrics := kprom.NewMetrics("loki_ingest_limits", kprom.Registerer(reg), - kprom.FetchAndProduceDetail(kprom.Batches, kprom.Records, kprom.CompressedBytes, kprom.UncompressedBytes)) + kprom.FetchAndProduceDetail(kprom.Batches, kprom.Records, kprom.CompressedBytes, kprom.UncompressedBytes), + ) // Create a copy of the config to modify the topic kCfg := cfg.KafkaConfig kCfg.Topic = kafka.MetadataTopicFor(kCfg.Topic) - s.client, err = client.NewReaderClient(kCfg, metrics, logger, - kgo.ConsumerGroup("ingest-limits"), - kgo.ConsumeTopics(kCfg.Topic), - kgo.Balancers(kgo.RoundRobinBalancer()), + s.client, err = consumer.NewGroupClient( + kCfg, + partitionRing, + consumerGroup, + metrics, + logger, kgo.ConsumeResetOffset(kgo.NewOffset().AfterMilli(time.Now().Add(-s.cfg.WindowSize).UnixMilli())), - kgo.DisableAutoCommit(), - kgo.OnPartitionsAssigned(func(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { - level.Debug(logger).Log("msg", "assigned partitions", "partitions", partitions) - }), - kgo.OnPartitionsRevoked(func(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { - level.Debug(logger).Log("msg", "revoked partitions", "partitions", partitions) - }), - kgo.OnPartitionsLost(func(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { - level.Debug(logger).Log("msg", "lost partitions", "partitions", partitions) - }), + kgo.OnPartitionsAssigned(s.onPartitionsAssigned), + kgo.OnPartitionsRevoked(s.onPartitionsRevoked), + kgo.OnPartitionsLost(s.onPartitionsLost), ) if err != nil { return nil, fmt.Errorf("failed to create kafka client: %w", err) @@ -152,6 +181,102 @@ func NewIngestLimits(cfg Config, logger log.Logger, reg prometheus.Registerer) ( return s, nil } +func (s *IngestLimits) Describe(descs chan<- *prometheus.Desc) { + descs <- tenantPartitionDesc + descs <- tenantRecordedStreamsDesc + descs <- tenantActiveStreamsDesc +} + +func (s *IngestLimits) Collect(m chan<- prometheus.Metric) { + s.mtx.RLock() + defer s.mtx.RUnlock() + s.mtxAssingedPartitions.RLock() + defer s.mtxAssingedPartitions.RUnlock() + + cutoff := time.Now().Add(-s.cfg.WindowSize).UnixNano() + + for tenant, partitions := range s.metadata { + var ( + recorded int + active int + ) + + for partitionID, partition := range partitions { + if _, assigned := s.assingedPartitions[partitionID]; !assigned { + continue + } + + recorded += len(partition) + + for _, stream := range partition { + if stream.lastSeenAt >= cutoff { + active++ + } + } + } + + m <- prometheus.MustNewConstMetric(tenantPartitionDesc, prometheus.GaugeValue, float64(len(partitions)), tenant) + m <- prometheus.MustNewConstMetric(tenantRecordedStreamsDesc, prometheus.GaugeValue, float64(recorded), tenant) + m <- prometheus.MustNewConstMetric(tenantActiveStreamsDesc, prometheus.GaugeValue, float64(active), tenant) + } +} + +func (s *IngestLimits) onPartitionsAssigned(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { + s.mtxAssingedPartitions.Lock() + defer s.mtxAssingedPartitions.Unlock() + + if s.assingedPartitions == nil { + s.assingedPartitions = make(map[int32]int64) + } + + var assigned []string + for _, partitionIDs := range partitions { + for _, partitionID := range partitionIDs { + s.assingedPartitions[partitionID] = time.Now().UnixNano() + assigned = append(assigned, strconv.Itoa(int(partitionID))) + } + } + + if len(assigned) > 0 { + level.Debug(s.logger).Log("msg", "assigned partitions", "partitions", strings.Join(assigned, ",")) + } +} + +func (s *IngestLimits) onPartitionsRevoked(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { + s.removePartitions(partitions) +} + +func (s *IngestLimits) onPartitionsLost(_ context.Context, _ *kgo.Client, partitions map[string][]int32) { + s.removePartitions(partitions) +} + +func (s *IngestLimits) removePartitions(partitions map[string][]int32) { + s.mtxAssingedPartitions.Lock() + defer s.mtxAssingedPartitions.Unlock() + + s.mtx.Lock() + defer s.mtx.Unlock() + + var dropped int + + for _, partitionIDs := range partitions { + dropped += len(partitionIDs) + for _, partitionID := range partitionIDs { + // Unassign the partition from the ingest limits instance + delete(s.assingedPartitions, partitionID) + + // Remove the partition from the metadata map + for _, partitions := range s.metadata { + delete(partitions, partitionID) + } + } + } + + if dropped > 0 { + level.Debug(s.logger).Log("msg", "removed partitions", "partitions", dropped) + } +} + func (s *IngestLimits) CheckReady(ctx context.Context) error { if s.State() != services.Running && s.State() != services.Stopping { return fmt.Errorf("ingest limits not ready: %v", s.State()) @@ -208,13 +333,15 @@ func (s *IngestLimits) running(ctx context.Context) error { default: startTime := time.Now() - fetches := s.client.PollFetches(ctx) + fetches := s.client.PollRecords(ctx, 100) if fetches.IsClientClosed() { return nil } if errs := fetches.Errors(); len(errs) > 0 { - level.Error(s.logger).Log("msg", "error fetching records", "err", errs) + for _, err := range errs { + level.Error(s.logger).Log("msg", "error fetching records", "err", err.Err.Error()) + } continue } @@ -234,7 +361,19 @@ func (s *IngestLimits) running(ctx context.Context) error { continue } - s.updateMetadata(metadata, string(record.Key), record.Timestamp) + partitionID, err := s.partitionRing.PartitionRing().ActivePartitionForKey(metadata.RingToken) + if err != nil { + level.Error(s.logger).Log("msg", "error getting active partition", "err", err, "token", metadata.RingToken) + continue + } + + evict := false + if partitionID != record.Partition { + level.Debug(s.logger).Log("msg", "skipping record from other instance", "token", metadata.RingToken, "ownerPartition", partitionID, "recordPartition", record.Partition) + evict = true + } + + s.updateMetadata(metadata, string(record.Key), record.Partition, record.Timestamp, evict) } s.metrics.kafkaReadBytesTotal.Add(float64(sizeBytes)) @@ -255,35 +394,28 @@ func (s *IngestLimits) evictOldStreams(ctx context.Context) { return case <-ticker.C: s.mtx.Lock() + cutoff := time.Now().Add(-s.cfg.WindowSize).UnixNano() - for tenant, streams := range s.metadata { - streamsBefore := len(streams) + + for tenant, partitions := range s.metadata { evictedCount := 0 - activeCount := 0 - for hash, lastSeen := range streams { - if lastSeen < cutoff { - delete(s.metadata[tenant], hash) - evictedCount++ - } else { - activeCount++ + + for partitionID, streams := range partitions { + for i, stream := range streams { + if stream.lastSeenAt < cutoff { + s.metadata[tenant][partitionID] = append(s.metadata[tenant][partitionID][:i], s.metadata[tenant][partitionID][i+1:]...) + evictedCount++ + } } } - // Update eviction counter if any streams were evicted - if evictedCount > 0 { - s.metrics.tenantStreamEvictionsTotal.WithLabelValues(tenant).Add(float64(evictedCount)) - } + // Clean up empty tenant maps and update gauges if len(s.metadata[tenant]) == 0 { delete(s.metadata, tenant) - s.metrics.tenantCurrentRecordedStreams.DeleteLabelValues(tenant) - s.metrics.tenantActiveStreams.DeleteLabelValues(tenant) - } else { - if len(streams) != streamsBefore { - // Only update recorded streams gauge if the number changed - s.metrics.tenantCurrentRecordedStreams.WithLabelValues(tenant).Set(float64(len(streams))) - } - // Always update active streams as they can change even if total count doesn't - s.metrics.tenantActiveStreams.WithLabelValues(tenant).Set(float64(activeCount)) + } + // Only update recorded streams gauge if the number changed + if evictedCount > 0 { + s.metrics.tenantStreamEvictionsTotal.WithLabelValues(tenant).Add(float64(evictedCount)) } } s.mtx.Unlock() @@ -293,33 +425,46 @@ func (s *IngestLimits) evictOldStreams(ctx context.Context) { // updateMetadata updates the metadata map with the provided StreamMetadata. // It uses the provided lastSeenAt timestamp as the last seen time. -func (s *IngestLimits) updateMetadata(metadata *logproto.StreamMetadata, tenant string, lastSeenAt time.Time) { +func (s *IngestLimits) updateMetadata(rec *logproto.StreamMetadata, tenant string, partition int32, lastSeenAt time.Time, evict bool) { s.mtx.Lock() defer s.mtx.Unlock() // Initialize tenant map if it doesn't exist if _, ok := s.metadata[tenant]; !ok { - s.metadata[tenant] = make(map[uint64]int64) + s.metadata[tenant] = make(map[int32][]streamMetadata) } - // Use the provided lastSeenAt timestamp as the last seen time - recordTime := lastSeenAt.UnixNano() - if current, ok := s.metadata[tenant][metadata.StreamHash]; !ok || recordTime > current { - s.metadata[tenant][metadata.StreamHash] = recordTime - - // Count active streams (within window) - cutoff := time.Now().Add(-s.cfg.WindowSize).UnixNano() - activeCount := 0 - for _, lastSeen := range s.metadata[tenant] { - if lastSeen >= cutoff { - activeCount++ + if s.metadata[tenant][partition] == nil { + s.metadata[tenant][partition] = make([]streamMetadata, 0) + } + + // Stream moved partition, evict it + if evict { + for i, stream := range s.metadata[tenant][partition] { + if stream.hash == rec.StreamHash { + s.metadata[tenant][partition] = append(s.metadata[tenant][partition][:i], s.metadata[tenant][partition][i+1:]...) + break } } - // Update gauges - s.metrics.tenantCurrentRecordedStreams.WithLabelValues(tenant).Set(float64(len(s.metadata[tenant]))) - s.metrics.tenantActiveStreams.WithLabelValues(tenant).Set(float64(activeCount)) + return + } + + // Use the provided lastSeenAt timestamp as the last seen time + recordTime := lastSeenAt.UnixNano() + + for i, stream := range s.metadata[tenant][partition] { + if stream.hash == rec.StreamHash { + stream.lastSeenAt = recordTime + s.metadata[tenant][partition][i] = stream + return + } } + + s.metadata[tenant][partition] = append(s.metadata[tenant][partition], streamMetadata{ + hash: rec.StreamHash, + lastSeenAt: recordTime, + }) } // stopping implements the Service interface's stopping method. @@ -343,7 +488,7 @@ func (s *IngestLimits) stopping(failureCase error) error { // ServeHTTP implements the http.Handler interface. // It returns the current stream counts and status per tenant as a JSON response. -func (s *IngestLimits) ServeHTTP(w http.ResponseWriter, _ *http.Request) { +func (s *IngestLimits) ServeHTTP(w http.ResponseWriter, r *http.Request) { s.mtx.RLock() defer s.mtx.RUnlock() @@ -354,35 +499,82 @@ func (s *IngestLimits) ServeHTTP(w http.ResponseWriter, _ *http.Request) { type tenantLimits struct { Tenant string `json:"tenant"` ActiveStreams uint64 `json:"activeStreams"` - RecordedStreams []uint64 `json:"recordedStreams"` + AssignedStreams []uint64 `json:"assignedStreams"` } - response := make(map[string]tenantLimits) - for tenant, streams := range s.metadata { - var activeStreams uint64 - recordedStreams := make([]uint64, 0, len(streams)) + // Get tenant and partitions from query parameters + params := r.URL.Query() + tenant := params.Get("tenant") + partitionsStr := params.Get("partitions") + + var requestedPartitions []int32 + if partitionsStr != "" { + // Split comma-separated partition list + partitionStrs := strings.Split(partitionsStr, ",") + requestedPartitions = make([]int32, 0, len(partitionStrs)) + + // Convert each partition string to int32 + for _, p := range partitionStrs { + if val, err := strconv.ParseInt(strings.TrimSpace(p), 10, 32); err == nil { + requestedPartitions = append(requestedPartitions, int32(val)) + } + } + } - // Count active streams and record their status - for hash, lastSeen := range streams { - isActive := lastSeen >= cutoff - if isActive { - activeStreams++ + partitions := s.metadata[tenant] + + var ( + activeStreams uint64 + assignedStreams = make([]uint64, 0) + response = make(map[string]tenantLimits) + ) + + for _, requestedID := range requestedPartitions { + // Consider the recorded stream if it's partition + // is one of the partitions we are still assigned to. + assigned := false + for assignedID := range partitions { + if requestedID == assignedID { + assigned = true + break } - recordedStreams = append(recordedStreams, hash) } - if activeStreams > 0 || len(recordedStreams) > 0 { - response[tenant] = tenantLimits{ - Tenant: tenant, - ActiveStreams: activeStreams, - RecordedStreams: recordedStreams, + if !assigned { + continue + } + + // If the stream is written into a partition we are + // assigned to and has been seen within the window, + // it is an active stream. + for _, stream := range partitions[requestedID] { + if stream.lastSeenAt >= cutoff { + activeStreams++ + assignedStreams = append(assignedStreams, stream.hash) } } } + if activeStreams > 0 || len(assignedStreams) > 0 { + response[tenant] = tenantLimits{ + Tenant: tenant, + ActiveStreams: activeStreams, + AssignedStreams: assignedStreams, + } + } + util.WriteJSONResponse(w, response) } +// GetAssignedPartitions implements the logproto.IngestLimitsServer interface. +// It returns the partitions that the tenant is assigned to and the instance still owns. +func (s *IngestLimits) GetAssignedPartitions(_ context.Context, _ *logproto.GetAssignedPartitionsRequest) (*logproto.GetAssignedPartitionsResponse, error) { + s.mtxAssingedPartitions.RLock() + defer s.mtxAssingedPartitions.RUnlock() + + return &logproto.GetAssignedPartitionsResponse{AssignedPartitions: s.assingedPartitions}, nil +} + // GetStreamUsage implements the logproto.IngestLimitsServer interface. // It returns the number of active streams for a tenant and the status of requested streams. func (s *IngestLimits) GetStreamUsage(_ context.Context, req *logproto.GetStreamUsageRequest) (*logproto.GetStreamUsageResponse, error) { @@ -393,8 +585,8 @@ func (s *IngestLimits) GetStreamUsage(_ context.Context, req *logproto.GetStream cutoff := time.Now().Add(-s.cfg.WindowSize).UnixNano() // Get the tenant's streams - streams := s.metadata[req.Tenant] - if streams == nil { + partitions := s.metadata[req.Tenant] + if partitions == nil { // If tenant not found, return zero active streams and all requested streams as not recorded return &logproto.GetStreamUsageResponse{ Tenant: req.Tenant, @@ -406,22 +598,55 @@ func (s *IngestLimits) GetStreamUsage(_ context.Context, req *logproto.GetStream // across all assigned partitions and record // the streams that have been seen within the // window - var ( - activeStreams uint64 - recordedStreams = make([]*logproto.RecordedStreams, 0, len(streams)) - ) - for hash, lastSeen := range streams { - if lastSeen >= cutoff { - activeStreams++ - recordedStreams = append(recordedStreams, &logproto.RecordedStreams{ - StreamHash: hash, - }) + var activeStreams uint64 + for _, requestedID := range req.Partitions { + // Consider the recorded stream if it's partition + // is one of the partitions we are still assigned to. + assigned := false + for assignedID := range partitions { + if requestedID == assignedID { + assigned = true + break + } + } + + if !assigned { + continue + } + + // If the stream is written into a partition we are + // assigned to and has been seen within the window, + // it is an active stream. + for _, stream := range partitions[requestedID] { + if stream.lastSeenAt >= cutoff { + activeStreams++ + } + } + } + + // Get the unknown streams + var unknownStreams []uint64 + for _, reqHash := range req.StreamHashes { + found := false + + outer: + for _, streams := range partitions { + for _, stream := range streams { + if stream.hash == reqHash && stream.lastSeenAt >= cutoff { + found = true + break outer + } + } + } + + if !found { + unknownStreams = append(unknownStreams, reqHash) } } return &logproto.GetStreamUsageResponse{ - Tenant: req.Tenant, - ActiveStreams: activeStreams, - RecordedStreams: recordedStreams, + Tenant: req.Tenant, + ActiveStreams: activeStreams, + UnknownStreams: unknownStreams, }, nil } diff --git a/pkg/limits/ingest_limits_test.go b/pkg/limits/ingest_limits_test.go index bb9f25fedcbd7..785c54b519eec 100644 --- a/pkg/limits/ingest_limits_test.go +++ b/pkg/limits/ingest_limits_test.go @@ -15,84 +15,195 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" ) +type mockPartitionRing struct { + ring.PartitionRingReader +} + +func (m *mockPartitionRing) PartitionRing() *ring.PartitionRing { + return ring.NewPartitionRing(ring.PartitionRingDesc{ + Partitions: map[int32]ring.PartitionDesc{ + 0: {Id: 0, Tokens: []uint32{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}}, + }, + }) +} + func TestIngestLimits_GetStreamUsage(t *testing.T) { tests := []struct { - name string - tenant string - setupMetadata map[string]map[uint64]int64 - windowSize time.Duration - expectedActive uint64 - expectedRecordedStreams []uint64 + name string + tenant string + partitions []int32 + streamHashes []uint64 + setupMetadata map[string]map[int32][]streamMetadata + windowSize time.Duration + expectedActive uint64 + expectedUnknownStreams []uint64 + assignedPartitions map[int32]int64 }{ { - name: "tenant not found", - tenant: "tenant1", - setupMetadata: map[string]map[uint64]int64{ + name: "tenant not found", + tenant: "tenant1", + partitions: []int32{0}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{4, 5}, + setupMetadata: map[string]map[int32][]streamMetadata{ "tenant2": { - 4: time.Now().UnixNano(), - 5: time.Now().UnixNano(), + 0: []streamMetadata{ + {hash: 4, lastSeenAt: time.Now().UnixNano()}, + {hash: 5, lastSeenAt: time.Now().UnixNano()}, + }, }, }, - windowSize: time.Hour, - expectedActive: 0, - expectedRecordedStreams: []uint64{}, + windowSize: time.Hour, + expectedActive: 0, }, { - name: "all streams active", - tenant: "tenant1", - setupMetadata: map[string]map[uint64]int64{ + name: "all streams active", + tenant: "tenant1", + partitions: []int32{0}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{1, 2, 3, 4}, + setupMetadata: map[string]map[int32][]streamMetadata{ "tenant1": { - 1: time.Now().UnixNano(), - 2: time.Now().UnixNano(), - 3: time.Now().UnixNano(), - 4: time.Now().UnixNano(), // Additional active stream + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().UnixNano()}, + {hash: 3, lastSeenAt: time.Now().UnixNano()}, + {hash: 4, lastSeenAt: time.Now().UnixNano()}, + }, }, }, - windowSize: time.Hour, - expectedActive: 4, // Total active streams for tenant - expectedRecordedStreams: []uint64{1, 2, 3, 4}, + windowSize: time.Hour, + expectedActive: 4, }, { - name: "mixed active and expired streams", - tenant: "tenant1", - setupMetadata: map[string]map[uint64]int64{ + name: "mixed active and expired streams", + tenant: "tenant1", + partitions: []int32{0}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{1, 3, 5}, + setupMetadata: map[string]map[int32][]streamMetadata{ "tenant1": { - 1: time.Now().UnixNano(), - 2: time.Now().Add(-2 * time.Hour).UnixNano(), // expired - 3: time.Now().UnixNano(), - 4: time.Now().Add(-2 * time.Hour).UnixNano(), // expired - 5: time.Now().UnixNano(), // Additional active stream + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().Add(-2 * time.Hour).UnixNano()}, // expired + {hash: 3, lastSeenAt: time.Now().UnixNano()}, + {hash: 4, lastSeenAt: time.Now().Add(-2 * time.Hour).UnixNano()}, // expired + {hash: 5, lastSeenAt: time.Now().UnixNano()}, + }, }, }, - windowSize: time.Hour, - expectedActive: 3, // Total active streams for tenant - expectedRecordedStreams: []uint64{1, 3, 5}, + windowSize: time.Hour, + expectedActive: 3, }, { name: "all streams expired", tenant: "tenant1", - setupMetadata: map[string]map[uint64]int64{ + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + setupMetadata: map[string]map[int32][]streamMetadata{ "tenant1": { - 1: time.Now().Add(-2 * time.Hour).UnixNano(), - 2: time.Now().Add(-2 * time.Hour).UnixNano(), + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().Add(-2 * time.Hour).UnixNano()}, + {hash: 2, lastSeenAt: time.Now().Add(-2 * time.Hour).UnixNano()}, + }, }, }, - windowSize: time.Hour, - expectedActive: 0, - expectedRecordedStreams: []uint64{}, + windowSize: time.Hour, + expectedActive: 0, }, { - name: "empty stream hashes", - tenant: "tenant1", - setupMetadata: map[string]map[uint64]int64{ + name: "empty stream hashes", + tenant: "tenant1", + partitions: []int32{0}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{}, + setupMetadata: map[string]map[int32][]streamMetadata{ "tenant1": { - 1: time.Now().UnixNano(), - 2: time.Now().UnixNano(), + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().UnixNano()}, + }, }, }, - windowSize: time.Hour, - expectedActive: 2, - expectedRecordedStreams: []uint64{1, 2}, + windowSize: time.Hour, + expectedActive: 2, + }, + { + name: "unknown streams requested", + tenant: "tenant1", + partitions: []int32{0}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{6, 7, 8}, + setupMetadata: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().UnixNano()}, + {hash: 3, lastSeenAt: time.Now().UnixNano()}, + {hash: 4, lastSeenAt: time.Now().UnixNano()}, + {hash: 5, lastSeenAt: time.Now().UnixNano()}, + }, + }, + }, + windowSize: time.Hour, + expectedActive: 5, + expectedUnknownStreams: []uint64{6, 7, 8}, + }, + { + name: "multiple assigned partitions", + tenant: "tenant1", + partitions: []int32{0, 1}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + 1: time.Now().UnixNano(), + }, + streamHashes: []uint64{1, 2, 3, 4, 5}, + setupMetadata: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().UnixNano()}, + }, + 1: []streamMetadata{ + {hash: 3, lastSeenAt: time.Now().UnixNano()}, + {hash: 4, lastSeenAt: time.Now().UnixNano()}, + {hash: 5, lastSeenAt: time.Now().UnixNano()}, + }, + }, + }, + windowSize: time.Hour, + expectedActive: 5, + }, + { + name: "multiple partitions with unasigned partitions", + tenant: "tenant1", + partitions: []int32{0, 1}, + assignedPartitions: map[int32]int64{ + 0: time.Now().UnixNano(), + }, + streamHashes: []uint64{1, 2, 3, 4, 5}, + setupMetadata: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: []streamMetadata{ + {hash: 1, lastSeenAt: time.Now().UnixNano()}, + {hash: 2, lastSeenAt: time.Now().UnixNano()}, + }, + }, + }, + windowSize: time.Hour, + expectedActive: 2, + expectedUnknownStreams: []uint64{3, 4, 5}, }, } @@ -117,13 +228,18 @@ func TestIngestLimits_GetStreamUsage(t *testing.T) { ObservePeriod: 100 * time.Millisecond, }, }, - logger: log.NewNopLogger(), - metadata: tt.setupMetadata, + partitionRing: &mockPartitionRing{}, + logger: log.NewNopLogger(), + metrics: newMetrics(prometheus.NewRegistry()), + metadata: tt.setupMetadata, + assingedPartitions: tt.assignedPartitions, } // Create request req := &logproto.GetStreamUsageRequest{ - Tenant: tt.tenant, + Tenant: tt.tenant, + Partitions: tt.partitions, + StreamHashes: tt.streamHashes, } // Call GetStreamUsage @@ -132,7 +248,7 @@ func TestIngestLimits_GetStreamUsage(t *testing.T) { require.NotNil(t, resp) require.Equal(t, tt.tenant, resp.Tenant) require.Equal(t, tt.expectedActive, resp.ActiveStreams) - require.Len(t, resp.RecordedStreams, len(tt.expectedRecordedStreams)) + require.Len(t, resp.UnknownStreams, len(tt.expectedUnknownStreams)) }) } } @@ -140,13 +256,15 @@ func TestIngestLimits_GetStreamUsage(t *testing.T) { func TestIngestLimits_GetStreamUsage_Concurrent(t *testing.T) { // Setup test data with a mix of active and expired streams now := time.Now() - metadata := map[string]map[uint64]int64{ + metadata := map[string]map[int32][]streamMetadata{ "tenant1": { - 1: now.UnixNano(), // active - 2: now.Add(-30 * time.Minute).UnixNano(), // active - 3: now.Add(-2 * time.Hour).UnixNano(), // expired - 4: now.Add(-45 * time.Minute).UnixNano(), // active - 5: now.Add(-3 * time.Hour).UnixNano(), // expired + 0: []streamMetadata{ + {hash: 1, lastSeenAt: now.UnixNano()}, // active + {hash: 2, lastSeenAt: now.Add(-30 * time.Minute).UnixNano()}, // active + {hash: 3, lastSeenAt: now.Add(-2 * time.Hour).UnixNano()}, // expired + {hash: 4, lastSeenAt: now.Add(-45 * time.Minute).UnixNano()}, // active + {hash: 5, lastSeenAt: now.Add(-3 * time.Hour).UnixNano()}, // expired + }, }, } @@ -168,8 +286,10 @@ func TestIngestLimits_GetStreamUsage_Concurrent(t *testing.T) { ObservePeriod: 100 * time.Millisecond, }, }, - logger: log.NewNopLogger(), - metadata: metadata, + partitionRing: &mockPartitionRing{}, + logger: log.NewNopLogger(), + metadata: metadata, + metrics: newMetrics(prometheus.NewRegistry()), } // Run concurrent requests @@ -180,7 +300,9 @@ func TestIngestLimits_GetStreamUsage_Concurrent(t *testing.T) { defer func() { done <- struct{}{} }() req := &logproto.GetStreamUsageRequest{ - Tenant: "tenant1", + Tenant: "tenant1", + Partitions: []int32{0}, + StreamHashes: []uint64{1, 2, 3, 4, 5}, } resp, err := s.GetStreamUsage(context.Background(), req) @@ -197,6 +319,127 @@ func TestIngestLimits_GetStreamUsage_Concurrent(t *testing.T) { } } +func TestIngestLimits_UpdateMetadata(t *testing.T) { + tests := []struct { + name string + tenant string + partition int32 + metadata *logproto.StreamMetadata + lastSeenAt time.Time + evict bool + existingData map[string]map[int32][]streamMetadata + expectedData map[string]map[int32][]streamMetadata + }{ + { + name: "new tenant, new partition", + tenant: "tenant1", + partition: 0, + metadata: &logproto.StreamMetadata{ + StreamHash: 123, + }, + lastSeenAt: time.Unix(100, 0), + evict: false, + existingData: map[string]map[int32][]streamMetadata{}, + expectedData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(100, 0).UnixNano()}, + }, + }, + }, + }, + { + name: "existing tenant, new partition", + tenant: "tenant1", + partition: 1, + metadata: &logproto.StreamMetadata{ + StreamHash: 456, + }, + lastSeenAt: time.Unix(200, 0), + evict: false, + existingData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(100, 0).UnixNano()}, + }, + }, + }, + expectedData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(100, 0).UnixNano()}, + }, + 1: { + {hash: 456, lastSeenAt: time.Unix(200, 0).UnixNano()}, + }, + }, + }, + }, + { + name: "update existing stream", + tenant: "tenant1", + partition: 0, + metadata: &logproto.StreamMetadata{ + StreamHash: 123, + }, + lastSeenAt: time.Unix(300, 0), + evict: false, + existingData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(100, 0).UnixNano()}, + }, + }, + }, + expectedData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(300, 0).UnixNano()}, + }, + }, + }, + }, + { + name: "evict stream from partition", + tenant: "tenant1", + partition: 0, + metadata: &logproto.StreamMetadata{ + StreamHash: 123, + }, + lastSeenAt: time.Unix(400, 0), + evict: true, + existingData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 123, lastSeenAt: time.Unix(100, 0).UnixNano()}, + {hash: 456, lastSeenAt: time.Unix(200, 0).UnixNano()}, + }, + }, + }, + expectedData: map[string]map[int32][]streamMetadata{ + "tenant1": { + 0: { + {hash: 456, lastSeenAt: time.Unix(200, 0).UnixNano()}, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + s := &IngestLimits{ + metadata: tt.existingData, + metrics: newMetrics(prometheus.NewRegistry()), + } + + s.updateMetadata(tt.metadata, tt.tenant, tt.partition, tt.lastSeenAt, tt.evict) + + require.Equal(t, tt.expectedData, s.metadata) + }) + } +} + func TestNewIngestLimits(t *testing.T) { cfg := Config{ KafkaConfig: kafka.Config{ @@ -219,7 +462,7 @@ func TestNewIngestLimits(t *testing.T) { }, } - s, err := NewIngestLimits(cfg, log.NewNopLogger(), prometheus.NewRegistry()) + s, err := NewIngestLimits(cfg, &mockPartitionRing{}, log.NewNopLogger(), prometheus.NewRegistry()) require.NoError(t, err) require.NotNil(t, s) require.NotNil(t, s.client) diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index 6f34c4e713a66..dd7b451edae51 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -231,6 +231,7 @@ func (m *StreamRatesResponse) GetStreamRates() []*StreamRate { type StreamMetadata struct { StreamHash uint64 `protobuf:"varint,1,opt,name=streamHash,proto3" json:"streamHash,omitempty"` + RingToken uint32 `protobuf:"varint,2,opt,name=ringToken,proto3" json:"ringToken,omitempty"` } func (m *StreamMetadata) Reset() { *m = StreamMetadata{} } @@ -272,6 +273,13 @@ func (m *StreamMetadata) GetStreamHash() uint64 { return 0 } +func (m *StreamMetadata) GetRingToken() uint32 { + if m != nil { + return m.RingToken + } + return 0 +} + type ExceedsLimitsRequest struct { Tenant string `protobuf:"bytes,1,opt,name=tenant,proto3" json:"tenant,omitempty"` Streams []*StreamMetadataWithSize `protobuf:"bytes,2,rep,name=streams,proto3" json:"streams,omitempty"` @@ -477,7 +485,9 @@ func (m *RejectedStream) GetReason() string { } type GetStreamUsageRequest struct { - Tenant string `protobuf:"bytes,1,opt,name=tenant,proto3" json:"tenant,omitempty"` + Tenant string `protobuf:"bytes,1,opt,name=tenant,proto3" json:"tenant,omitempty"` + Partitions []int32 `protobuf:"varint,2,rep,packed,name=partitions,proto3" json:"partitions,omitempty"` + StreamHashes []uint64 `protobuf:"varint,3,rep,packed,name=streamHashes,proto3" json:"streamHashes,omitempty"` } func (m *GetStreamUsageRequest) Reset() { *m = GetStreamUsageRequest{} } @@ -519,10 +529,24 @@ func (m *GetStreamUsageRequest) GetTenant() string { return "" } +func (m *GetStreamUsageRequest) GetPartitions() []int32 { + if m != nil { + return m.Partitions + } + return nil +} + +func (m *GetStreamUsageRequest) GetStreamHashes() []uint64 { + if m != nil { + return m.StreamHashes + } + return nil +} + type GetStreamUsageResponse struct { - Tenant string `protobuf:"bytes,1,opt,name=tenant,proto3" json:"tenant,omitempty"` - ActiveStreams uint64 `protobuf:"varint,2,opt,name=activeStreams,proto3" json:"activeStreams,omitempty"` - RecordedStreams []*RecordedStreams `protobuf:"bytes,3,rep,name=recordedStreams,proto3" json:"recordedStreams,omitempty"` + Tenant string `protobuf:"bytes,1,opt,name=tenant,proto3" json:"tenant,omitempty"` + ActiveStreams uint64 `protobuf:"varint,2,opt,name=activeStreams,proto3" json:"activeStreams,omitempty"` + UnknownStreams []uint64 `protobuf:"varint,3,rep,packed,name=unknownStreams,proto3" json:"unknownStreams,omitempty"` } func (m *GetStreamUsageResponse) Reset() { *m = GetStreamUsageResponse{} } @@ -571,56 +595,13 @@ func (m *GetStreamUsageResponse) GetActiveStreams() uint64 { return 0 } -func (m *GetStreamUsageResponse) GetRecordedStreams() []*RecordedStreams { +func (m *GetStreamUsageResponse) GetUnknownStreams() []uint64 { if m != nil { - return m.RecordedStreams + return m.UnknownStreams } return nil } -type RecordedStreams struct { - StreamHash uint64 `protobuf:"varint,1,opt,name=streamHash,proto3" json:"streamHash,omitempty"` -} - -func (m *RecordedStreams) Reset() { *m = RecordedStreams{} } -func (*RecordedStreams) ProtoMessage() {} -func (*RecordedStreams) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{11} -} -func (m *RecordedStreams) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RecordedStreams) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_RecordedStreams.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 *RecordedStreams) XXX_Merge(src proto.Message) { - xxx_messageInfo_RecordedStreams.Merge(m, src) -} -func (m *RecordedStreams) XXX_Size() int { - return m.Size() -} -func (m *RecordedStreams) XXX_DiscardUnknown() { - xxx_messageInfo_RecordedStreams.DiscardUnknown(m) -} - -var xxx_messageInfo_RecordedStreams proto.InternalMessageInfo - -func (m *RecordedStreams) GetStreamHash() uint64 { - if m != nil { - return m.StreamHash - } - return 0 -} - type StreamRate struct { StreamHash uint64 `protobuf:"varint,1,opt,name=streamHash,proto3" json:"streamHash,omitempty"` StreamHashNoShard uint64 `protobuf:"varint,2,opt,name=streamHashNoShard,proto3" json:"streamHashNoShard,omitempty"` @@ -632,7 +613,7 @@ type StreamRate struct { func (m *StreamRate) Reset() { *m = StreamRate{} } func (*StreamRate) ProtoMessage() {} func (*StreamRate) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{12} + return fileDescriptor_c28a5f14f1f4c79a, []int{11} } func (m *StreamRate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -696,6 +677,84 @@ func (m *StreamRate) GetPushes() uint32 { return 0 } +type GetAssignedPartitionsRequest struct { +} + +func (m *GetAssignedPartitionsRequest) Reset() { *m = GetAssignedPartitionsRequest{} } +func (*GetAssignedPartitionsRequest) ProtoMessage() {} +func (*GetAssignedPartitionsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_c28a5f14f1f4c79a, []int{12} +} +func (m *GetAssignedPartitionsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetAssignedPartitionsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_GetAssignedPartitionsRequest.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 *GetAssignedPartitionsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetAssignedPartitionsRequest.Merge(m, src) +} +func (m *GetAssignedPartitionsRequest) XXX_Size() int { + return m.Size() +} +func (m *GetAssignedPartitionsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetAssignedPartitionsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetAssignedPartitionsRequest proto.InternalMessageInfo + +type GetAssignedPartitionsResponse struct { + AssignedPartitions map[int32]int64 `protobuf:"bytes,1,rep,name=assignedPartitions,proto3" json:"assignedPartitions,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` +} + +func (m *GetAssignedPartitionsResponse) Reset() { *m = GetAssignedPartitionsResponse{} } +func (*GetAssignedPartitionsResponse) ProtoMessage() {} +func (*GetAssignedPartitionsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_c28a5f14f1f4c79a, []int{13} +} +func (m *GetAssignedPartitionsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetAssignedPartitionsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_GetAssignedPartitionsResponse.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 *GetAssignedPartitionsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetAssignedPartitionsResponse.Merge(m, src) +} +func (m *GetAssignedPartitionsResponse) XXX_Size() int { + return m.Size() +} +func (m *GetAssignedPartitionsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetAssignedPartitionsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetAssignedPartitionsResponse proto.InternalMessageInfo + +func (m *GetAssignedPartitionsResponse) GetAssignedPartitions() map[int32]int64 { + if m != nil { + return m.AssignedPartitions + } + return nil +} + type QueryRequest struct { Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` // Deprecated: Do not use. Limit uint32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` @@ -713,7 +772,7 @@ type QueryRequest struct { func (m *QueryRequest) Reset() { *m = QueryRequest{} } func (*QueryRequest) ProtoMessage() {} func (*QueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{13} + return fileDescriptor_c28a5f14f1f4c79a, []int{14} } func (m *QueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -814,7 +873,7 @@ type SampleQueryRequest struct { func (m *SampleQueryRequest) Reset() { *m = SampleQueryRequest{} } func (*SampleQueryRequest) ProtoMessage() {} func (*SampleQueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{14} + return fileDescriptor_c28a5f14f1f4c79a, []int{15} } func (m *SampleQueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +953,7 @@ type Plan struct { func (m *Plan) Reset() { *m = Plan{} } func (*Plan) ProtoMessage() {} func (*Plan) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{15} + return fileDescriptor_c28a5f14f1f4c79a, []int{16} } func (m *Plan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -939,7 +998,7 @@ type Delete struct { func (m *Delete) Reset() { *m = Delete{} } func (*Delete) ProtoMessage() {} func (*Delete) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{16} + return fileDescriptor_c28a5f14f1f4c79a, []int{17} } func (m *Delete) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -998,7 +1057,7 @@ type QueryResponse struct { func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{17} + return fileDescriptor_c28a5f14f1f4c79a, []int{18} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1050,7 +1109,7 @@ type SampleQueryResponse struct { func (m *SampleQueryResponse) Reset() { *m = SampleQueryResponse{} } func (*SampleQueryResponse) ProtoMessage() {} func (*SampleQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{18} + return fileDescriptor_c28a5f14f1f4c79a, []int{19} } func (m *SampleQueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1104,7 +1163,7 @@ type LabelRequest struct { func (m *LabelRequest) Reset() { *m = LabelRequest{} } func (*LabelRequest) ProtoMessage() {} func (*LabelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{19} + return fileDescriptor_c28a5f14f1f4c79a, []int{20} } func (m *LabelRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1175,7 +1234,7 @@ type LabelResponse struct { func (m *LabelResponse) Reset() { *m = LabelResponse{} } func (*LabelResponse) ProtoMessage() {} func (*LabelResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{20} + return fileDescriptor_c28a5f14f1f4c79a, []int{21} } func (m *LabelResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1220,7 +1279,7 @@ type Sample struct { func (m *Sample) Reset() { *m = Sample{} } func (*Sample) ProtoMessage() {} func (*Sample) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{21} + return fileDescriptor_c28a5f14f1f4c79a, []int{22} } func (m *Sample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1279,7 +1338,7 @@ type LegacySample struct { func (m *LegacySample) Reset() { *m = LegacySample{} } func (*LegacySample) ProtoMessage() {} func (*LegacySample) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{22} + return fileDescriptor_c28a5f14f1f4c79a, []int{23} } func (m *LegacySample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1331,7 +1390,7 @@ type Series struct { func (m *Series) Reset() { *m = Series{} } func (*Series) ProtoMessage() {} func (*Series) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{23} + return fileDescriptor_c28a5f14f1f4c79a, []int{24} } func (m *Series) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1392,7 +1451,7 @@ type TailRequest struct { func (m *TailRequest) Reset() { *m = TailRequest{} } func (*TailRequest) ProtoMessage() {} func (*TailRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{24} + return fileDescriptor_c28a5f14f1f4c79a, []int{25} } func (m *TailRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1517,7 @@ type TailResponse struct { func (m *TailResponse) Reset() { *m = TailResponse{} } func (*TailResponse) ProtoMessage() {} func (*TailResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{25} + return fileDescriptor_c28a5f14f1f4c79a, []int{26} } func (m *TailResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1504,7 +1563,7 @@ type SeriesRequest struct { func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } func (*SeriesRequest) ProtoMessage() {} func (*SeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{26} + return fileDescriptor_c28a5f14f1f4c79a, []int{27} } func (m *SeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1568,7 +1627,7 @@ type SeriesResponse struct { func (m *SeriesResponse) Reset() { *m = SeriesResponse{} } func (*SeriesResponse) ProtoMessage() {} func (*SeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{27} + return fileDescriptor_c28a5f14f1f4c79a, []int{28} } func (m *SeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1611,7 +1670,7 @@ type SeriesIdentifier struct { func (m *SeriesIdentifier) Reset() { *m = SeriesIdentifier{} } func (*SeriesIdentifier) ProtoMessage() {} func (*SeriesIdentifier) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{28} + return fileDescriptor_c28a5f14f1f4c79a, []int{29} } func (m *SeriesIdentifier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1655,7 +1714,7 @@ type SeriesIdentifier_LabelsEntry struct { func (m *SeriesIdentifier_LabelsEntry) Reset() { *m = SeriesIdentifier_LabelsEntry{} } func (*SeriesIdentifier_LabelsEntry) ProtoMessage() {} func (*SeriesIdentifier_LabelsEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{28, 0} + return fileDescriptor_c28a5f14f1f4c79a, []int{29, 0} } func (m *SeriesIdentifier_LabelsEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1707,7 +1766,7 @@ type DroppedStream struct { func (m *DroppedStream) Reset() { *m = DroppedStream{} } func (*DroppedStream) ProtoMessage() {} func (*DroppedStream) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{29} + return fileDescriptor_c28a5f14f1f4c79a, []int{30} } func (m *DroppedStream) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1765,7 +1824,7 @@ type LabelPair struct { func (m *LabelPair) Reset() { *m = LabelPair{} } func (*LabelPair) ProtoMessage() {} func (*LabelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{30} + return fileDescriptor_c28a5f14f1f4c79a, []int{31} } func (m *LabelPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1818,7 +1877,7 @@ type LegacyLabelPair struct { func (m *LegacyLabelPair) Reset() { *m = LegacyLabelPair{} } func (*LegacyLabelPair) ProtoMessage() {} func (*LegacyLabelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{31} + return fileDescriptor_c28a5f14f1f4c79a, []int{32} } func (m *LegacyLabelPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1927,7 @@ type Chunk struct { func (m *Chunk) Reset() { *m = Chunk{} } func (*Chunk) ProtoMessage() {} func (*Chunk) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{32} + return fileDescriptor_c28a5f14f1f4c79a, []int{33} } func (m *Chunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1910,7 +1969,7 @@ type TailersCountRequest struct { func (m *TailersCountRequest) Reset() { *m = TailersCountRequest{} } func (*TailersCountRequest) ProtoMessage() {} func (*TailersCountRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{33} + return fileDescriptor_c28a5f14f1f4c79a, []int{34} } func (m *TailersCountRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +2005,7 @@ type TailersCountResponse struct { func (m *TailersCountResponse) Reset() { *m = TailersCountResponse{} } func (*TailersCountResponse) ProtoMessage() {} func (*TailersCountResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{34} + return fileDescriptor_c28a5f14f1f4c79a, []int{35} } func (m *TailersCountResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1991,7 +2050,7 @@ type GetChunkIDsRequest struct { func (m *GetChunkIDsRequest) Reset() { *m = GetChunkIDsRequest{} } func (*GetChunkIDsRequest) ProtoMessage() {} func (*GetChunkIDsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{35} + return fileDescriptor_c28a5f14f1f4c79a, []int{36} } func (m *GetChunkIDsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2048,7 +2107,7 @@ type GetChunkIDsResponse struct { func (m *GetChunkIDsResponse) Reset() { *m = GetChunkIDsResponse{} } func (*GetChunkIDsResponse) ProtoMessage() {} func (*GetChunkIDsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{36} + return fileDescriptor_c28a5f14f1f4c79a, []int{37} } func (m *GetChunkIDsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2101,7 +2160,7 @@ type ChunkRef struct { func (m *ChunkRef) Reset() { *m = ChunkRef{} } func (*ChunkRef) ProtoMessage() {} func (*ChunkRef) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{37} + return fileDescriptor_c28a5f14f1f4c79a, []int{38} } func (m *ChunkRef) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2158,7 +2217,7 @@ type ChunkRefGroup struct { func (m *ChunkRefGroup) Reset() { *m = ChunkRefGroup{} } func (*ChunkRefGroup) ProtoMessage() {} func (*ChunkRefGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{38} + return fileDescriptor_c28a5f14f1f4c79a, []int{39} } func (m *ChunkRefGroup) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2205,7 +2264,7 @@ type LabelValuesForMetricNameRequest struct { func (m *LabelValuesForMetricNameRequest) Reset() { *m = LabelValuesForMetricNameRequest{} } func (*LabelValuesForMetricNameRequest) ProtoMessage() {} func (*LabelValuesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{39} + return fileDescriptor_c28a5f14f1f4c79a, []int{40} } func (m *LabelValuesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2265,7 +2324,7 @@ type LabelNamesForMetricNameRequest struct { func (m *LabelNamesForMetricNameRequest) Reset() { *m = LabelNamesForMetricNameRequest{} } func (*LabelNamesForMetricNameRequest) ProtoMessage() {} func (*LabelNamesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{40} + return fileDescriptor_c28a5f14f1f4c79a, []int{41} } func (m *LabelNamesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2316,7 +2375,7 @@ type LineFilter struct { func (m *LineFilter) Reset() { *m = LineFilter{} } func (*LineFilter) ProtoMessage() {} func (*LineFilter) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{41} + return fileDescriptor_c28a5f14f1f4c79a, []int{42} } func (m *LineFilter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2423,7 @@ type GetChunkRefRequest struct { func (m *GetChunkRefRequest) Reset() { *m = GetChunkRefRequest{} } func (*GetChunkRefRequest) ProtoMessage() {} func (*GetChunkRefRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{42} + return fileDescriptor_c28a5f14f1f4c79a, []int{43} } func (m *GetChunkRefRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2408,7 +2467,7 @@ type GetChunkRefResponse struct { func (m *GetChunkRefResponse) Reset() { *m = GetChunkRefResponse{} } func (*GetChunkRefResponse) ProtoMessage() {} func (*GetChunkRefResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{43} + return fileDescriptor_c28a5f14f1f4c79a, []int{44} } func (m *GetChunkRefResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2460,7 +2519,7 @@ type GetSeriesRequest struct { func (m *GetSeriesRequest) Reset() { *m = GetSeriesRequest{} } func (*GetSeriesRequest) ProtoMessage() {} func (*GetSeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{44} + return fileDescriptor_c28a5f14f1f4c79a, []int{45} } func (m *GetSeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2503,7 +2562,7 @@ type GetSeriesResponse struct { func (m *GetSeriesResponse) Reset() { *m = GetSeriesResponse{} } func (*GetSeriesResponse) ProtoMessage() {} func (*GetSeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{45} + return fileDescriptor_c28a5f14f1f4c79a, []int{46} } func (m *GetSeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2547,7 +2606,7 @@ type IndexSeries struct { func (m *IndexSeries) Reset() { *m = IndexSeries{} } func (*IndexSeries) ProtoMessage() {} func (*IndexSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{46} + return fileDescriptor_c28a5f14f1f4c79a, []int{47} } func (m *IndexSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2584,7 +2643,7 @@ type QueryIndexResponse struct { func (m *QueryIndexResponse) Reset() { *m = QueryIndexResponse{} } func (*QueryIndexResponse) ProtoMessage() {} func (*QueryIndexResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{47} + return fileDescriptor_c28a5f14f1f4c79a, []int{48} } func (m *QueryIndexResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2635,7 +2694,7 @@ type Row struct { func (m *Row) Reset() { *m = Row{} } func (*Row) ProtoMessage() {} func (*Row) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{48} + return fileDescriptor_c28a5f14f1f4c79a, []int{49} } func (m *Row) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2685,7 +2744,7 @@ type QueryIndexRequest struct { func (m *QueryIndexRequest) Reset() { *m = QueryIndexRequest{} } func (*QueryIndexRequest) ProtoMessage() {} func (*QueryIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{49} + return fileDescriptor_c28a5f14f1f4c79a, []int{50} } func (m *QueryIndexRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2732,7 +2791,7 @@ type IndexQuery struct { func (m *IndexQuery) Reset() { *m = IndexQuery{} } func (*IndexQuery) ProtoMessage() {} func (*IndexQuery) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{50} + return fileDescriptor_c28a5f14f1f4c79a, []int{51} } func (m *IndexQuery) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2805,7 +2864,7 @@ type IndexStatsRequest struct { func (m *IndexStatsRequest) Reset() { *m = IndexStatsRequest{} } func (*IndexStatsRequest) ProtoMessage() {} func (*IndexStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{51} + return fileDescriptor_c28a5f14f1f4c79a, []int{52} } func (m *IndexStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2851,7 +2910,7 @@ type IndexStatsResponse struct { func (m *IndexStatsResponse) Reset() { *m = IndexStatsResponse{} } func (*IndexStatsResponse) ProtoMessage() {} func (*IndexStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{52} + return fileDescriptor_c28a5f14f1f4c79a, []int{53} } func (m *IndexStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2922,7 +2981,7 @@ type VolumeRequest struct { func (m *VolumeRequest) Reset() { *m = VolumeRequest{} } func (*VolumeRequest) ProtoMessage() {} func (*VolumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{53} + return fileDescriptor_c28a5f14f1f4c79a, []int{54} } func (m *VolumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3001,7 +3060,7 @@ type VolumeResponse struct { func (m *VolumeResponse) Reset() { *m = VolumeResponse{} } func (*VolumeResponse) ProtoMessage() {} func (*VolumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{54} + return fileDescriptor_c28a5f14f1f4c79a, []int{55} } func (m *VolumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3052,7 +3111,7 @@ type Volume struct { func (m *Volume) Reset() { *m = Volume{} } func (*Volume) ProtoMessage() {} func (*Volume) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{55} + return fileDescriptor_c28a5f14f1f4c79a, []int{56} } func (m *Volume) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3109,7 +3168,7 @@ type DetectedFieldsRequest struct { func (m *DetectedFieldsRequest) Reset() { *m = DetectedFieldsRequest{} } func (*DetectedFieldsRequest) ProtoMessage() {} func (*DetectedFieldsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{56} + return fileDescriptor_c28a5f14f1f4c79a, []int{57} } func (m *DetectedFieldsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3203,7 +3262,7 @@ type DetectedFieldsResponse struct { func (m *DetectedFieldsResponse) Reset() { *m = DetectedFieldsResponse{} } func (*DetectedFieldsResponse) ProtoMessage() {} func (*DetectedFieldsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{57} + return fileDescriptor_c28a5f14f1f4c79a, []int{58} } func (m *DetectedFieldsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3266,7 +3325,7 @@ type DetectedField struct { func (m *DetectedField) Reset() { *m = DetectedField{} } func (*DetectedField) ProtoMessage() {} func (*DetectedField) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{58} + return fileDescriptor_c28a5f14f1f4c79a, []int{59} } func (m *DetectedField) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3339,7 +3398,7 @@ type DetectedLabelsRequest struct { func (m *DetectedLabelsRequest) Reset() { *m = DetectedLabelsRequest{} } func (*DetectedLabelsRequest) ProtoMessage() {} func (*DetectedLabelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{59} + return fileDescriptor_c28a5f14f1f4c79a, []int{60} } func (m *DetectedLabelsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3396,7 +3455,7 @@ type DetectedLabelsResponse struct { func (m *DetectedLabelsResponse) Reset() { *m = DetectedLabelsResponse{} } func (*DetectedLabelsResponse) ProtoMessage() {} func (*DetectedLabelsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{60} + return fileDescriptor_c28a5f14f1f4c79a, []int{61} } func (m *DetectedLabelsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3441,7 +3500,7 @@ type DetectedLabel struct { func (m *DetectedLabel) Reset() { *m = DetectedLabel{} } func (*DetectedLabel) ProtoMessage() {} func (*DetectedLabel) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{61} + return fileDescriptor_c28a5f14f1f4c79a, []int{62} } func (m *DetectedLabel) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3505,8 +3564,10 @@ func init() { proto.RegisterType((*RejectedStream)(nil), "logproto.RejectedStream") proto.RegisterType((*GetStreamUsageRequest)(nil), "logproto.GetStreamUsageRequest") proto.RegisterType((*GetStreamUsageResponse)(nil), "logproto.GetStreamUsageResponse") - proto.RegisterType((*RecordedStreams)(nil), "logproto.RecordedStreams") proto.RegisterType((*StreamRate)(nil), "logproto.StreamRate") + proto.RegisterType((*GetAssignedPartitionsRequest)(nil), "logproto.GetAssignedPartitionsRequest") + proto.RegisterType((*GetAssignedPartitionsResponse)(nil), "logproto.GetAssignedPartitionsResponse") + proto.RegisterMapType((map[int32]int64)(nil), "logproto.GetAssignedPartitionsResponse.AssignedPartitionsEntry") proto.RegisterType((*QueryRequest)(nil), "logproto.QueryRequest") proto.RegisterType((*SampleQueryRequest)(nil), "logproto.SampleQueryRequest") proto.RegisterType((*Plan)(nil), "logproto.Plan") @@ -3562,196 +3623,203 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) } var fileDescriptor_c28a5f14f1f4c79a = []byte{ - // 3019 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3a, 0xcd, 0x6f, 0x1b, 0xc7, - 0xf5, 0x5c, 0x7e, 0x89, 0x7c, 0x24, 0x25, 0x79, 0x44, 0xcb, 0xfc, 0xc9, 0x0e, 0xa9, 0x0c, 0xf2, - 0x4b, 0x94, 0xd8, 0x11, 0x6d, 0xe5, 0x97, 0xfc, 0x12, 0xa7, 0x69, 0x6a, 0x4a, 0xb6, 0x62, 0x47, - 0xfe, 0xc8, 0xc8, 0x76, 0xd2, 0xa2, 0x41, 0xb0, 0x26, 0x47, 0xe4, 0xc6, 0xe4, 0x2e, 0xbd, 0x3b, - 0xb4, 0xad, 0x9c, 0xfa, 0x0f, 0x14, 0x0d, 0x50, 0x14, 0x6d, 0x51, 0xa0, 0x40, 0x81, 0x02, 0x2d, - 0x0a, 0xe4, 0x52, 0xf4, 0xd0, 0x43, 0xd1, 0x5e, 0x0a, 0x34, 0xbd, 0xe5, 0x18, 0xe4, 0xc0, 0x36, - 0xca, 0xa5, 0x10, 0x50, 0x20, 0xa7, 0x16, 0xc8, 0xa9, 0x98, 0xaf, 0xdd, 0xd9, 0x15, 0x55, 0x45, - 0xae, 0x8b, 0x24, 0x17, 0x72, 0xe7, 0xcd, 0x9b, 0x37, 0xef, 0x6b, 0xde, 0x7b, 0xf3, 0x76, 0xe1, - 0xf8, 0xf0, 0x76, 0xb7, 0xd9, 0xf7, 0xba, 0x43, 0xdf, 0x63, 0x5e, 0xf8, 0xb0, 0x2c, 0x7e, 0x51, - 0x41, 0x8f, 0x17, 0xaa, 0x5d, 0xaf, 0xeb, 0x49, 0x1c, 0xfe, 0x24, 0xe7, 0x17, 0x1a, 0x5d, 0xcf, - 0xeb, 0xf6, 0x69, 0x53, 0x8c, 0x6e, 0x8d, 0xb6, 0x9a, 0xcc, 0x19, 0xd0, 0x80, 0xd9, 0x83, 0xa1, - 0x42, 0x58, 0x54, 0xd4, 0xef, 0xf4, 0x07, 0x5e, 0x87, 0xf6, 0x9b, 0x01, 0xb3, 0x59, 0x20, 0x7f, - 0x15, 0xc6, 0x1c, 0xc7, 0x18, 0x8e, 0x82, 0x9e, 0xf8, 0x51, 0xc0, 0xd3, 0x1c, 0x18, 0x30, 0xcf, - 0xb7, 0xbb, 0xb4, 0xd9, 0xee, 0x8d, 0xdc, 0xdb, 0xcd, 0xb6, 0xdd, 0xee, 0xd1, 0xa6, 0x4f, 0x83, - 0x51, 0x9f, 0x05, 0x72, 0xc0, 0xb6, 0x87, 0x54, 0x91, 0xc1, 0xbf, 0xb1, 0xe0, 0xe8, 0x86, 0x7d, - 0x8b, 0xf6, 0xaf, 0x7b, 0x37, 0xed, 0xfe, 0x88, 0x06, 0x84, 0x06, 0x43, 0xcf, 0x0d, 0x28, 0x5a, - 0x85, 0x7c, 0x9f, 0x4f, 0x04, 0x35, 0x6b, 0x31, 0xb3, 0x54, 0x5a, 0x39, 0xb9, 0x1c, 0x0a, 0x39, - 0x71, 0x81, 0x84, 0x06, 0xe7, 0x5d, 0xe6, 0x6f, 0x13, 0xb5, 0x74, 0xe1, 0x26, 0x94, 0x0c, 0x30, - 0x9a, 0x85, 0xcc, 0x6d, 0xba, 0x5d, 0xb3, 0x16, 0xad, 0xa5, 0x22, 0xe1, 0x8f, 0xe8, 0x0c, 0xe4, - 0xee, 0x72, 0x32, 0xb5, 0xf4, 0xa2, 0xb5, 0x54, 0x5a, 0x39, 0x1e, 0x6d, 0x72, 0xc3, 0x75, 0xee, - 0x8c, 0xa8, 0x58, 0xad, 0x36, 0x92, 0x98, 0x67, 0xd3, 0xcf, 0x5b, 0xf8, 0x24, 0x1c, 0xd9, 0x33, - 0x8f, 0xe6, 0x21, 0x2f, 0x30, 0x24, 0xc7, 0x45, 0xa2, 0x46, 0xb8, 0x0a, 0x68, 0x93, 0xf9, 0xd4, - 0x1e, 0x10, 0x9b, 0x71, 0x7e, 0xef, 0x8c, 0x68, 0xc0, 0xf0, 0x65, 0x98, 0x8b, 0x41, 0x95, 0xd8, - 0xcf, 0x41, 0x29, 0x88, 0xc0, 0x4a, 0xf6, 0x6a, 0xc4, 0x56, 0xb4, 0x86, 0x98, 0x88, 0xf8, 0x34, - 0x4c, 0xcb, 0xa9, 0xcb, 0x94, 0xd9, 0x1d, 0x9b, 0xd9, 0xa8, 0x0e, 0x20, 0x11, 0x5e, 0xb1, 0x83, - 0x9e, 0x90, 0x39, 0x4b, 0x0c, 0x08, 0x7e, 0x1b, 0xaa, 0xe7, 0xef, 0xb7, 0x29, 0xed, 0x04, 0x1b, - 0xce, 0xc0, 0x61, 0x9a, 0x31, 0x2e, 0x06, 0xa3, 0xae, 0xed, 0x32, 0xa5, 0x27, 0x35, 0x42, 0x67, - 0x61, 0x4a, 0xae, 0x0e, 0x6a, 0x69, 0xc1, 0xd5, 0x62, 0x92, 0x2b, 0xbd, 0xf5, 0xeb, 0x0e, 0xeb, - 0x6d, 0x3a, 0xef, 0x50, 0xa2, 0x17, 0xe0, 0x0d, 0x98, 0x9f, 0x8c, 0x72, 0x10, 0x97, 0x08, 0x41, - 0x36, 0x70, 0xde, 0x91, 0xf6, 0xc9, 0x12, 0xf1, 0x8c, 0x03, 0x38, 0x9a, 0xe0, 0x5c, 0x29, 0x6f, - 0x3f, 0xd6, 0x5b, 0x30, 0xe3, 0xd3, 0xb7, 0x69, 0x9b, 0xd1, 0xce, 0x66, 0x4c, 0x84, 0x5a, 0x24, - 0x02, 0x89, 0x21, 0x90, 0xe4, 0x02, 0xfc, 0x0a, 0x4c, 0xc7, 0x51, 0x0e, 0x64, 0x7d, 0x1e, 0xf2, - 0x3e, 0xb5, 0x03, 0xcf, 0x15, 0xcc, 0x17, 0x89, 0x1a, 0xe1, 0x26, 0x1c, 0x5d, 0xa7, 0x4c, 0x12, - 0xb9, 0x11, 0xd8, 0x5d, 0x7a, 0x80, 0xe6, 0xf1, 0x4f, 0x2c, 0x98, 0x4f, 0xae, 0x38, 0x40, 0xe2, - 0xc7, 0xa0, 0x62, 0xb7, 0x99, 0x73, 0x97, 0x46, 0xf2, 0x72, 0xf6, 0xe2, 0x40, 0xb4, 0xca, 0xf5, - 0xd2, 0xf6, 0xfc, 0x4e, 0xa4, 0x97, 0x8c, 0xd0, 0xcb, 0xff, 0x98, 0x7a, 0x89, 0x21, 0x90, 0xe4, - 0x0a, 0x7c, 0x06, 0x66, 0x12, 0x38, 0x07, 0xba, 0xde, 0x4f, 0x2d, 0x80, 0xc8, 0x91, 0x0f, 0x54, - 0xe4, 0x29, 0x38, 0x12, 0x8d, 0xae, 0x78, 0x9b, 0x3d, 0xdb, 0xef, 0x28, 0x81, 0xf6, 0x4e, 0x70, - 0x8f, 0xf1, 0x6d, 0x46, 0x6b, 0x99, 0x45, 0x6b, 0x29, 0x43, 0xc4, 0xb3, 0xa1, 0xa6, 0x6c, 0x4c, - 0x4d, 0xf3, 0x90, 0xe7, 0xe1, 0x8b, 0x06, 0xb5, 0xdc, 0xa2, 0xb5, 0x54, 0x21, 0x6a, 0x84, 0xff, - 0x91, 0x81, 0xf2, 0x6b, 0x23, 0xea, 0x6f, 0x6b, 0xd3, 0xd4, 0xa1, 0x10, 0xd0, 0x3e, 0x6d, 0x33, - 0xcf, 0x97, 0x9a, 0x6e, 0xa5, 0x6b, 0x16, 0x09, 0x61, 0xa8, 0x0a, 0xb9, 0x3e, 0xf7, 0x45, 0xc1, - 0x56, 0x85, 0xc8, 0x01, 0x3a, 0x0b, 0xb9, 0x80, 0xd9, 0x3e, 0x13, 0xbc, 0x94, 0x56, 0x16, 0x96, - 0x65, 0xdc, 0x5d, 0xd6, 0x71, 0x77, 0xf9, 0xba, 0x8e, 0xbb, 0xad, 0xc2, 0xfb, 0xe3, 0x46, 0xea, - 0xdd, 0xbf, 0x34, 0x2c, 0x22, 0x97, 0xa0, 0xe7, 0x20, 0x43, 0xdd, 0x8e, 0xe0, 0xf7, 0xf3, 0xae, - 0xe4, 0x0b, 0xd0, 0x19, 0x28, 0x76, 0x1c, 0x9f, 0xb6, 0x99, 0xe3, 0xb9, 0x42, 0xaa, 0xe9, 0x95, - 0xb9, 0xc8, 0x9a, 0x6b, 0x7a, 0x8a, 0x44, 0x58, 0xe8, 0x14, 0xe4, 0x03, 0xae, 0xba, 0xa0, 0x36, - 0xc5, 0x03, 0x57, 0xab, 0xba, 0x3b, 0x6e, 0xcc, 0x4a, 0xc8, 0x29, 0x6f, 0xe0, 0x30, 0x3a, 0x18, - 0xb2, 0x6d, 0xa2, 0x70, 0xd0, 0x53, 0x30, 0xd5, 0xa1, 0x7d, 0xca, 0xa3, 0x53, 0x41, 0x38, 0xcb, - 0xac, 0x41, 0x5e, 0x4c, 0x10, 0x8d, 0x80, 0xde, 0x84, 0xec, 0xb0, 0x6f, 0xbb, 0xb5, 0xa2, 0x90, - 0x62, 0x3a, 0x42, 0xbc, 0xd6, 0xb7, 0xdd, 0xd6, 0x0b, 0x1f, 0x8d, 0x1b, 0xcf, 0x76, 0x1d, 0xd6, - 0x1b, 0xdd, 0x5a, 0x6e, 0x7b, 0x83, 0x66, 0xd7, 0xb7, 0xb7, 0x6c, 0xd7, 0x6e, 0xf6, 0xbd, 0xdb, - 0x4e, 0xf3, 0xee, 0x33, 0x4d, 0x9e, 0x4d, 0xee, 0x8c, 0xa8, 0xef, 0x50, 0xbf, 0xc9, 0xc9, 0x2c, - 0x0b, 0x93, 0xf0, 0xa5, 0x44, 0x90, 0x45, 0x97, 0x78, 0xb0, 0xf4, 0x7c, 0xba, 0xca, 0x53, 0x4d, - 0x50, 0x03, 0xb1, 0xcb, 0xb1, 0x68, 0x17, 0x01, 0x27, 0x74, 0x6b, 0xdd, 0xf7, 0x46, 0xc3, 0xd6, - 0xcc, 0xee, 0xb8, 0x61, 0xe2, 0x13, 0x73, 0x70, 0x29, 0x5b, 0xc8, 0xcf, 0x4e, 0xe1, 0xf7, 0x32, - 0x80, 0x36, 0xed, 0xc1, 0xb0, 0x4f, 0x0f, 0x65, 0xfe, 0xd0, 0xd0, 0xe9, 0x07, 0x36, 0x74, 0xe6, - 0xb0, 0x86, 0x8e, 0xac, 0x96, 0x3d, 0x9c, 0xd5, 0x72, 0x9f, 0xd7, 0x6a, 0xf9, 0x2f, 0xbd, 0xd5, - 0x70, 0x0d, 0xb2, 0x9c, 0x32, 0xcf, 0xec, 0xbe, 0x7d, 0x4f, 0xd8, 0xa6, 0x4c, 0xf8, 0x23, 0xde, - 0x80, 0xbc, 0x94, 0x0b, 0x2d, 0x24, 0x8d, 0x17, 0x3f, 0xb7, 0x91, 0xe1, 0x32, 0xda, 0x24, 0xb3, - 0x91, 0x49, 0x32, 0x42, 0xd9, 0xf8, 0x77, 0x16, 0x54, 0x94, 0x47, 0xa8, 0xc8, 0x7b, 0x2b, 0x4a, - 0x87, 0x32, 0x49, 0x1f, 0x4b, 0xa6, 0xc3, 0x73, 0x1d, 0x7b, 0xc8, 0xa8, 0xdf, 0x6a, 0xbe, 0x3f, - 0x6e, 0x58, 0x1f, 0x8d, 0x1b, 0x4f, 0xec, 0xa7, 0x34, 0x5d, 0x4a, 0xe9, 0xe4, 0xae, 0x09, 0xa3, - 0x93, 0x82, 0x3b, 0x16, 0x28, 0xb7, 0x9a, 0x59, 0x96, 0x15, 0xd8, 0x45, 0xb7, 0x4b, 0x03, 0x4e, - 0x39, 0xcb, 0x3d, 0x82, 0x48, 0x1c, 0x2e, 0xe6, 0x3d, 0xdb, 0x77, 0x1d, 0xb7, 0x2b, 0xa3, 0x78, - 0x91, 0x84, 0x63, 0xfc, 0x23, 0x0b, 0xe6, 0x62, 0x6e, 0xad, 0x84, 0x78, 0x1e, 0xf2, 0x01, 0xb7, - 0x94, 0x96, 0xc1, 0x70, 0x8a, 0x4d, 0x01, 0x6f, 0x4d, 0x2b, 0xe6, 0xf3, 0x72, 0x4c, 0x14, 0xfe, - 0xc3, 0x63, 0xed, 0x8f, 0x16, 0x94, 0x45, 0x15, 0xa5, 0xcf, 0x1a, 0x82, 0xac, 0x6b, 0x0f, 0xa8, - 0x32, 0x95, 0x78, 0x36, 0x4a, 0x2b, 0xbe, 0x5d, 0x41, 0x97, 0x56, 0x87, 0x0d, 0xb0, 0xd6, 0x03, - 0x07, 0x58, 0x2b, 0x3a, 0x77, 0x55, 0xc8, 0x71, 0xf7, 0xde, 0x16, 0xc1, 0xb5, 0x48, 0xe4, 0x00, - 0x3f, 0x01, 0x15, 0x25, 0x45, 0x94, 0x99, 0x27, 0x56, 0x83, 0x03, 0xc8, 0x4b, 0x4b, 0xa0, 0xc7, - 0xa0, 0x18, 0xd6, 0xdd, 0x42, 0xda, 0x4c, 0x2b, 0xbf, 0x3b, 0x6e, 0xa4, 0x59, 0x40, 0xa2, 0x09, - 0xd4, 0x30, 0x2b, 0x54, 0xab, 0x55, 0xdc, 0x1d, 0x37, 0x24, 0x40, 0xd5, 0xa3, 0xe8, 0x04, 0x64, - 0x7b, 0x3c, 0x6f, 0x72, 0x15, 0x64, 0x5b, 0x85, 0xdd, 0x71, 0x43, 0x8c, 0x89, 0xf8, 0xc5, 0xeb, - 0x50, 0xde, 0xa0, 0x5d, 0xbb, 0xbd, 0xad, 0x36, 0xad, 0x6a, 0x72, 0x7c, 0x43, 0x4b, 0xd3, 0x78, - 0x14, 0xca, 0xe1, 0x8e, 0x6f, 0xa9, 0x6a, 0x21, 0x43, 0x4a, 0x21, 0xec, 0x72, 0x80, 0x7f, 0x6c, - 0x81, 0xf2, 0x01, 0x84, 0x8d, 0xd2, 0x9c, 0xc7, 0x42, 0xd8, 0x1d, 0x37, 0x14, 0x44, 0x57, 0xde, - 0xe8, 0x45, 0x98, 0x0a, 0xc4, 0x8e, 0xba, 0xd4, 0x32, 0x5d, 0x4b, 0x4c, 0xb4, 0x66, 0xb8, 0x8b, - 0xec, 0x8e, 0x1b, 0x1a, 0x91, 0xe8, 0x07, 0xb4, 0x1c, 0x2b, 0x08, 0xa4, 0x60, 0xd3, 0xbb, 0xe3, - 0x86, 0x01, 0x8d, 0xd5, 0x13, 0x9f, 0x59, 0x50, 0xba, 0x6e, 0x3b, 0xa1, 0x0b, 0xd5, 0xb4, 0x89, - 0xa2, 0x58, 0x2d, 0x01, 0xdc, 0x13, 0x3b, 0xb4, 0x6f, 0x6f, 0x5f, 0xf0, 0x7c, 0x41, 0xb7, 0x42, - 0xc2, 0x71, 0x94, 0xc3, 0xb3, 0x13, 0x73, 0x78, 0xee, 0xf0, 0xa1, 0xfd, 0xbf, 0x1b, 0x48, 0x2f, - 0x65, 0x0b, 0xe9, 0xd9, 0x0c, 0x7e, 0xcf, 0x82, 0xb2, 0x14, 0x5e, 0x79, 0xde, 0xb7, 0x21, 0x2f, - 0x75, 0x23, 0xc4, 0xff, 0x37, 0x81, 0xe9, 0xe4, 0x61, 0x82, 0x92, 0xa2, 0x89, 0x5e, 0x86, 0xe9, - 0x8e, 0xef, 0x0d, 0x87, 0xc9, 0x52, 0xda, 0xd8, 0x65, 0xcd, 0x9c, 0x27, 0x09, 0x74, 0xfc, 0x67, - 0x0b, 0x2a, 0x2a, 0x98, 0x28, 0x73, 0x85, 0x2a, 0xb6, 0x1e, 0x38, 0x7b, 0xa6, 0x0f, 0x9b, 0x3d, - 0xe7, 0x21, 0xdf, 0xe5, 0xf9, 0x45, 0x07, 0x24, 0x35, 0x3a, 0x5c, 0x56, 0xc5, 0x97, 0x60, 0x5a, - 0x8b, 0xb2, 0x4f, 0x44, 0x5d, 0x48, 0x46, 0xd4, 0x8b, 0x1d, 0xea, 0x32, 0x67, 0xcb, 0x09, 0x63, - 0xa4, 0xc2, 0xc7, 0xdf, 0xb3, 0x60, 0x36, 0x89, 0x82, 0xd6, 0x12, 0xb7, 0xe0, 0xc7, 0xf7, 0x27, - 0x67, 0x5e, 0x80, 0x35, 0x69, 0x75, 0x0d, 0x7e, 0xf6, 0xa0, 0x6b, 0x70, 0xd5, 0x0c, 0x32, 0x45, - 0x15, 0x15, 0xf0, 0x0f, 0x2d, 0xa8, 0xc4, 0x6c, 0x89, 0x9e, 0x87, 0xec, 0x96, 0xef, 0x0d, 0x0e, - 0x65, 0x28, 0xb1, 0x02, 0xfd, 0x1f, 0xa4, 0x99, 0x77, 0x28, 0x33, 0xa5, 0x99, 0xc7, 0xad, 0xa4, - 0xc4, 0xcf, 0xc8, 0xba, 0x5d, 0x8e, 0xf0, 0xb3, 0x50, 0x14, 0x02, 0x5d, 0xb3, 0x1d, 0x7f, 0x62, - 0xc2, 0x98, 0x2c, 0xd0, 0x8b, 0x30, 0x23, 0x83, 0xe1, 0xe4, 0xc5, 0xe5, 0x49, 0x8b, 0xcb, 0x7a, - 0xf1, 0x71, 0xc8, 0x89, 0xa2, 0x83, 0x2f, 0xe1, 0x57, 0x58, 0xbd, 0x84, 0x3f, 0xe3, 0xa3, 0x30, - 0xc7, 0xcf, 0x20, 0xf5, 0x83, 0x55, 0x6f, 0xe4, 0x32, 0x7d, 0xc9, 0x3f, 0x05, 0xd5, 0x38, 0x58, - 0x79, 0x49, 0x15, 0x72, 0x6d, 0x0e, 0x10, 0x34, 0x2a, 0x44, 0x0e, 0xf0, 0xcf, 0x2d, 0x40, 0xeb, - 0x94, 0x89, 0x5d, 0x2e, 0xae, 0x85, 0xc7, 0x63, 0x01, 0x0a, 0x03, 0x9b, 0xb5, 0x7b, 0xd4, 0x0f, - 0x74, 0xfd, 0xa2, 0xc7, 0x5f, 0x44, 0xe1, 0x89, 0xcf, 0xc0, 0x5c, 0x8c, 0x4b, 0x25, 0xd3, 0x02, - 0x14, 0xda, 0x0a, 0xa6, 0x52, 0x5e, 0x38, 0xc6, 0xbf, 0x4e, 0x43, 0x41, 0x97, 0x75, 0xe8, 0x0c, - 0x94, 0xb6, 0x1c, 0xb7, 0x4b, 0xfd, 0xa1, 0xef, 0x28, 0x15, 0x64, 0x65, 0x99, 0x67, 0x80, 0x89, - 0x39, 0x40, 0x4f, 0xc3, 0xd4, 0x28, 0xa0, 0xfe, 0x5b, 0x8e, 0x3c, 0xe9, 0xc5, 0x56, 0x75, 0x67, - 0xdc, 0xc8, 0xdf, 0x08, 0xa8, 0x7f, 0x71, 0x8d, 0x27, 0x9f, 0x91, 0x78, 0x22, 0xf2, 0xbf, 0x83, - 0x5e, 0x55, 0x6e, 0x2a, 0x0a, 0xb8, 0xd6, 0xff, 0x73, 0xf6, 0x13, 0xa1, 0x6e, 0xe8, 0x7b, 0x03, - 0xca, 0x7a, 0x74, 0x14, 0x34, 0xdb, 0xde, 0x60, 0xe0, 0xb9, 0x4d, 0xd1, 0xe8, 0x12, 0x42, 0xf3, - 0x0c, 0xca, 0x97, 0x2b, 0xcf, 0xbd, 0x0e, 0x53, 0xac, 0xe7, 0x7b, 0xa3, 0x6e, 0x4f, 0x24, 0x86, - 0x4c, 0xeb, 0xec, 0xe1, 0xe9, 0x69, 0x0a, 0x44, 0x3f, 0xa0, 0x47, 0xb9, 0xb6, 0x68, 0xfb, 0x76, - 0x30, 0x1a, 0xc8, 0xbb, 0x67, 0x2b, 0xb7, 0x3b, 0x6e, 0x58, 0x4f, 0x93, 0x10, 0x8c, 0xcf, 0x41, - 0x25, 0x56, 0x0a, 0xa3, 0xd3, 0x90, 0xf5, 0xe9, 0x96, 0x0e, 0x05, 0x68, 0x6f, 0xc5, 0x2c, 0xb3, - 0x3f, 0xc7, 0x21, 0xe2, 0x17, 0x7f, 0x37, 0x0d, 0x0d, 0xa3, 0x45, 0x75, 0xc1, 0xf3, 0x2f, 0x53, - 0xe6, 0x3b, 0xed, 0x2b, 0xf6, 0x20, 0xec, 0x3a, 0x34, 0xa0, 0x34, 0x10, 0xc0, 0xb7, 0x8c, 0x53, - 0x04, 0x83, 0x10, 0x0f, 0x3d, 0x02, 0x20, 0x8e, 0x9d, 0x9c, 0x97, 0x07, 0xaa, 0x28, 0x20, 0x62, - 0x7a, 0x35, 0xa6, 0xec, 0xe6, 0x21, 0x95, 0xa3, 0x94, 0x7c, 0x31, 0xa9, 0xe4, 0x43, 0xd3, 0x09, - 0x35, 0x6b, 0x1e, 0x97, 0x5c, 0xfc, 0xb8, 0xe0, 0xbf, 0x5b, 0x50, 0xdf, 0xd0, 0x9c, 0x3f, 0xa0, - 0x3a, 0xb4, 0xbc, 0xe9, 0x87, 0x24, 0x6f, 0xe6, 0x21, 0xca, 0x9b, 0x4d, 0xc8, 0x5b, 0x07, 0xd8, - 0x70, 0x5c, 0x7a, 0xc1, 0xe9, 0x33, 0xea, 0x4f, 0xb8, 0x24, 0x7d, 0x3f, 0x13, 0x45, 0x1c, 0x42, - 0xb7, 0xb4, 0x0e, 0x56, 0x8d, 0x30, 0xff, 0x30, 0x44, 0x4c, 0x3f, 0x44, 0x11, 0x33, 0x89, 0x08, - 0xe8, 0xc2, 0xd4, 0x96, 0x10, 0x4f, 0x66, 0xec, 0x58, 0xb3, 0x34, 0x92, 0xbd, 0xf5, 0x75, 0xb5, - 0xf9, 0x73, 0x07, 0x14, 0x5c, 0xa2, 0xe9, 0xdd, 0x0c, 0xb6, 0x5d, 0x66, 0xdf, 0x37, 0xd6, 0x13, - 0xbd, 0x09, 0xb2, 0x55, 0x4d, 0x97, 0x9b, 0x58, 0xd3, 0xbd, 0xa4, 0xb6, 0xf9, 0x4f, 0xea, 0x3a, - 0xdc, 0x8d, 0x02, 0xac, 0x30, 0x8a, 0x0a, 0xb0, 0x8f, 0x1f, 0x74, 0xfc, 0xe5, 0xa1, 0x47, 0x4b, - 0xf1, 0xab, 0x59, 0x39, 0xbc, 0x9a, 0x75, 0xe8, 0xfd, 0xd8, 0xbd, 0x0c, 0xff, 0xde, 0x82, 0xd9, - 0x75, 0xca, 0xe2, 0xd5, 0xd8, 0x57, 0xc8, 0xf8, 0xf8, 0x15, 0x38, 0x62, 0xf0, 0xaf, 0xf4, 0xf4, - 0x4c, 0xa2, 0x04, 0x3b, 0x1a, 0x69, 0x4a, 0xe8, 0x40, 0xdd, 0x6c, 0xe3, 0xd5, 0xd7, 0x35, 0x28, - 0x19, 0x93, 0xe8, 0x5c, 0xa2, 0xee, 0x9a, 0x4b, 0xbc, 0x7d, 0xe0, 0xb5, 0x43, 0xab, 0xaa, 0x64, - 0x92, 0xf7, 0x57, 0x55, 0x55, 0x87, 0x35, 0xca, 0x26, 0x20, 0x61, 0x58, 0x41, 0xd6, 0xcc, 0x92, - 0x02, 0xfa, 0x6a, 0x58, 0x80, 0x85, 0x63, 0xf4, 0x28, 0x64, 0x7d, 0xef, 0x9e, 0x2e, 0xa8, 0x2b, - 0x46, 0x0f, 0xd6, 0xbb, 0x47, 0xc4, 0x14, 0x7e, 0x11, 0x32, 0xc4, 0xbb, 0x87, 0xea, 0x00, 0xbe, - 0xed, 0x76, 0xe9, 0xcd, 0xf0, 0x2a, 0x57, 0x26, 0x06, 0x64, 0x9f, 0x0a, 0x66, 0x15, 0x8e, 0x98, - 0x1c, 0x49, 0x73, 0x2f, 0xc3, 0xd4, 0x6b, 0x23, 0x53, 0x5d, 0xd5, 0x84, 0xba, 0x64, 0xc7, 0x40, - 0x23, 0x71, 0x9f, 0x81, 0x08, 0x8e, 0x4e, 0x40, 0x91, 0xd9, 0xb7, 0xfa, 0xf4, 0x4a, 0x14, 0x2c, - 0x23, 0x00, 0x9f, 0xe5, 0xb7, 0xd0, 0x9b, 0x46, 0x29, 0x16, 0x01, 0xd0, 0x53, 0x30, 0x1b, 0xf1, - 0x7c, 0xcd, 0xa7, 0x5b, 0xce, 0x7d, 0x61, 0xe1, 0x32, 0xd9, 0x03, 0x47, 0x4b, 0x30, 0x13, 0xc1, - 0x36, 0x45, 0xc9, 0x93, 0x15, 0xa8, 0x49, 0x30, 0xd7, 0x8d, 0x10, 0xf7, 0xfc, 0x9d, 0x91, 0xdd, - 0x17, 0xc7, 0xb4, 0x4c, 0x0c, 0x08, 0xfe, 0x83, 0x05, 0x47, 0xa4, 0xa9, 0xf9, 0x19, 0xf8, 0x2a, - 0x7a, 0xfd, 0x2f, 0x2c, 0x40, 0xa6, 0x04, 0xca, 0xb5, 0xfe, 0xd7, 0xec, 0x48, 0xf1, 0x9a, 0xaa, - 0x24, 0x2e, 0xd7, 0xaa, 0x6f, 0x1f, 0x36, 0x95, 0x30, 0xe4, 0xdb, 0xb2, 0xf3, 0x26, 0x5a, 0xe8, - 0xf2, 0xf6, 0x2e, 0x21, 0x44, 0xfd, 0xa3, 0x06, 0xe4, 0x6e, 0x6d, 0x33, 0x1a, 0xa8, 0xbb, 0xb7, - 0x68, 0x3a, 0x08, 0x00, 0x91, 0x7f, 0x7c, 0x2f, 0xea, 0x32, 0xe1, 0x35, 0xd9, 0x68, 0x2f, 0x05, - 0x22, 0xfa, 0x01, 0xff, 0x33, 0x0d, 0x95, 0x9b, 0x5e, 0x7f, 0x14, 0xa5, 0xd7, 0xaf, 0x52, 0x6a, - 0x89, 0x35, 0x04, 0x72, 0xba, 0x21, 0x80, 0x20, 0x1b, 0x30, 0x3a, 0x14, 0x9e, 0x95, 0x21, 0xe2, - 0x19, 0x61, 0x28, 0x33, 0xdb, 0xef, 0x52, 0x26, 0xaf, 0x59, 0xb5, 0xbc, 0xa8, 0x7f, 0x63, 0x30, - 0xb4, 0x08, 0x25, 0xbb, 0xdb, 0xf5, 0x69, 0xd7, 0x66, 0xb4, 0xb5, 0x5d, 0x9b, 0x12, 0x9b, 0x99, - 0x20, 0x74, 0x09, 0xa6, 0xdb, 0x76, 0xbb, 0xe7, 0xb8, 0xdd, 0xab, 0x43, 0xe6, 0x78, 0x6e, 0x50, - 0x2b, 0x88, 0x08, 0x7e, 0x62, 0xd9, 0x7c, 0x7f, 0xba, 0xbc, 0x1a, 0xc3, 0x51, 0x71, 0x2c, 0xb1, - 0x12, 0xbf, 0x01, 0xd3, 0x5a, 0xf1, 0xca, 0x3d, 0x4e, 0xc3, 0xd4, 0x5d, 0x01, 0x99, 0xd0, 0xec, - 0x93, 0xa8, 0x8a, 0x94, 0x46, 0x8b, 0xbf, 0xd4, 0xd0, 0xf2, 0xe3, 0x4b, 0x90, 0x97, 0xe8, 0xe8, - 0x84, 0x79, 0xf1, 0x92, 0xb5, 0x27, 0x1f, 0xab, 0x5b, 0x14, 0x86, 0xbc, 0x24, 0xa4, 0x9c, 0x48, - 0xf8, 0x99, 0x84, 0x10, 0xf5, 0x8f, 0x7f, 0x90, 0x86, 0xa3, 0x6b, 0x94, 0x89, 0xb7, 0x6a, 0x17, - 0x1c, 0xda, 0xef, 0x7c, 0xa1, 0x3d, 0x81, 0xb0, 0xb3, 0x97, 0x31, 0x3a, 0x7b, 0x3c, 0x86, 0xf5, - 0x1d, 0x97, 0x6e, 0x18, 0xad, 0xa1, 0x08, 0x10, 0xe9, 0x28, 0x67, 0x36, 0x8d, 0xb4, 0x8f, 0xe4, - 0x0d, 0x1f, 0x89, 0x1a, 0x82, 0x53, 0xb1, 0x1e, 0xa6, 0xbe, 0x81, 0x16, 0xa2, 0xeb, 0x2b, 0xfe, - 0xad, 0x05, 0xf3, 0x49, 0xbd, 0x28, 0x33, 0x9e, 0x87, 0xfc, 0x96, 0x80, 0xec, 0x6d, 0x3b, 0xc7, - 0x56, 0xc8, 0xce, 0x85, 0x44, 0x35, 0x3b, 0x17, 0x12, 0x82, 0x9e, 0x8c, 0xbd, 0xb0, 0x6a, 0xcd, - 0xed, 0x8e, 0x1b, 0x33, 0x02, 0x60, 0xe0, 0x2a, 0x61, 0x4e, 0x85, 0x8c, 0x67, 0xa2, 0x96, 0x88, - 0x84, 0x98, 0x84, 0x55, 0x7f, 0xf3, 0x4f, 0x16, 0x54, 0x62, 0x8c, 0x08, 0x15, 0xf1, 0x23, 0xa0, - 0xd2, 0x83, 0x1c, 0xa0, 0x27, 0x21, 0xcb, 0xb6, 0x87, 0x2a, 0x2b, 0xb4, 0x8e, 0x7e, 0x36, 0x6e, - 0x1c, 0x89, 0x2d, 0xbb, 0xbe, 0x3d, 0xa4, 0x44, 0xa0, 0xf0, 0x93, 0xd3, 0xb6, 0xfd, 0x8e, 0xe3, - 0xda, 0x7d, 0x87, 0x49, 0xeb, 0x64, 0x89, 0x09, 0xe2, 0xe1, 0x68, 0x68, 0xfb, 0x81, 0x2e, 0x02, - 0x8b, 0x32, 0x1c, 0x29, 0x10, 0xd1, 0x0f, 0xa2, 0xb9, 0x73, 0x9b, 0xb2, 0x76, 0x4f, 0xa6, 0x05, - 0xd5, 0xdc, 0x11, 0x90, 0x58, 0x73, 0x47, 0x40, 0xf0, 0xcf, 0xac, 0xc8, 0x39, 0xe5, 0x19, 0xfe, - 0xd2, 0x39, 0x27, 0xfe, 0x66, 0xe4, 0x27, 0x9a, 0x45, 0xe5, 0x27, 0x2f, 0xc3, 0x74, 0x27, 0x36, - 0xb3, 0xbf, 0xbf, 0xc8, 0xc6, 0x75, 0x02, 0x1d, 0x8f, 0x22, 0x3b, 0x0a, 0xc8, 0x3e, 0x76, 0x4c, - 0x18, 0x27, 0xbd, 0xd7, 0x38, 0x91, 0xd6, 0x33, 0x07, 0x6b, 0xfd, 0xa9, 0xc7, 0xa1, 0x18, 0xbe, - 0xa4, 0x44, 0x25, 0x98, 0xba, 0x70, 0x95, 0xbc, 0x7e, 0x8e, 0xac, 0xcd, 0xa6, 0x50, 0x19, 0x0a, - 0xad, 0x73, 0xab, 0xaf, 0x8a, 0x91, 0xb5, 0xf2, 0xab, 0xbc, 0x2e, 0x5c, 0x7c, 0xf4, 0x35, 0xc8, - 0xc9, 0x6a, 0x64, 0x3e, 0x12, 0xce, 0x7c, 0x7f, 0xb7, 0x70, 0x6c, 0x0f, 0x5c, 0x6a, 0x09, 0xa7, - 0x4e, 0x5b, 0xe8, 0x0a, 0x94, 0x04, 0x50, 0x75, 0xc8, 0x4f, 0x24, 0x1b, 0xd5, 0x31, 0x4a, 0x8f, - 0xec, 0x33, 0x6b, 0xd0, 0x3b, 0x0b, 0x39, 0xa9, 0xb0, 0xf9, 0x44, 0xd1, 0x38, 0x81, 0x9b, 0xd8, - 0x3b, 0x03, 0x9c, 0x42, 0x2f, 0x40, 0xf6, 0xba, 0xed, 0xf4, 0x91, 0x51, 0xb3, 0x1a, 0x8d, 0xed, - 0x85, 0xf9, 0x24, 0xd8, 0xd8, 0xf6, 0xa5, 0xb0, 0x3f, 0x7f, 0x2c, 0xd9, 0x24, 0xd4, 0xcb, 0x6b, - 0x7b, 0x27, 0xc2, 0x9d, 0xaf, 0xca, 0x2e, 0xb2, 0x6e, 0x55, 0xa1, 0x47, 0xe2, 0x5b, 0x25, 0x3a, - 0x5b, 0x0b, 0xf5, 0xfd, 0xa6, 0x43, 0x82, 0x1b, 0x50, 0x32, 0xda, 0x44, 0xa6, 0x5a, 0xf7, 0xf6, - 0xb8, 0x4c, 0xb5, 0x4e, 0xe8, 0x2d, 0xe1, 0x14, 0x5a, 0x87, 0x82, 0xf8, 0x04, 0xc2, 0x66, 0x01, - 0x3a, 0x9e, 0x2c, 0xe8, 0x8d, 0x42, 0x6e, 0xe1, 0xc4, 0xe4, 0xc9, 0x90, 0xd0, 0x37, 0xa0, 0xb8, - 0x4e, 0x99, 0xca, 0x60, 0xc7, 0x92, 0x29, 0x70, 0x82, 0xa6, 0xe2, 0x69, 0x14, 0xa7, 0xd0, 0x1b, - 0xe2, 0xd2, 0x11, 0x0f, 0xcf, 0xa8, 0xb1, 0x4f, 0x18, 0x0e, 0xf9, 0x5a, 0xdc, 0x1f, 0x21, 0xa4, - 0xfc, 0x7a, 0x8c, 0xb2, 0xaa, 0x1b, 0x1a, 0xfb, 0x1c, 0xd8, 0x90, 0x72, 0xe3, 0x80, 0x2f, 0xa3, - 0x70, 0x6a, 0xe5, 0x4d, 0xfd, 0xbd, 0xc5, 0x9a, 0xcd, 0x6c, 0x74, 0x15, 0xa6, 0xc3, 0xcf, 0x49, - 0xc4, 0xd7, 0x43, 0x31, 0x9f, 0xdf, 0xf3, 0xa9, 0x52, 0xcc, 0xe7, 0xf7, 0x7e, 0xb2, 0x84, 0x53, - 0x2b, 0x6f, 0x43, 0x55, 0xbe, 0xf8, 0x93, 0xdf, 0xe3, 0x5c, 0xf0, 0x3d, 0x97, 0xf1, 0x98, 0x45, - 0xa0, 0x12, 0xfb, 0x50, 0x07, 0x19, 0x5e, 0x33, 0xe9, 0xdb, 0x23, 0x53, 0x94, 0x89, 0x5f, 0xf8, - 0xe0, 0xd4, 0x0a, 0x85, 0xb2, 0xb9, 0x17, 0xba, 0x61, 0x08, 0x23, 0xbe, 0x8d, 0x31, 0x15, 0x36, - 0xf1, 0x3b, 0x1b, 0xd3, 0x14, 0x93, 0x3f, 0xab, 0xc1, 0xa9, 0xd6, 0x9b, 0x1f, 0x7c, 0x5c, 0x4f, - 0x7d, 0xf8, 0x71, 0x3d, 0xf5, 0xe9, 0xc7, 0x75, 0xeb, 0x3b, 0x3b, 0x75, 0xeb, 0x97, 0x3b, 0x75, - 0xeb, 0xfd, 0x9d, 0xba, 0xf5, 0xc1, 0x4e, 0xdd, 0xfa, 0xeb, 0x4e, 0xdd, 0xfa, 0xdb, 0x4e, 0x3d, - 0xf5, 0xe9, 0x4e, 0xdd, 0x7a, 0xf7, 0x93, 0x7a, 0xea, 0x83, 0x4f, 0xea, 0xa9, 0x0f, 0x3f, 0xa9, - 0xa7, 0xbe, 0xf5, 0xc4, 0xc1, 0xdd, 0x05, 0x19, 0xe9, 0xf3, 0xe2, 0xef, 0x99, 0x7f, 0x05, 0x00, - 0x00, 0xff, 0xff, 0x75, 0x54, 0x18, 0x22, 0xc7, 0x27, 0x00, 0x00, + // 3125 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x1a, 0x4d, 0x6f, 0x1b, 0xc7, + 0x95, 0xcb, 0x2f, 0x91, 0x8f, 0x94, 0x2c, 0x8f, 0x64, 0x99, 0x90, 0x65, 0x52, 0x19, 0xa4, 0xb6, + 0x13, 0x3b, 0xa2, 0xad, 0x34, 0x69, 0xe2, 0x34, 0x4d, 0x4d, 0xc9, 0x56, 0xec, 0xc8, 0x1f, 0x19, + 0xc9, 0x4e, 0x5a, 0x34, 0x08, 0xd6, 0xe4, 0x88, 0x5a, 0x8b, 0xdc, 0xa5, 0x77, 0x87, 0xb6, 0x95, + 0x43, 0xd1, 0x3f, 0x50, 0x34, 0x40, 0x51, 0xb4, 0xbd, 0x14, 0x28, 0x50, 0xa0, 0x45, 0x81, 0x5c, + 0x8a, 0x1e, 0x7a, 0x28, 0xda, 0x4b, 0x81, 0xa6, 0xb7, 0xf4, 0xd4, 0x20, 0x07, 0xb6, 0x51, 0x2e, + 0x85, 0x80, 0x02, 0x39, 0xb5, 0x40, 0x4e, 0xc5, 0x7c, 0xed, 0xce, 0xae, 0xc8, 0x28, 0x72, 0x5d, + 0x24, 0xb9, 0x90, 0x33, 0x6f, 0xde, 0xbc, 0x99, 0xf7, 0x31, 0xef, 0xbd, 0x79, 0x3b, 0x70, 0xac, + 0xb7, 0xd5, 0xae, 0x77, 0xbc, 0x76, 0xcf, 0xf7, 0x98, 0x17, 0x36, 0x16, 0xc4, 0x2f, 0x2a, 0xe8, + 0xfe, 0xec, 0x74, 0xdb, 0x6b, 0x7b, 0x12, 0x87, 0xb7, 0xe4, 0xf8, 0x6c, 0xad, 0xed, 0x79, 0xed, + 0x0e, 0xad, 0x8b, 0xde, 0xed, 0xfe, 0x46, 0x9d, 0x39, 0x5d, 0x1a, 0x30, 0xbb, 0xdb, 0x53, 0x08, + 0xf3, 0x8a, 0xfa, 0xdd, 0x4e, 0xd7, 0x6b, 0xd1, 0x4e, 0x3d, 0x60, 0x36, 0x0b, 0xe4, 0xaf, 0xc2, + 0x98, 0xe2, 0x18, 0xbd, 0x7e, 0xb0, 0x29, 0x7e, 0x14, 0xf0, 0x2c, 0x07, 0x06, 0xcc, 0xf3, 0xed, + 0x36, 0xad, 0x37, 0x37, 0xfb, 0xee, 0x56, 0xbd, 0x69, 0x37, 0x37, 0x69, 0xdd, 0xa7, 0x41, 0xbf, + 0xc3, 0x02, 0xd9, 0x61, 0xdb, 0x3d, 0xaa, 0xc8, 0xe0, 0xdf, 0x5a, 0x70, 0x64, 0xd5, 0xbe, 0x4d, + 0x3b, 0xeb, 0xde, 0x2d, 0xbb, 0xd3, 0xa7, 0x01, 0xa1, 0x41, 0xcf, 0x73, 0x03, 0x8a, 0x96, 0x20, + 0xdf, 0xe1, 0x03, 0x41, 0xc5, 0x9a, 0xcf, 0x9c, 0x2a, 0x2d, 0x9e, 0x5e, 0x08, 0x99, 0x1c, 0x3a, + 0x41, 0x42, 0x83, 0x8b, 0x2e, 0xf3, 0xb7, 0x89, 0x9a, 0x3a, 0x7b, 0x0b, 0x4a, 0x06, 0x18, 0x4d, + 0x42, 0x66, 0x8b, 0x6e, 0x57, 0xac, 0x79, 0xeb, 0x54, 0x91, 0xf0, 0x26, 0x3a, 0x07, 0xb9, 0x7b, + 0x9c, 0x4c, 0x25, 0x3d, 0x6f, 0x9d, 0x2a, 0x2d, 0x1e, 0x8b, 0x16, 0xb9, 0xe9, 0x3a, 0x77, 0xfb, + 0x54, 0xcc, 0x56, 0x0b, 0x49, 0xcc, 0xf3, 0xe9, 0xe7, 0x2c, 0x7c, 0x1a, 0x0e, 0xef, 0x19, 0x47, + 0x33, 0x90, 0x17, 0x18, 0x72, 0xc7, 0x45, 0xa2, 0x7a, 0x78, 0x1a, 0xd0, 0x1a, 0xf3, 0xa9, 0xdd, + 0x25, 0x36, 0xe3, 0xfb, 0xbd, 0xdb, 0xa7, 0x01, 0xc3, 0x57, 0x61, 0x2a, 0x06, 0x55, 0x6c, 0x3f, + 0x0b, 0xa5, 0x20, 0x02, 0x2b, 0xde, 0xa7, 0xa3, 0x6d, 0x45, 0x73, 0x88, 0x89, 0x88, 0xaf, 0xc1, + 0x84, 0x1c, 0xba, 0x4a, 0x99, 0xdd, 0xb2, 0x99, 0x8d, 0xaa, 0x00, 0x12, 0xe1, 0x65, 0x3b, 0xd8, + 0x14, 0x3c, 0x67, 0x89, 0x01, 0x41, 0x73, 0x50, 0xf4, 0x1d, 0xb7, 0xbd, 0xee, 0x6d, 0x51, 0x57, + 0xb0, 0x3f, 0x4e, 0x22, 0x00, 0xbe, 0x03, 0xd3, 0x17, 0x1f, 0x34, 0x29, 0x6d, 0x05, 0xab, 0x4e, + 0xd7, 0x61, 0x7a, 0xdb, 0x9c, 0x49, 0x46, 0x5d, 0xdb, 0x65, 0x4a, 0x8a, 0xaa, 0x87, 0xce, 0xc3, + 0x98, 0xa4, 0x1d, 0x54, 0xd2, 0x62, 0xcf, 0xf3, 0xc9, 0x3d, 0xeb, 0x8d, 0xbd, 0xe6, 0xb0, 0xcd, + 0x35, 0xe7, 0x2d, 0x4a, 0xf4, 0x04, 0xbc, 0x0a, 0x33, 0xc3, 0x51, 0xf6, 0xe5, 0x01, 0x41, 0x36, + 0x70, 0xde, 0x92, 0xda, 0xcb, 0x12, 0xd1, 0xc6, 0x01, 0x1c, 0x49, 0xec, 0x5c, 0x89, 0x76, 0xd4, + 0xd6, 0x1b, 0x70, 0xc8, 0xa7, 0x77, 0x68, 0x93, 0xd1, 0xd6, 0x5a, 0x8c, 0x85, 0x4a, 0xc4, 0x02, + 0x89, 0x21, 0x90, 0xe4, 0x04, 0xfc, 0x32, 0x4c, 0xc4, 0x51, 0xf6, 0xdd, 0xfa, 0x0c, 0xe4, 0x7d, + 0x6a, 0x07, 0x9e, 0x94, 0x7d, 0x91, 0xa8, 0x1e, 0xdf, 0xfe, 0x0a, 0x65, 0x92, 0xc8, 0xcd, 0xc0, + 0x6e, 0xd3, 0xfd, 0x24, 0x5f, 0x05, 0xe8, 0xd9, 0x3e, 0x73, 0x98, 0xe3, 0xb9, 0x72, 0xe7, 0x39, + 0x62, 0x40, 0x10, 0x86, 0x72, 0xb4, 0x2c, 0x0d, 0x2a, 0x99, 0xf9, 0xcc, 0xa9, 0x2c, 0x89, 0xc1, + 0xf0, 0x77, 0x61, 0x26, 0xb9, 0xe8, 0x3e, 0x42, 0x7b, 0x1c, 0xc6, 0xed, 0x26, 0x73, 0xee, 0xd1, + 0x48, 0x64, 0x9c, 0xc3, 0x38, 0x10, 0x9d, 0x80, 0x89, 0xbe, 0xbb, 0xe5, 0x7a, 0xf7, 0x5d, 0x8d, + 0x26, 0x57, 0x4f, 0x40, 0xf1, 0xcf, 0x2c, 0x80, 0xc8, 0xb2, 0xf7, 0x95, 0xdd, 0x19, 0x38, 0x1c, + 0xf5, 0xae, 0x79, 0x6b, 0x9b, 0xb6, 0xdf, 0x52, 0x1b, 0xd8, 0x3b, 0xc0, 0x8d, 0xc4, 0xb7, 0x19, + 0xad, 0x64, 0xe6, 0xad, 0x53, 0x19, 0x22, 0xda, 0x06, 0x5b, 0xd9, 0x18, 0x5b, 0x33, 0x90, 0xe7, + 0xfe, 0x8c, 0x06, 0x95, 0x9c, 0x38, 0x11, 0xaa, 0x87, 0xab, 0x30, 0xb7, 0x42, 0xd9, 0x85, 0x20, + 0x70, 0xda, 0x2e, 0x6d, 0xdd, 0x08, 0xa5, 0xab, 0x4f, 0xf3, 0xdf, 0x2c, 0x38, 0x3e, 0x02, 0x41, + 0x09, 0xd2, 0x03, 0x64, 0xef, 0x19, 0x55, 0xe7, 0xfb, 0xa5, 0xc8, 0xd0, 0x3e, 0x95, 0xc8, 0xc2, + 0xde, 0x21, 0xe9, 0xef, 0x86, 0x90, 0x9e, 0xbd, 0x08, 0x47, 0x47, 0xa0, 0x9b, 0x7e, 0x30, 0x27, + 0xfd, 0xe0, 0xb4, 0xe9, 0x07, 0x33, 0xa6, 0xab, 0xfb, 0x77, 0x06, 0xca, 0xaf, 0xf6, 0xa9, 0xbf, + 0xad, 0xed, 0xb0, 0x0a, 0x85, 0x80, 0x76, 0x68, 0x93, 0x79, 0xbe, 0xb4, 0x89, 0x46, 0xba, 0x62, + 0x91, 0x10, 0xc6, 0x49, 0x75, 0xf8, 0xc1, 0x53, 0x3e, 0x45, 0x76, 0xd0, 0x79, 0xc8, 0x05, 0xcc, + 0xf6, 0x99, 0xd0, 0x42, 0x69, 0x71, 0x76, 0x41, 0x86, 0xa0, 0x05, 0x1d, 0x82, 0x16, 0xd6, 0x75, + 0x08, 0x6a, 0x14, 0xde, 0x1d, 0xd4, 0x52, 0x6f, 0xff, 0xbd, 0x66, 0x11, 0x39, 0x05, 0x3d, 0x0b, + 0x19, 0xea, 0xb6, 0x84, 0xa6, 0x3e, 0xeb, 0x4c, 0x3e, 0x01, 0x9d, 0x83, 0x62, 0xcb, 0xf1, 0x69, + 0x93, 0x73, 0x2e, 0xf4, 0x39, 0xb1, 0x38, 0x15, 0x49, 0x7a, 0x59, 0x0f, 0x91, 0x08, 0x0b, 0x9d, + 0x81, 0x7c, 0xc0, 0x8d, 0x26, 0xa8, 0x8c, 0x71, 0x1f, 0xde, 0x98, 0xde, 0x1d, 0xd4, 0x26, 0x25, + 0xe4, 0x8c, 0xd7, 0x75, 0x18, 0xed, 0xf6, 0xd8, 0x36, 0x51, 0x38, 0xe8, 0x49, 0x18, 0x6b, 0xd1, + 0x0e, 0xe5, 0x8e, 0xba, 0x20, 0x14, 0x39, 0x69, 0x90, 0x17, 0x03, 0x44, 0x23, 0xa0, 0x37, 0x20, + 0xdb, 0xeb, 0xd8, 0x6e, 0xa5, 0x28, 0xb8, 0x98, 0x88, 0x10, 0x6f, 0x74, 0x6c, 0xb7, 0xf1, 0xfc, + 0x07, 0x83, 0xda, 0x33, 0x6d, 0x87, 0x6d, 0xf6, 0x6f, 0x2f, 0x34, 0xbd, 0x6e, 0xbd, 0xed, 0xdb, + 0x1b, 0xb6, 0x6b, 0xd7, 0x3b, 0xde, 0x96, 0x53, 0xbf, 0xf7, 0x74, 0x9d, 0x07, 0xd6, 0xbb, 0x7d, + 0xea, 0x3b, 0xd4, 0xaf, 0x73, 0x32, 0x0b, 0x42, 0x25, 0x7c, 0x2a, 0x11, 0x64, 0xd1, 0x15, 0x1e, + 0x37, 0x3c, 0x9f, 0x2e, 0xf1, 0xa8, 0x1b, 0x54, 0x40, 0xac, 0x72, 0x34, 0x5a, 0x45, 0xc0, 0x09, + 0xdd, 0x58, 0xf1, 0xbd, 0x7e, 0xaf, 0x71, 0x68, 0x77, 0x50, 0x33, 0xf1, 0x89, 0xd9, 0xb9, 0x92, + 0x2d, 0xe4, 0x27, 0xc7, 0xf0, 0x3b, 0x19, 0x40, 0x6b, 0x76, 0xb7, 0xd7, 0xa1, 0x07, 0x52, 0x7f, + 0xa8, 0xe8, 0xf4, 0x43, 0x2b, 0x3a, 0x73, 0x50, 0x45, 0x47, 0x5a, 0xcb, 0x1e, 0x4c, 0x6b, 0xb9, + 0xcf, 0xaa, 0xb5, 0xfc, 0x17, 0x5e, 0x6b, 0xb8, 0x02, 0x59, 0x4e, 0x99, 0x1f, 0x6e, 0xdf, 0xbe, + 0x2f, 0x74, 0x53, 0x26, 0xbc, 0x89, 0x57, 0x21, 0x2f, 0xf9, 0x42, 0xb3, 0x49, 0xe5, 0xc5, 0xcf, + 0x6d, 0xa4, 0xb8, 0x8c, 0x56, 0xc9, 0x64, 0xa4, 0x92, 0x8c, 0x10, 0x36, 0xfe, 0xbd, 0x05, 0xe3, + 0xca, 0x22, 0x94, 0x6b, 0xbb, 0x1d, 0xc5, 0x7e, 0xe9, 0xcf, 0x8e, 0x26, 0x63, 0xff, 0x85, 0x96, + 0xdd, 0x63, 0xd4, 0x6f, 0xd4, 0xdf, 0x1d, 0xd4, 0xac, 0x0f, 0x06, 0xb5, 0x93, 0xa3, 0x84, 0xa6, + 0xb3, 0x4a, 0x9d, 0xe7, 0x68, 0xc2, 0xe8, 0xb4, 0xd8, 0x1d, 0x0b, 0x94, 0x59, 0x1d, 0x5a, 0x90, + 0xc9, 0xe8, 0x65, 0xb7, 0x4d, 0x03, 0x4e, 0x39, 0xcb, 0x2d, 0x82, 0x48, 0x1c, 0xce, 0xe6, 0x7d, + 0xdb, 0x77, 0x1d, 0xb7, 0x2d, 0x03, 0x4e, 0x91, 0x84, 0x7d, 0xfc, 0x13, 0x0b, 0xa6, 0x62, 0x66, + 0xad, 0x98, 0x78, 0x0e, 0xf2, 0x01, 0xd7, 0x94, 0xe6, 0xc1, 0x30, 0x8a, 0x35, 0x01, 0x6f, 0x4c, + 0xa8, 0xcd, 0xe7, 0x65, 0x9f, 0x28, 0xfc, 0x47, 0xb7, 0xb5, 0x3f, 0x59, 0x50, 0x16, 0x09, 0xa5, + 0x3e, 0x6b, 0x08, 0xb2, 0xae, 0xdd, 0xa5, 0x4a, 0x55, 0xa2, 0x6d, 0x64, 0x99, 0x7c, 0xb9, 0x82, + 0xce, 0x32, 0x0f, 0xea, 0x60, 0xad, 0x87, 0x76, 0xb0, 0x56, 0x74, 0xee, 0xa6, 0x21, 0xc7, 0xcd, + 0x7b, 0x5b, 0x38, 0xd7, 0x22, 0x91, 0x1d, 0x7c, 0x12, 0xc6, 0x15, 0x17, 0x51, 0x0e, 0x31, 0x34, + 0x31, 0xee, 0x42, 0x5e, 0x6a, 0x02, 0x3d, 0x0e, 0xc5, 0xf0, 0x0a, 0x22, 0xb8, 0xcd, 0x34, 0xf2, + 0xbb, 0x83, 0x5a, 0x9a, 0x05, 0x24, 0x1a, 0x40, 0x35, 0x33, 0x48, 0x59, 0x8d, 0xe2, 0xee, 0xa0, + 0x26, 0x01, 0x2a, 0x5e, 0xa1, 0x39, 0xc8, 0x6e, 0xf2, 0x8c, 0x81, 0x8b, 0x20, 0xdb, 0x28, 0xec, + 0x0e, 0x6a, 0xa2, 0x4f, 0xc4, 0x2f, 0x5e, 0x81, 0xf2, 0x2a, 0x6d, 0xdb, 0xcd, 0x6d, 0xb5, 0x68, + 0x18, 0xf3, 0xf8, 0x82, 0x96, 0xa6, 0xf1, 0x18, 0x94, 0xc3, 0x15, 0xdf, 0x54, 0x79, 0x4d, 0x86, + 0x94, 0x42, 0xd8, 0xd5, 0x00, 0xff, 0xd4, 0x02, 0x65, 0x03, 0x08, 0x1b, 0xb7, 0x14, 0xee, 0x0b, + 0x61, 0x77, 0x50, 0x53, 0x10, 0x7d, 0x09, 0x41, 0x2f, 0xc0, 0x58, 0x20, 0x56, 0xd4, 0x79, 0xa5, + 0x69, 0x5a, 0x62, 0xa0, 0x71, 0x88, 0x9b, 0xc8, 0xee, 0xa0, 0xa6, 0x11, 0x89, 0x6e, 0xa0, 0x85, + 0x58, 0x2a, 0x24, 0x19, 0x9b, 0xd8, 0x1d, 0xd4, 0x0c, 0xa8, 0x99, 0x1a, 0xe1, 0x4f, 0x2c, 0x28, + 0xad, 0xdb, 0x4e, 0x68, 0x42, 0x15, 0xad, 0xa2, 0xc8, 0x57, 0x4b, 0x00, 0xb7, 0xc4, 0x16, 0xed, + 0xd8, 0xdb, 0x97, 0x3c, 0x5f, 0xd0, 0x1d, 0x27, 0x61, 0x3f, 0x8a, 0xe1, 0xd9, 0xa1, 0x31, 0x3c, + 0x77, 0x70, 0xd7, 0xfe, 0xff, 0x75, 0xa4, 0x57, 0xb2, 0x85, 0xf4, 0x64, 0x06, 0xbf, 0x63, 0x41, + 0x59, 0x32, 0xaf, 0x2c, 0xef, 0x3b, 0x90, 0x97, 0xb2, 0x11, 0xec, 0x7f, 0x8a, 0x63, 0x3a, 0x7d, + 0x10, 0xa7, 0xa4, 0x68, 0xa2, 0x97, 0x60, 0xa2, 0xe5, 0x7b, 0xbd, 0x5e, 0xf2, 0xde, 0x60, 0xac, + 0xb2, 0x6c, 0x8e, 0x93, 0x04, 0x3a, 0xfe, 0x8b, 0x05, 0xe3, 0xca, 0x99, 0x28, 0x75, 0x85, 0x22, + 0xb6, 0x1e, 0x3a, 0x7a, 0xa6, 0x0f, 0x1a, 0x3d, 0x67, 0x20, 0xdf, 0xe6, 0xf1, 0x45, 0x3b, 0x24, + 0xd5, 0x3b, 0x58, 0x54, 0xc5, 0x57, 0x60, 0x42, 0xb3, 0x32, 0xc2, 0xa3, 0xce, 0x26, 0x3d, 0xea, + 0xe5, 0x16, 0x75, 0x99, 0xb3, 0xe1, 0x84, 0x3e, 0x52, 0xe1, 0xe3, 0x1f, 0x58, 0x30, 0x99, 0x44, + 0x41, 0xcb, 0x89, 0x82, 0xc0, 0x89, 0xd1, 0xe4, 0xcc, 0x5a, 0x80, 0x26, 0xad, 0x2a, 0x02, 0xcf, + 0xec, 0x57, 0x11, 0x88, 0x65, 0xc2, 0x45, 0xe5, 0x15, 0xf0, 0x8f, 0x2d, 0x18, 0x8f, 0xe9, 0x12, + 0x3d, 0x07, 0xd9, 0x0d, 0xdf, 0xeb, 0x1e, 0x48, 0x51, 0x62, 0x06, 0xfa, 0x2a, 0xa4, 0x99, 0x77, + 0x20, 0x35, 0xa5, 0x99, 0xc7, 0xb5, 0xa4, 0xd8, 0xcf, 0xc8, 0x1b, 0x8b, 0xec, 0xe1, 0x67, 0xa0, + 0x28, 0x18, 0xba, 0x61, 0x3b, 0xfe, 0xd0, 0x80, 0x31, 0x9c, 0xa1, 0x17, 0xe0, 0x90, 0x74, 0x86, + 0xc3, 0x27, 0x97, 0x87, 0x4d, 0x2e, 0xeb, 0xc9, 0xc7, 0x20, 0x27, 0x92, 0x0e, 0x3e, 0x85, 0xdf, + 0xd7, 0xf5, 0x14, 0xde, 0xc6, 0x47, 0x60, 0x8a, 0x9f, 0x41, 0xea, 0x07, 0x4b, 0x5e, 0xdf, 0x65, + 0xfa, 0x86, 0x74, 0x06, 0xa6, 0xe3, 0x60, 0x65, 0x25, 0xd3, 0x90, 0x6b, 0x72, 0x80, 0xa0, 0x31, + 0x4e, 0x64, 0x07, 0xff, 0xc2, 0x02, 0xb4, 0x42, 0x99, 0x58, 0xe5, 0xf2, 0x72, 0x78, 0x3c, 0x66, + 0xa1, 0xd0, 0xb5, 0x59, 0x73, 0x93, 0xfa, 0x81, 0xce, 0x5f, 0x74, 0xff, 0xf3, 0x48, 0x3c, 0xf1, + 0x39, 0x98, 0x8a, 0xed, 0x52, 0xf1, 0x34, 0x0b, 0x85, 0xa6, 0x82, 0xa9, 0x90, 0x17, 0xf6, 0xf1, + 0x6f, 0xd2, 0x50, 0xd0, 0x69, 0x1d, 0x3a, 0x07, 0xa5, 0x0d, 0xc7, 0x6d, 0x53, 0xbf, 0xe7, 0x3b, + 0x4a, 0x04, 0x59, 0x99, 0xe6, 0x19, 0x60, 0x62, 0x76, 0xd0, 0x53, 0x30, 0xd6, 0x0f, 0xa8, 0xff, + 0xa6, 0x23, 0x4f, 0x7a, 0xb1, 0x31, 0xbd, 0x33, 0xa8, 0xe5, 0x6f, 0x06, 0xd4, 0xbf, 0xbc, 0xcc, + 0x83, 0x4f, 0x5f, 0xb4, 0x88, 0xfc, 0x6f, 0xa1, 0x57, 0x94, 0x99, 0x8a, 0x04, 0xae, 0xf1, 0x35, + 0xbe, 0xfd, 0x84, 0xab, 0xeb, 0xf9, 0x5e, 0x97, 0xb2, 0x4d, 0xda, 0x0f, 0xea, 0x4d, 0xaf, 0xdb, + 0xf5, 0xdc, 0xba, 0xa8, 0xf9, 0x09, 0xa6, 0x79, 0x04, 0xe5, 0xd3, 0x95, 0xe5, 0xae, 0xc3, 0x18, + 0xdb, 0xf4, 0xbd, 0x7e, 0x7b, 0x53, 0x04, 0x86, 0x4c, 0xe3, 0xfc, 0xc1, 0xe9, 0x69, 0x0a, 0x44, + 0x37, 0xd0, 0x63, 0x5c, 0x5a, 0xb4, 0xb9, 0x15, 0xf4, 0xbb, 0xf2, 0xd6, 0xdd, 0xc8, 0xed, 0x0e, + 0x6a, 0xd6, 0x53, 0x24, 0x04, 0xe3, 0x0b, 0x30, 0x1e, 0x4b, 0x85, 0xd1, 0x59, 0xc8, 0xfa, 0x74, + 0x43, 0xbb, 0x02, 0xb4, 0x37, 0x63, 0x96, 0xd1, 0x9f, 0xe3, 0x10, 0xf1, 0x8b, 0xbf, 0x9f, 0x86, + 0x9a, 0x51, 0xad, 0xbb, 0xe4, 0xf9, 0x57, 0x29, 0xf3, 0x9d, 0xe6, 0x35, 0xbb, 0x1b, 0x96, 0x58, + 0x6a, 0x50, 0xea, 0x0a, 0xe0, 0x9b, 0xc6, 0x29, 0x82, 0x6e, 0x88, 0x87, 0x8e, 0x03, 0x88, 0x63, + 0x27, 0xc7, 0xe5, 0x81, 0x2a, 0x0a, 0x88, 0x18, 0x5e, 0x8a, 0x09, 0xbb, 0x7e, 0x40, 0xe1, 0x28, + 0x21, 0x5f, 0x4e, 0x0a, 0xf9, 0xc0, 0x74, 0x42, 0xc9, 0x9a, 0xc7, 0x25, 0x17, 0x3f, 0x2e, 0xf8, + 0x5f, 0x16, 0x54, 0x57, 0xf5, 0xce, 0x1f, 0x52, 0x1c, 0x9a, 0xdf, 0xf4, 0x23, 0xe2, 0x37, 0xf3, + 0x08, 0xf9, 0xcd, 0x26, 0xf8, 0xad, 0x02, 0xac, 0x3a, 0x2e, 0xbd, 0xe4, 0x74, 0x18, 0xf5, 0x87, + 0x5c, 0x92, 0x7e, 0x98, 0x89, 0x3c, 0x0e, 0xa1, 0x1b, 0x5a, 0x06, 0x4b, 0x86, 0x9b, 0x7f, 0x14, + 0x2c, 0xa6, 0x1f, 0x21, 0x8b, 0x99, 0x84, 0x07, 0x74, 0x61, 0x6c, 0x43, 0xb0, 0x27, 0x23, 0x76, + 0xac, 0x6e, 0x1c, 0xf1, 0xde, 0xf8, 0x86, 0x5a, 0xfc, 0xd9, 0x7d, 0x12, 0x2e, 0x51, 0xff, 0xaf, + 0x07, 0xdb, 0x2e, 0xb3, 0x1f, 0x18, 0xf3, 0x89, 0x5e, 0x04, 0xd9, 0x2a, 0xa7, 0xcb, 0x0d, 0xcd, + 0xe9, 0x5e, 0x54, 0xcb, 0xfc, 0x2f, 0x79, 0x1d, 0x6e, 0x47, 0x0e, 0x56, 0x28, 0x45, 0x39, 0xd8, + 0x13, 0xfb, 0x1d, 0x7f, 0x79, 0xe8, 0xd1, 0xa9, 0xf8, 0xd5, 0xac, 0x1c, 0x5e, 0xcd, 0x5a, 0xf4, + 0x41, 0xec, 0x5e, 0x86, 0xff, 0x60, 0xc1, 0xe4, 0x0a, 0x65, 0xf1, 0x6c, 0xec, 0x4b, 0xa4, 0x7c, + 0xfc, 0x32, 0x1c, 0x36, 0xf6, 0xaf, 0xe4, 0xf4, 0x74, 0x22, 0x05, 0x3b, 0x12, 0x49, 0x4a, 0xc8, + 0x40, 0xdd, 0x6c, 0xe3, 0xd9, 0xd7, 0x0d, 0x28, 0x19, 0x83, 0xe8, 0x42, 0x22, 0xef, 0x9a, 0x4a, + 0x7c, 0x88, 0xe1, 0xb9, 0x43, 0x63, 0x5a, 0xf1, 0x24, 0xef, 0xaf, 0x2a, 0xab, 0x0e, 0x73, 0x94, + 0x35, 0x40, 0x42, 0xb1, 0x82, 0xac, 0x19, 0x25, 0x05, 0xf4, 0x95, 0x30, 0x01, 0x0b, 0xfb, 0xe8, + 0x31, 0xc8, 0xfa, 0xde, 0x7d, 0x9d, 0x50, 0x8f, 0x1b, 0x85, 0x78, 0xef, 0x3e, 0x11, 0x43, 0xf8, + 0x05, 0xc8, 0x10, 0xef, 0x3e, 0xaa, 0x02, 0xf8, 0xb6, 0xdb, 0xa6, 0xb7, 0xc2, 0xab, 0x5c, 0x99, + 0x18, 0x90, 0x11, 0x19, 0xcc, 0x12, 0x1c, 0x36, 0x77, 0x24, 0xd5, 0xbd, 0x00, 0x63, 0xaf, 0xf6, + 0x4d, 0x71, 0x4d, 0x27, 0xc4, 0x25, 0x2b, 0x06, 0x1a, 0x89, 0xdb, 0x0c, 0x44, 0x70, 0x34, 0x07, + 0x45, 0x66, 0xdf, 0xee, 0xd0, 0x6b, 0x91, 0xb3, 0x8c, 0x00, 0x7c, 0x94, 0xdf, 0x42, 0x6f, 0x19, + 0xa9, 0x58, 0x04, 0x40, 0x4f, 0xc2, 0x64, 0xb4, 0xe7, 0x1b, 0x3e, 0xdd, 0x70, 0x1e, 0x08, 0x0d, + 0x97, 0xc9, 0x1e, 0x38, 0x3a, 0x05, 0x87, 0x22, 0xd8, 0x9a, 0x48, 0x79, 0xb2, 0x02, 0x35, 0x09, + 0xe6, 0xb2, 0x11, 0xec, 0x5e, 0xbc, 0xdb, 0xb7, 0x3b, 0xe2, 0x98, 0x96, 0x89, 0x01, 0xc1, 0x7f, + 0xb4, 0xe0, 0xb0, 0x54, 0x35, 0x3f, 0x03, 0x5f, 0x46, 0xab, 0xff, 0xa5, 0x05, 0xc8, 0xe4, 0x40, + 0x99, 0xd6, 0x57, 0xcc, 0x8a, 0x14, 0xcf, 0xa9, 0x4a, 0xe2, 0x72, 0x2d, 0x41, 0x51, 0x51, 0x09, + 0x43, 0xbe, 0x29, 0x2b, 0x6f, 0xe2, 0xe3, 0x81, 0xbc, 0xbd, 0x4b, 0x08, 0x51, 0xff, 0xa8, 0x06, + 0xb9, 0xdb, 0xdb, 0x8c, 0x06, 0xea, 0xee, 0x2d, 0x8a, 0x0e, 0x02, 0x40, 0xe4, 0x1f, 0x5f, 0x8b, + 0xba, 0x4c, 0x58, 0x4d, 0x36, 0x5a, 0x4b, 0x81, 0x88, 0x6e, 0xe0, 0xff, 0xa4, 0x61, 0xfc, 0x96, + 0xd7, 0xe9, 0x47, 0xe1, 0xf5, 0xcb, 0x14, 0x5a, 0x62, 0x05, 0x81, 0x9c, 0x2e, 0x08, 0x20, 0xc8, + 0x06, 0x8c, 0xf6, 0x84, 0x65, 0x65, 0x88, 0x68, 0x23, 0x0c, 0x65, 0x66, 0xfb, 0x6d, 0xca, 0xe4, + 0x35, 0xab, 0x92, 0x17, 0xf9, 0x6f, 0x0c, 0x86, 0xe6, 0xa1, 0x64, 0xb7, 0xdb, 0x3e, 0x6d, 0xdb, + 0x8c, 0x36, 0xb6, 0x2b, 0x63, 0x62, 0x31, 0x13, 0x84, 0xae, 0xc0, 0x44, 0xd3, 0x6e, 0x6e, 0x3a, + 0x6e, 0xfb, 0x7a, 0x4f, 0x7e, 0x29, 0x29, 0x08, 0x0f, 0x3e, 0xb7, 0x60, 0x7e, 0x4a, 0x5e, 0x58, + 0x8a, 0xe1, 0x28, 0x3f, 0x96, 0x98, 0x89, 0x5f, 0x87, 0x09, 0x2d, 0x78, 0x65, 0x1e, 0x67, 0x61, + 0xec, 0x9e, 0x80, 0x0c, 0x29, 0xf6, 0x49, 0x54, 0x45, 0x4a, 0xa3, 0xc5, 0x3f, 0x6a, 0x68, 0xfe, + 0xf1, 0x15, 0xc8, 0x4b, 0x74, 0x34, 0x67, 0x5e, 0xbc, 0x64, 0xee, 0xc9, 0xfb, 0xea, 0x16, 0x85, + 0x21, 0x2f, 0x09, 0x29, 0x23, 0x12, 0x76, 0x26, 0x21, 0x44, 0xfd, 0xe3, 0x1f, 0xa5, 0xe1, 0xc8, + 0x32, 0x65, 0xe2, 0x13, 0xe2, 0x25, 0x87, 0x76, 0x5a, 0x9f, 0x6b, 0x4d, 0x20, 0xac, 0xec, 0x65, + 0x8c, 0xca, 0x1e, 0xf7, 0x61, 0x1d, 0xc7, 0xa5, 0xab, 0x46, 0x69, 0x28, 0x02, 0x44, 0x32, 0xca, + 0x99, 0x45, 0x23, 0x6d, 0x23, 0x79, 0xc3, 0x46, 0xa2, 0x82, 0xe0, 0x58, 0xac, 0x86, 0xa9, 0x6f, + 0xa0, 0x85, 0xe8, 0xfa, 0x8a, 0x7f, 0x67, 0xc1, 0x4c, 0x52, 0x2e, 0x4a, 0x8d, 0x17, 0x21, 0xbf, + 0x21, 0x20, 0x7b, 0xcb, 0xce, 0xb1, 0x19, 0xb2, 0x72, 0x21, 0x51, 0xcd, 0xca, 0x85, 0x84, 0xa0, + 0x27, 0x62, 0x1f, 0xac, 0x1a, 0x53, 0xbb, 0x83, 0xda, 0x21, 0x01, 0x30, 0x70, 0x15, 0x33, 0x67, + 0xc2, 0x8d, 0x67, 0xa2, 0x92, 0x88, 0x84, 0x98, 0x84, 0x55, 0x7d, 0xf3, 0xcf, 0x16, 0x8c, 0xc7, + 0x36, 0x22, 0x44, 0xc4, 0x8f, 0x80, 0x0a, 0x0f, 0xb2, 0x83, 0x9e, 0x80, 0x2c, 0xdb, 0xee, 0xa9, + 0xa8, 0xd0, 0x38, 0xf2, 0xc9, 0xa0, 0x76, 0x38, 0x36, 0x6d, 0x7d, 0xbb, 0x47, 0x89, 0x40, 0xe1, + 0x27, 0xa7, 0x69, 0xfb, 0x2d, 0xc7, 0xb5, 0x3b, 0x0e, 0x93, 0xda, 0xc9, 0x12, 0x13, 0xc4, 0xdd, + 0x51, 0xcf, 0xf6, 0x03, 0x9d, 0x04, 0x16, 0xa5, 0x3b, 0x52, 0x20, 0xa2, 0x1b, 0xa2, 0xb8, 0xb3, + 0x45, 0x59, 0x73, 0x53, 0x86, 0x05, 0x55, 0xdc, 0x11, 0x90, 0x58, 0x71, 0x47, 0x40, 0xf0, 0xcf, + 0xad, 0xc8, 0x38, 0xe5, 0x19, 0xfe, 0xc2, 0x19, 0x27, 0xfe, 0x56, 0x64, 0x27, 0x7a, 0x8b, 0xca, + 0x4e, 0x5e, 0x82, 0x89, 0x56, 0x6c, 0x64, 0xb4, 0xbd, 0xc8, 0xc2, 0x75, 0x02, 0x1d, 0xf7, 0x23, + 0x3d, 0x0a, 0xc8, 0x08, 0x3d, 0x26, 0x94, 0x93, 0xde, 0xab, 0x9c, 0x48, 0xea, 0x99, 0xfd, 0xa5, + 0xfe, 0xe4, 0x09, 0x28, 0x86, 0x1f, 0x29, 0x51, 0x09, 0xc6, 0x2e, 0x5d, 0x27, 0xaf, 0x5d, 0x20, + 0xcb, 0x93, 0x29, 0x54, 0x86, 0x42, 0xe3, 0xc2, 0xd2, 0x2b, 0xa2, 0x67, 0x2d, 0xfe, 0x3a, 0xaf, + 0x13, 0x17, 0x1f, 0x7d, 0x1d, 0x72, 0x32, 0x1b, 0x99, 0x89, 0x98, 0x33, 0xbf, 0xdf, 0xcd, 0x1e, + 0xdd, 0x03, 0x97, 0x52, 0xc2, 0xa9, 0xb3, 0x16, 0xba, 0x06, 0x25, 0x01, 0x54, 0x15, 0xf2, 0xb9, + 0x64, 0xa1, 0x3a, 0x46, 0xe9, 0xf8, 0x88, 0x51, 0x83, 0xde, 0x79, 0xc8, 0x49, 0x81, 0xcd, 0x24, + 0x92, 0xc6, 0x21, 0xbb, 0x89, 0x7d, 0x33, 0xc0, 0x29, 0xf4, 0x3c, 0x64, 0xd7, 0x6d, 0xa7, 0x83, + 0x8c, 0x9c, 0xd5, 0x28, 0x6c, 0xcf, 0xce, 0x24, 0xc1, 0xc6, 0xb2, 0x2f, 0x86, 0xf5, 0xf9, 0xa3, + 0xc9, 0x22, 0xa1, 0x9e, 0x5e, 0xd9, 0x3b, 0x10, 0xae, 0x7c, 0x5d, 0x56, 0x91, 0x75, 0xa9, 0x0a, + 0x1d, 0x8f, 0x2f, 0x95, 0xa8, 0x6c, 0xcd, 0x56, 0x47, 0x0d, 0x87, 0x04, 0x57, 0xa1, 0x64, 0x94, + 0x89, 0x4c, 0xb1, 0xee, 0xad, 0x71, 0x99, 0x62, 0x1d, 0x52, 0x5b, 0xc2, 0x29, 0xb4, 0x02, 0x05, + 0xf1, 0x58, 0xc3, 0x66, 0x01, 0x3a, 0x96, 0x4c, 0xe8, 0x8d, 0x44, 0x6e, 0x76, 0x6e, 0xf8, 0x60, + 0x48, 0xe8, 0x9b, 0x50, 0x5c, 0xa1, 0x4c, 0x45, 0xb0, 0xa3, 0xc9, 0x10, 0x38, 0x44, 0x52, 0xf1, + 0x30, 0x8a, 0x53, 0xe8, 0x75, 0x71, 0xe9, 0x88, 0xbb, 0x67, 0x54, 0x1b, 0xe1, 0x86, 0xc3, 0x7d, + 0xcd, 0x8f, 0x46, 0x08, 0x29, 0xbf, 0x16, 0xa3, 0xac, 0xf2, 0x86, 0xda, 0x88, 0x03, 0x1b, 0x52, + 0xae, 0xed, 0xf3, 0x48, 0x0c, 0xa7, 0x16, 0xdf, 0xd0, 0x2f, 0x4d, 0x96, 0x6d, 0x66, 0xa3, 0xeb, + 0x30, 0x11, 0x3e, 0x7c, 0x11, 0x0f, 0xa9, 0x62, 0x36, 0xbf, 0xe7, 0xd5, 0x56, 0xcc, 0xe6, 0xf7, + 0xbe, 0xde, 0xc2, 0xa9, 0xc5, 0x3b, 0x30, 0x2d, 0x3f, 0xfc, 0xc9, 0xc7, 0x47, 0x97, 0x7c, 0xcf, + 0x65, 0xdc, 0x67, 0x11, 0x18, 0x8f, 0xbd, 0x4a, 0x42, 0x86, 0xd5, 0x0c, 0x7b, 0x68, 0x65, 0xb2, + 0x32, 0xf4, 0x39, 0x13, 0x4e, 0x2d, 0xfe, 0xd5, 0x82, 0xb2, 0xb9, 0x18, 0xba, 0x69, 0x70, 0x23, + 0x9e, 0xf1, 0x98, 0x12, 0x1b, 0xfa, 0xaa, 0xc8, 0xd4, 0xc5, 0xf0, 0x17, 0x40, 0x38, 0x85, 0xee, + 0x88, 0x27, 0x49, 0x7b, 0x1f, 0x93, 0xa0, 0x13, 0xfb, 0xbe, 0x5b, 0x91, 0x8b, 0x9c, 0xfc, 0x8c, + 0xef, 0x5b, 0x70, 0xaa, 0xf1, 0xc6, 0x7b, 0x1f, 0x56, 0x53, 0xef, 0x7f, 0x58, 0x4d, 0x7d, 0xfc, + 0x61, 0xd5, 0xfa, 0xde, 0x4e, 0xd5, 0xfa, 0xd5, 0x4e, 0xd5, 0x7a, 0x77, 0xa7, 0x6a, 0xbd, 0xb7, + 0x53, 0xb5, 0xfe, 0xb1, 0x53, 0xb5, 0xfe, 0xb9, 0x53, 0x4d, 0x7d, 0xbc, 0x53, 0xb5, 0xde, 0xfe, + 0xa8, 0x9a, 0x7a, 0xef, 0xa3, 0x6a, 0xea, 0xfd, 0x8f, 0xaa, 0xa9, 0x6f, 0x9f, 0xdc, 0xbf, 0x94, + 0x21, 0xc3, 0x4a, 0x5e, 0xfc, 0x3d, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x64, 0xdb, 0xb5, + 0x9f, 0x3f, 0x29, 0x00, 0x00, } func (x Direction) String() string { @@ -3891,6 +3959,9 @@ func (this *StreamMetadata) Equal(that interface{}) bool { if this.StreamHash != that1.StreamHash { return false } + if this.RingToken != that1.RingToken { + return false + } return true } func (this *ExceedsLimitsRequest) Equal(that interface{}) bool { @@ -4033,6 +4104,22 @@ func (this *GetStreamUsageRequest) Equal(that interface{}) bool { if this.Tenant != that1.Tenant { return false } + if len(this.Partitions) != len(that1.Partitions) { + return false + } + for i := range this.Partitions { + if this.Partitions[i] != that1.Partitions[i] { + return false + } + } + if len(this.StreamHashes) != len(that1.StreamHashes) { + return false + } + for i := range this.StreamHashes { + if this.StreamHashes[i] != that1.StreamHashes[i] { + return false + } + } return true } func (this *GetStreamUsageResponse) Equal(that interface{}) bool { @@ -4060,24 +4147,24 @@ func (this *GetStreamUsageResponse) Equal(that interface{}) bool { if this.ActiveStreams != that1.ActiveStreams { return false } - if len(this.RecordedStreams) != len(that1.RecordedStreams) { + if len(this.UnknownStreams) != len(that1.UnknownStreams) { return false } - for i := range this.RecordedStreams { - if !this.RecordedStreams[i].Equal(that1.RecordedStreams[i]) { + for i := range this.UnknownStreams { + if this.UnknownStreams[i] != that1.UnknownStreams[i] { return false } } return true } -func (this *RecordedStreams) Equal(that interface{}) bool { +func (this *StreamRate) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*RecordedStreams) + that1, ok := that.(*StreamRate) if !ok { - that2, ok := that.(RecordedStreams) + that2, ok := that.(StreamRate) if ok { that1 = &that2 } else { @@ -4092,16 +4179,28 @@ func (this *RecordedStreams) Equal(that interface{}) bool { if this.StreamHash != that1.StreamHash { return false } + if this.StreamHashNoShard != that1.StreamHashNoShard { + return false + } + if this.Rate != that1.Rate { + return false + } + if this.Tenant != that1.Tenant { + return false + } + if this.Pushes != that1.Pushes { + return false + } return true } -func (this *StreamRate) Equal(that interface{}) bool { +func (this *GetAssignedPartitionsRequest) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*StreamRate) + that1, ok := that.(*GetAssignedPartitionsRequest) if !ok { - that2, ok := that.(StreamRate) + that2, ok := that.(GetAssignedPartitionsRequest) if ok { that1 = &that2 } else { @@ -4113,20 +4212,34 @@ func (this *StreamRate) Equal(that interface{}) bool { } else if this == nil { return false } - if this.StreamHash != that1.StreamHash { - return false + return true +} +func (this *GetAssignedPartitionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil } - if this.StreamHashNoShard != that1.StreamHashNoShard { - return false + + that1, ok := that.(*GetAssignedPartitionsResponse) + if !ok { + that2, ok := that.(GetAssignedPartitionsResponse) + if ok { + that1 = &that2 + } else { + return false + } } - if this.Rate != that1.Rate { + if that1 == nil { + return this == nil + } else if this == nil { return false } - if this.Tenant != that1.Tenant { + if len(this.AssignedPartitions) != len(that1.AssignedPartitions) { return false } - if this.Pushes != that1.Pushes { - return false + for i := range this.AssignedPartitions { + if this.AssignedPartitions[i] != that1.AssignedPartitions[i] { + return false + } } return true } @@ -5850,9 +5963,10 @@ func (this *StreamMetadata) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 5) + s := make([]string, 0, 6) s = append(s, "&logproto.StreamMetadata{") s = append(s, "StreamHash: "+fmt.Sprintf("%#v", this.StreamHash)+",\n") + s = append(s, "RingToken: "+fmt.Sprintf("%#v", this.RingToken)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -5908,9 +6022,11 @@ func (this *GetStreamUsageRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 5) + s := make([]string, 0, 7) s = append(s, "&logproto.GetStreamUsageRequest{") s = append(s, "Tenant: "+fmt.Sprintf("%#v", this.Tenant)+",\n") + s = append(s, "Partitions: "+fmt.Sprintf("%#v", this.Partitions)+",\n") + s = append(s, "StreamHashes: "+fmt.Sprintf("%#v", this.StreamHashes)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -5922,33 +6038,52 @@ func (this *GetStreamUsageResponse) GoString() string { s = append(s, "&logproto.GetStreamUsageResponse{") s = append(s, "Tenant: "+fmt.Sprintf("%#v", this.Tenant)+",\n") s = append(s, "ActiveStreams: "+fmt.Sprintf("%#v", this.ActiveStreams)+",\n") - if this.RecordedStreams != nil { - s = append(s, "RecordedStreams: "+fmt.Sprintf("%#v", this.RecordedStreams)+",\n") - } + s = append(s, "UnknownStreams: "+fmt.Sprintf("%#v", this.UnknownStreams)+",\n") s = append(s, "}") return strings.Join(s, "") } -func (this *RecordedStreams) GoString() string { +func (this *StreamRate) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 5) - s = append(s, "&logproto.RecordedStreams{") + s := make([]string, 0, 9) + s = append(s, "&logproto.StreamRate{") s = append(s, "StreamHash: "+fmt.Sprintf("%#v", this.StreamHash)+",\n") - s = append(s, "}") + s = append(s, "StreamHashNoShard: "+fmt.Sprintf("%#v", this.StreamHashNoShard)+",\n") + s = append(s, "Rate: "+fmt.Sprintf("%#v", this.Rate)+",\n") + s = append(s, "Tenant: "+fmt.Sprintf("%#v", this.Tenant)+",\n") + s = append(s, "Pushes: "+fmt.Sprintf("%#v", this.Pushes)+",\n") + s = append(s, "}") return strings.Join(s, "") } -func (this *StreamRate) GoString() string { +func (this *GetAssignedPartitionsRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) - s = append(s, "&logproto.StreamRate{") - s = append(s, "StreamHash: "+fmt.Sprintf("%#v", this.StreamHash)+",\n") - s = append(s, "StreamHashNoShard: "+fmt.Sprintf("%#v", this.StreamHashNoShard)+",\n") - s = append(s, "Rate: "+fmt.Sprintf("%#v", this.Rate)+",\n") - s = append(s, "Tenant: "+fmt.Sprintf("%#v", this.Tenant)+",\n") - s = append(s, "Pushes: "+fmt.Sprintf("%#v", this.Pushes)+",\n") + s := make([]string, 0, 4) + s = append(s, "&logproto.GetAssignedPartitionsRequest{") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *GetAssignedPartitionsResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&logproto.GetAssignedPartitionsResponse{") + keysForAssignedPartitions := make([]int32, 0, len(this.AssignedPartitions)) + for k, _ := range this.AssignedPartitions { + keysForAssignedPartitions = append(keysForAssignedPartitions, k) + } + github_com_gogo_protobuf_sortkeys.Int32s(keysForAssignedPartitions) + mapStringForAssignedPartitions := "map[int32]int64{" + for _, k := range keysForAssignedPartitions { + mapStringForAssignedPartitions += fmt.Sprintf("%#v: %#v,", k, this.AssignedPartitions[k]) + } + mapStringForAssignedPartitions += "}" + if this.AssignedPartitions != nil { + s = append(s, "AssignedPartitions: "+mapStringForAssignedPartitions+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -7290,6 +7425,7 @@ var _IngestLimitsFrontend_serviceDesc = grpc.ServiceDesc{ // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type IngestLimitsClient interface { GetStreamUsage(ctx context.Context, in *GetStreamUsageRequest, opts ...grpc.CallOption) (*GetStreamUsageResponse, error) + GetAssignedPartitions(ctx context.Context, in *GetAssignedPartitionsRequest, opts ...grpc.CallOption) (*GetAssignedPartitionsResponse, error) } type ingestLimitsClient struct { @@ -7309,9 +7445,19 @@ func (c *ingestLimitsClient) GetStreamUsage(ctx context.Context, in *GetStreamUs return out, nil } +func (c *ingestLimitsClient) GetAssignedPartitions(ctx context.Context, in *GetAssignedPartitionsRequest, opts ...grpc.CallOption) (*GetAssignedPartitionsResponse, error) { + out := new(GetAssignedPartitionsResponse) + err := c.cc.Invoke(ctx, "/logproto.IngestLimits/GetAssignedPartitions", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // IngestLimitsServer is the server API for IngestLimits service. type IngestLimitsServer interface { GetStreamUsage(context.Context, *GetStreamUsageRequest) (*GetStreamUsageResponse, error) + GetAssignedPartitions(context.Context, *GetAssignedPartitionsRequest) (*GetAssignedPartitionsResponse, error) } // UnimplementedIngestLimitsServer can be embedded to have forward compatible implementations. @@ -7321,6 +7467,9 @@ type UnimplementedIngestLimitsServer struct { func (*UnimplementedIngestLimitsServer) GetStreamUsage(ctx context.Context, req *GetStreamUsageRequest) (*GetStreamUsageResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetStreamUsage not implemented") } +func (*UnimplementedIngestLimitsServer) GetAssignedPartitions(ctx context.Context, req *GetAssignedPartitionsRequest) (*GetAssignedPartitionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetAssignedPartitions not implemented") +} func RegisterIngestLimitsServer(s *grpc.Server, srv IngestLimitsServer) { s.RegisterService(&_IngestLimits_serviceDesc, srv) @@ -7344,6 +7493,24 @@ func _IngestLimits_GetStreamUsage_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _IngestLimits_GetAssignedPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetAssignedPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IngestLimitsServer).GetAssignedPartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/logproto.IngestLimits/GetAssignedPartitions", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IngestLimitsServer).GetAssignedPartitions(ctx, req.(*GetAssignedPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _IngestLimits_serviceDesc = grpc.ServiceDesc{ ServiceName: "logproto.IngestLimits", HandlerType: (*IngestLimitsServer)(nil), @@ -7352,6 +7519,10 @@ var _IngestLimits_serviceDesc = grpc.ServiceDesc{ MethodName: "GetStreamUsage", Handler: _IngestLimits_GetStreamUsage_Handler, }, + { + MethodName: "GetAssignedPartitions", + Handler: _IngestLimits_GetAssignedPartitions_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "pkg/logproto/logproto.proto", @@ -7518,6 +7689,11 @@ func (m *StreamMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.RingToken != 0 { + i = encodeVarintLogproto(dAtA, i, uint64(m.RingToken)) + i-- + dAtA[i] = 0x10 + } if m.StreamHash != 0 { i = encodeVarintLogproto(dAtA, i, uint64(m.StreamHash)) i-- @@ -7702,6 +7878,43 @@ func (m *GetStreamUsageRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StreamHashes) > 0 { + dAtA3 := make([]byte, len(m.StreamHashes)*10) + var j2 int + for _, num := range m.StreamHashes { + for num >= 1<<7 { + dAtA3[j2] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j2++ + } + dAtA3[j2] = uint8(num) + j2++ + } + i -= j2 + copy(dAtA[i:], dAtA3[:j2]) + i = encodeVarintLogproto(dAtA, i, uint64(j2)) + i-- + dAtA[i] = 0x1a + } + if len(m.Partitions) > 0 { + dAtA5 := make([]byte, len(m.Partitions)*10) + var j4 int + for _, num1 := range m.Partitions { + num := uint64(num1) + for num >= 1<<7 { + dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j4++ + } + dAtA5[j4] = uint8(num) + j4++ + } + i -= j4 + copy(dAtA[i:], dAtA5[:j4]) + i = encodeVarintLogproto(dAtA, i, uint64(j4)) + i-- + dAtA[i] = 0x12 + } if len(m.Tenant) > 0 { i -= len(m.Tenant) copy(dAtA[i:], m.Tenant) @@ -7732,19 +7945,23 @@ func (m *GetStreamUsageResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) _ = i var l int _ = l - if len(m.RecordedStreams) > 0 { - for iNdEx := len(m.RecordedStreams) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.RecordedStreams[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintLogproto(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } + if len(m.UnknownStreams) > 0 { + dAtA7 := make([]byte, len(m.UnknownStreams)*10) + var j6 int + for _, num := range m.UnknownStreams { + for num >= 1<<7 { + dAtA7[j6] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j6++ + } + dAtA7[j6] = uint8(num) + j6++ + } + i -= j6 + copy(dAtA[i:], dAtA7[:j6]) + i = encodeVarintLogproto(dAtA, i, uint64(j6)) + i-- + dAtA[i] = 0x1a } if m.ActiveStreams != 0 { i = encodeVarintLogproto(dAtA, i, uint64(m.ActiveStreams)) @@ -7761,34 +7978,6 @@ func (m *GetStreamUsageResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } -func (m *RecordedStreams) 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 *RecordedStreams) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *RecordedStreams) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.StreamHash != 0 { - i = encodeVarintLogproto(dAtA, i, uint64(m.StreamHash)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - func (m *StreamRate) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -7839,6 +8028,67 @@ func (m *StreamRate) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *GetAssignedPartitionsRequest) 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 *GetAssignedPartitionsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetAssignedPartitionsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + +func (m *GetAssignedPartitionsResponse) 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 *GetAssignedPartitionsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetAssignedPartitionsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.AssignedPartitions) > 0 { + for k := range m.AssignedPartitions { + v := m.AssignedPartitions[k] + baseI := i + i = encodeVarintLogproto(dAtA, i, uint64(v)) + i-- + dAtA[i] = 0x10 + i = encodeVarintLogproto(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintLogproto(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *QueryRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -7911,20 +8161,20 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x28 } - n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err4 != nil { - return 0, err4 + n10, err10 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err10 != nil { + return 0, err10 } - i -= n4 - i = encodeVarintLogproto(dAtA, i, uint64(n4)) + i -= n10 + i = encodeVarintLogproto(dAtA, i, uint64(n10)) i-- dAtA[i] = 0x22 - n5, err5 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err5 != nil { - return 0, err5 + n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err11 != nil { + return 0, err11 } - i -= n5 - i = encodeVarintLogproto(dAtA, i, uint64(n5)) + i -= n11 + i = encodeVarintLogproto(dAtA, i, uint64(n11)) i-- dAtA[i] = 0x1a if m.Limit != 0 { @@ -8009,20 +8259,20 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x22 } } - n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err8 != nil { - return 0, err8 + n14, err14 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err14 != nil { + return 0, err14 } - i -= n8 - i = encodeVarintLogproto(dAtA, i, uint64(n8)) + i -= n14 + i = encodeVarintLogproto(dAtA, i, uint64(n14)) i-- dAtA[i] = 0x1a - n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err9 != nil { - return 0, err9 + n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err15 != nil { + return 0, err15 } - i -= n9 - i = encodeVarintLogproto(dAtA, i, uint64(n9)) + i -= n15 + i = encodeVarintLogproto(dAtA, i, uint64(n15)) i-- dAtA[i] = 0x12 if len(m.Selector) > 0 { @@ -8245,22 +8495,22 @@ func (m *LabelRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x2a } if m.End != nil { - n12, err12 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) - if err12 != nil { - return 0, err12 + n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) + if err18 != nil { + return 0, err18 } - i -= n12 - i = encodeVarintLogproto(dAtA, i, uint64(n12)) + i -= n18 + i = encodeVarintLogproto(dAtA, i, uint64(n18)) i-- dAtA[i] = 0x22 } if m.Start != nil { - n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) - if err13 != nil { - return 0, err13 + n19, err19 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) + if err19 != nil { + return 0, err19 } - i -= n13 - i = encodeVarintLogproto(dAtA, i, uint64(n13)) + i -= n19 + i = encodeVarintLogproto(dAtA, i, uint64(n19)) i-- dAtA[i] = 0x1a } @@ -8470,12 +8720,12 @@ func (m *TailRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x32 } - n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err15 != nil { - return 0, err15 + n21, err21 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err21 != nil { + return 0, err21 } - i -= n15 - i = encodeVarintLogproto(dAtA, i, uint64(n15)) + i -= n21 + i = encodeVarintLogproto(dAtA, i, uint64(n21)) i-- dAtA[i] = 0x2a if m.Limit != 0 { @@ -8585,20 +8835,20 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x1a } } - n17, err17 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err17 != nil { - return 0, err17 + n23, err23 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err23 != nil { + return 0, err23 } - i -= n17 - i = encodeVarintLogproto(dAtA, i, uint64(n17)) + i -= n23 + i = encodeVarintLogproto(dAtA, i, uint64(n23)) i-- dAtA[i] = 0x12 - n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err18 != nil { - return 0, err18 + n24, err24 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err24 != nil { + return 0, err24 } - i -= n18 - i = encodeVarintLogproto(dAtA, i, uint64(n18)) + i -= n24 + i = encodeVarintLogproto(dAtA, i, uint64(n24)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -8742,20 +8992,20 @@ func (m *DroppedStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n19, err19 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) - if err19 != nil { - return 0, err19 + n25, err25 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) + if err25 != nil { + return 0, err25 } - i -= n19 - i = encodeVarintLogproto(dAtA, i, uint64(n19)) + i -= n25 + i = encodeVarintLogproto(dAtA, i, uint64(n25)) i-- dAtA[i] = 0x12 - n20, err20 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) - if err20 != nil { - return 0, err20 + n26, err26 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) + if err26 != nil { + return 0, err26 } - i -= n20 - i = encodeVarintLogproto(dAtA, i, uint64(n20)) + i -= n26 + i = encodeVarintLogproto(dAtA, i, uint64(n26)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -8936,20 +9186,20 @@ func (m *GetChunkIDsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n21, err21 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err21 != nil { - return 0, err21 + n27, err27 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err27 != nil { + return 0, err27 } - i -= n21 - i = encodeVarintLogproto(dAtA, i, uint64(n21)) + i -= n27 + i = encodeVarintLogproto(dAtA, i, uint64(n27)) i-- dAtA[i] = 0x1a - n22, err22 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err22 != nil { - return 0, err22 + n28, err28 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err28 != nil { + return 0, err28 } - i -= n22 - i = encodeVarintLogproto(dAtA, i, uint64(n22)) + i -= n28 + i = encodeVarintLogproto(dAtA, i, uint64(n28)) i-- dAtA[i] = 0x12 if len(m.Matchers) > 0 { @@ -9908,20 +10158,20 @@ func (m *DetectedFieldsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n26, err26 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err26 != nil { - return 0, err26 + n32, err32 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err32 != nil { + return 0, err32 } - i -= n26 - i = encodeVarintLogproto(dAtA, i, uint64(n26)) + i -= n32 + i = encodeVarintLogproto(dAtA, i, uint64(n32)) i-- dAtA[i] = 0x12 - n27, err27 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err27 != nil { - return 0, err27 + n33, err33 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err33 != nil { + return 0, err33 } - i -= n27 - i = encodeVarintLogproto(dAtA, i, uint64(n27)) + i -= n33 + i = encodeVarintLogproto(dAtA, i, uint64(n33)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -10063,20 +10313,20 @@ func (m *DetectedLabelsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n28, err28 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err28 != nil { - return 0, err28 + n34, err34 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err34 != nil { + return 0, err34 } - i -= n28 - i = encodeVarintLogproto(dAtA, i, uint64(n28)) + i -= n34 + i = encodeVarintLogproto(dAtA, i, uint64(n34)) i-- dAtA[i] = 0x12 - n29, err29 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err29 != nil { - return 0, err29 + n35, err35 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err35 != nil { + return 0, err35 } - i -= n29 - i = encodeVarintLogproto(dAtA, i, uint64(n29)) + i -= n35 + i = encodeVarintLogproto(dAtA, i, uint64(n35)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -10242,6 +10492,9 @@ func (m *StreamMetadata) Size() (n int) { if m.StreamHash != 0 { n += 1 + sovLogproto(uint64(m.StreamHash)) } + if m.RingToken != 0 { + n += 1 + sovLogproto(uint64(m.RingToken)) + } return n } @@ -10324,6 +10577,20 @@ func (m *GetStreamUsageRequest) Size() (n int) { if l > 0 { n += 1 + l + sovLogproto(uint64(l)) } + if len(m.Partitions) > 0 { + l = 0 + for _, e := range m.Partitions { + l += sovLogproto(uint64(e)) + } + n += 1 + sovLogproto(uint64(l)) + l + } + if len(m.StreamHashes) > 0 { + l = 0 + for _, e := range m.StreamHashes { + l += sovLogproto(uint64(e)) + } + n += 1 + sovLogproto(uint64(l)) + l + } return n } @@ -10340,23 +10607,12 @@ func (m *GetStreamUsageResponse) Size() (n int) { if m.ActiveStreams != 0 { n += 1 + sovLogproto(uint64(m.ActiveStreams)) } - if len(m.RecordedStreams) > 0 { - for _, e := range m.RecordedStreams { - l = e.Size() - n += 1 + l + sovLogproto(uint64(l)) + if len(m.UnknownStreams) > 0 { + l = 0 + for _, e := range m.UnknownStreams { + l += sovLogproto(uint64(e)) } - } - return n -} - -func (m *RecordedStreams) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.StreamHash != 0 { - n += 1 + sovLogproto(uint64(m.StreamHash)) + n += 1 + sovLogproto(uint64(l)) + l } return n } @@ -10386,6 +10642,32 @@ func (m *StreamRate) Size() (n int) { return n } +func (m *GetAssignedPartitionsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *GetAssignedPartitionsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.AssignedPartitions) > 0 { + for k, v := range m.AssignedPartitions { + _ = k + _ = v + mapEntrySize := 1 + sovLogproto(uint64(k)) + 1 + sovLogproto(uint64(v)) + n += mapEntrySize + 1 + sovLogproto(uint64(mapEntrySize)) + } + } + return n +} + func (m *QueryRequest) Size() (n int) { if m == nil { return 0 @@ -11467,6 +11749,7 @@ func (this *StreamMetadata) String() string { } s := strings.Join([]string{`&StreamMetadata{`, `StreamHash:` + fmt.Sprintf("%v", this.StreamHash) + `,`, + `RingToken:` + fmt.Sprintf("%v", this.RingToken) + `,`, `}`, }, "") return s @@ -11531,6 +11814,8 @@ func (this *GetStreamUsageRequest) String() string { } s := strings.Join([]string{`&GetStreamUsageRequest{`, `Tenant:` + fmt.Sprintf("%v", this.Tenant) + `,`, + `Partitions:` + fmt.Sprintf("%v", this.Partitions) + `,`, + `StreamHashes:` + fmt.Sprintf("%v", this.StreamHashes) + `,`, `}`, }, "") return s @@ -11539,39 +11824,53 @@ func (this *GetStreamUsageResponse) String() string { if this == nil { return "nil" } - repeatedStringForRecordedStreams := "[]*RecordedStreams{" - for _, f := range this.RecordedStreams { - repeatedStringForRecordedStreams += strings.Replace(f.String(), "RecordedStreams", "RecordedStreams", 1) + "," - } - repeatedStringForRecordedStreams += "}" s := strings.Join([]string{`&GetStreamUsageResponse{`, `Tenant:` + fmt.Sprintf("%v", this.Tenant) + `,`, `ActiveStreams:` + fmt.Sprintf("%v", this.ActiveStreams) + `,`, - `RecordedStreams:` + repeatedStringForRecordedStreams + `,`, + `UnknownStreams:` + fmt.Sprintf("%v", this.UnknownStreams) + `,`, `}`, }, "") return s } -func (this *RecordedStreams) String() string { +func (this *StreamRate) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&RecordedStreams{`, + s := strings.Join([]string{`&StreamRate{`, `StreamHash:` + fmt.Sprintf("%v", this.StreamHash) + `,`, + `StreamHashNoShard:` + fmt.Sprintf("%v", this.StreamHashNoShard) + `,`, + `Rate:` + fmt.Sprintf("%v", this.Rate) + `,`, + `Tenant:` + fmt.Sprintf("%v", this.Tenant) + `,`, + `Pushes:` + fmt.Sprintf("%v", this.Pushes) + `,`, `}`, }, "") return s } -func (this *StreamRate) String() string { +func (this *GetAssignedPartitionsRequest) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&StreamRate{`, - `StreamHash:` + fmt.Sprintf("%v", this.StreamHash) + `,`, - `StreamHashNoShard:` + fmt.Sprintf("%v", this.StreamHashNoShard) + `,`, - `Rate:` + fmt.Sprintf("%v", this.Rate) + `,`, - `Tenant:` + fmt.Sprintf("%v", this.Tenant) + `,`, - `Pushes:` + fmt.Sprintf("%v", this.Pushes) + `,`, + s := strings.Join([]string{`&GetAssignedPartitionsRequest{`, + `}`, + }, "") + return s +} +func (this *GetAssignedPartitionsResponse) String() string { + if this == nil { + return "nil" + } + keysForAssignedPartitions := make([]int32, 0, len(this.AssignedPartitions)) + for k, _ := range this.AssignedPartitions { + keysForAssignedPartitions = append(keysForAssignedPartitions, k) + } + github_com_gogo_protobuf_sortkeys.Int32s(keysForAssignedPartitions) + mapStringForAssignedPartitions := "map[int32]int64{" + for _, k := range keysForAssignedPartitions { + mapStringForAssignedPartitions += fmt.Sprintf("%v: %v,", k, this.AssignedPartitions[k]) + } + mapStringForAssignedPartitions += "}" + s := strings.Join([]string{`&GetAssignedPartitionsResponse{`, + `AssignedPartitions:` + mapStringForAssignedPartitions + `,`, `}`, }, "") return s @@ -12707,17 +13006,36 @@ func (m *StreamMetadata) Unmarshal(dAtA []byte) error { break } } - default: - iNdEx = preIndex - skippy, err := skipLogproto(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthLogproto + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RingToken", wireType) } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthLogproto + m.RingToken = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RingToken |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthLogproto } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -13225,6 +13543,158 @@ func (m *GetStreamUsageRequest) Unmarshal(dAtA []byte) error { } m.Tenant = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 2: + if wireType == 0 { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Partitions = append(m.Partitions, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Partitions) == 0 { + m.Partitions = make([]int32, 0, elementCount) + } + for iNdEx < postIndex { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Partitions = append(m.Partitions, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Partitions", wireType) + } + case 3: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.StreamHashes = append(m.StreamHashes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.StreamHashes) == 0 { + m.StreamHashes = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.StreamHashes = append(m.StreamHashes, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field StreamHashes", wireType) + } default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) @@ -13330,110 +13800,80 @@ func (m *GetStreamUsageResponse) Unmarshal(dAtA []byte) error { } } case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RecordedStreams", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowLogproto + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + m.UnknownStreams = append(m.UnknownStreams, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break + if packedLen < 0 { + return ErrInvalidLengthLogproto } - } - if msglen < 0 { - return ErrInvalidLengthLogproto - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthLogproto - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.RecordedStreams = append(m.RecordedStreams, &RecordedStreams{}) - if err := m.RecordedStreams[len(m.RecordedStreams)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipLogproto(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthLogproto - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthLogproto - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *RecordedStreams) 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 ErrIntOverflowLogproto - } - 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: RecordedStreams: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RecordedStreams: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StreamHash", wireType) - } - m.StreamHash = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowLogproto + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthLogproto } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - m.StreamHash |= uint64(b&0x7F) << shift - if b < 0x80 { - break + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.UnknownStreams) == 0 { + m.UnknownStreams = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.UnknownStreams = append(m.UnknownStreams, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field UnknownStreams", wireType) } default: iNdEx = preIndex @@ -13620,6 +14060,211 @@ func (m *StreamRate) Unmarshal(dAtA []byte) error { } return nil } +func (m *GetAssignedPartitionsRequest) 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 ErrIntOverflowLogproto + } + 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: GetAssignedPartitionsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetAssignedPartitionsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetAssignedPartitionsResponse) 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 ErrIntOverflowLogproto + } + 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: GetAssignedPartitionsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetAssignedPartitionsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AssignedPartitions", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AssignedPartitions == nil { + m.AssignedPartitions = make(map[int32]int64) + } + var mapkey int32 + var mapvalue int64 + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapvalue |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else { + iNdEx = entryPreIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.AssignedPartitions[mapkey] = mapvalue + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *QueryRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 4a7fa23e106bf..3576ba7c84312 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -57,6 +57,7 @@ message StreamRatesResponse { message StreamMetadata { uint64 streamHash = 1; + uint32 ringToken = 2; } service IngestLimitsFrontend { @@ -85,20 +86,19 @@ message RejectedStream { service IngestLimits { rpc GetStreamUsage(GetStreamUsageRequest) returns (GetStreamUsageResponse) {} + rpc GetAssignedPartitions(GetAssignedPartitionsRequest) returns (GetAssignedPartitionsResponse) {} } message GetStreamUsageRequest { string tenant = 1; + repeated int32 partitions = 2; + repeated uint64 streamHashes = 3; } message GetStreamUsageResponse { string tenant = 1; uint64 activeStreams = 2; - repeated RecordedStreams recordedStreams = 3; -} - -message RecordedStreams { - uint64 streamHash = 1; + repeated uint64 unknownStreams = 3; } message StreamRate { @@ -109,6 +109,12 @@ message StreamRate { uint32 pushes = 5; } +message GetAssignedPartitionsRequest {} + +message GetAssignedPartitionsResponse { + map assignedPartitions = 1; +} + message QueryRequest { string selector = 1 [deprecated = true]; uint32 limit = 2; diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index c7e224b6c8b8e..a363f4e909d3d 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -763,7 +763,7 @@ func (t *Loki) setupModuleManager() error { UI: {Server}, Distributor: {Ring, Server, Overrides, TenantConfigs, PatternRingClient, PatternIngesterTee, Analytics, PartitionRing, IngestLimitsFrontendRing, UI}, IngestLimitsRing: {RuntimeConfig, Server, MemberlistKV}, - IngestLimits: {MemberlistKV, Server}, + IngestLimits: {MemberlistKV, Server, PartitionRing}, IngestLimitsFrontend: {IngestLimitsRing, Overrides, Server, MemberlistKV}, IngestLimitsFrontendRing: {RuntimeConfig, Server, MemberlistKV}, Store: {Overrides, IndexGatewayRing}, diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 2b0e4a8f33fa8..c391ec6655952 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -441,6 +441,7 @@ func (t *Loki) initIngestLimits() (services.Service, error) { ingestLimits, err := limits.NewIngestLimits( t.Cfg.IngestLimits, + t.partitionRingWatcher, util_log.Logger, prometheus.DefaultRegisterer, ) diff --git a/tools/stream-metadata-generator/main.go b/tools/stream-metadata-generator/main.go index ad657258822fb..e50e91ad0c3cf 100644 --- a/tools/stream-metadata-generator/main.go +++ b/tools/stream-metadata-generator/main.go @@ -401,7 +401,7 @@ func (s *generator) sendStreamsToKafka(ctx context.Context, streams []distributo startTime := time.Now() // Add metadata record - metadataRecord := kafka.EncodeStreamMetadata(partitionID, s.cfg.Kafka.Topic, tenant, stream.HashNoShard) + metadataRecord := kafka.EncodeStreamMetadata(partitionID, s.cfg.Kafka.Topic, tenant, stream.HashNoShard, stream.RingToken) // Send to Kafka produceResults := s.writer.ProduceSync(ctx, []*kgo.Record{metadataRecord})