chore: move protos related to limits to limits package (#17592)

pull/17598/head
George Robinson 2 weeks ago committed by GitHub
parent bf6f130cdf
commit e2aa96ba61
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 49
      pkg/distributor/distributor_test.go
  2. 16
      pkg/distributor/ingest_limits.go
  3. 63
      pkg/distributor/ingest_limits_test.go
  4. 39
      pkg/kafka/encoding.go
  5. 82
      pkg/kafka/encoding_test.go
  6. 6
      pkg/limits/client/client.go
  7. 6
      pkg/limits/frontend/client/client.go
  8. 18
      pkg/limits/frontend/frontend.go
  9. 64
      pkg/limits/frontend/frontend_test.go
  10. 4
      pkg/limits/frontend/gather.go
  11. 10
      pkg/limits/frontend/http.go
  12. 26
      pkg/limits/frontend/http_test.go
  13. 30
      pkg/limits/frontend/mock_test.go
  14. 20
      pkg/limits/frontend/ring.go
  15. 192
      pkg/limits/frontend/ring_test.go
  16. 20
      pkg/limits/ingest_limits.go
  17. 34
      pkg/limits/ingest_limits_test.go
  18. 1939
      pkg/limits/proto/limits.pb.go
  19. 38
      pkg/limits/proto/limits.proto
  20. 14
      pkg/limits/stream_metadata_bench_test.go
  21. 18
      pkg/limits/stream_metadata_test.go
  22. 3586
      pkg/logproto/logproto.pb.go
  23. 35
      pkg/logproto/logproto.proto
  24. 5
      pkg/loki/modules.go
  25. 23
      tools/stream-generator/generator/kafka.go

@ -43,6 +43,7 @@ import (
"github.com/grafana/loki/v3/pkg/limits"
limits_frontend "github.com/grafana/loki/v3/pkg/limits/frontend"
limits_frontend_client "github.com/grafana/loki/v3/pkg/limits/frontend/client"
limitsproto "github.com/grafana/loki/v3/pkg/limits/proto"
loghttp_push "github.com/grafana/loki/v3/pkg/loghttp/push"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql/syntax"
@ -2396,8 +2397,8 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
tenant string
streams logproto.PushRequest
expectedLimitsCalls uint64
expectedLimitsRequest *logproto.ExceedsLimitsRequest
limitsResponse *logproto.ExceedsLimitsResponse
expectedLimitsRequest *limitsproto.ExceedsLimitsRequest
limitsResponse *limitsproto.ExceedsLimitsResponse
limitsResponseErr error
expectedErr string
}{{
@ -2424,16 +2425,16 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,
}},
},
limitsResponse: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{},
limitsResponse: &limitsproto.ExceedsLimitsResponse{
Results: []*limitsproto.ExceedsLimitsResult{},
},
}, {
name: "max stream limit is exceeded",
@ -2449,16 +2450,16 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,
}},
},
limitsResponse: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
limitsResponse: &limitsproto.ExceedsLimitsResponse{
Results: []*limitsproto.ExceedsLimitsResult{{
StreamHash: 0x90eb45def17f924,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -2478,16 +2479,16 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,
}},
},
limitsResponse: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
limitsResponse: &limitsproto.ExceedsLimitsResponse{
Results: []*limitsproto.ExceedsLimitsResult{{
StreamHash: 0x90eb45def17f924,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
@ -2513,9 +2514,9 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,
@ -2525,8 +2526,8 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
StructuredMetadataSize: 0x0,
}},
},
limitsResponse: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
limitsResponse: &limitsproto.ExceedsLimitsResponse{
Results: []*limitsproto.ExceedsLimitsResult{{
StreamHash: 1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -2546,16 +2547,16 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,
}},
},
limitsResponse: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
limitsResponse: &limitsproto.ExceedsLimitsResponse{
Results: []*limitsproto.ExceedsLimitsResult{{
StreamHash: 1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -2574,9 +2575,9 @@ func TestDistributor_PushIngestLimits(t *testing.T) {
}},
},
expectedLimitsCalls: 1,
expectedLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedLimitsRequest: &limitsproto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*limitsproto.StreamMetadata{{
StreamHash: 0x90eb45def17f924,
EntriesSize: 0x3,
StructuredMetadataSize: 0x0,

@ -13,12 +13,12 @@ import (
"github.com/grafana/loki/v3/pkg/limits"
limits_frontend_client "github.com/grafana/loki/v3/pkg/limits/frontend/client"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
// ingestLimitsFrontendClient is used for tests.
type ingestLimitsFrontendClient interface {
exceedsLimits(context.Context, *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error)
exceedsLimits(context.Context, *proto.ExceedsLimitsRequest) (*proto.ExceedsLimitsResponse, error)
}
// ingestLimitsFrontendRingClient uses the ring to query ingest-limits frontends.
@ -35,7 +35,7 @@ func newIngestLimitsFrontendRingClient(ring ring.ReadRing, pool *ring_client.Poo
}
// Implements the ingestLimitsFrontendClient interface.
func (c *ingestLimitsFrontendRingClient) exceedsLimits(ctx context.Context, req *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error) {
func (c *ingestLimitsFrontendRingClient) exceedsLimits(ctx context.Context, req *proto.ExceedsLimitsRequest) (*proto.ExceedsLimitsResponse, error) {
// We use an FNV-1 of all stream hashes in the request to load balance requests
// to limits-frontends instances.
h := fnv.New32()
@ -65,7 +65,7 @@ func (c *ingestLimitsFrontendRingClient) exceedsLimits(ctx context.Context, req
lastErr = err
continue
}
client := c.(logproto.IngestLimitsFrontendClient)
client := c.(proto.IngestLimitsFrontendClient)
resp, err := client.ExceedsLimits(ctx, req)
if err != nil {
lastErr = err
@ -138,21 +138,21 @@ func (l *ingestLimits) exceedsLimits(ctx context.Context, tenant string, streams
return true, reasonsForHashes, nil
}
func newExceedsLimitsRequest(tenant string, streams []KeyedStream) (*logproto.ExceedsLimitsRequest, error) {
func newExceedsLimitsRequest(tenant string, streams []KeyedStream) (*proto.ExceedsLimitsRequest, error) {
// The distributor sends the hashes of all streams in the request to the
// limits-frontend. The limits-frontend is responsible for deciding if
// the request would exceed the tenants limits, and if so, which streams
// from the request caused it to exceed its limits.
streamMetadata := make([]*logproto.StreamMetadata, 0, len(streams))
streamMetadata := make([]*proto.StreamMetadata, 0, len(streams))
for _, stream := range streams {
entriesSize, structuredMetadataSize := calculateStreamSizes(stream.Stream)
streamMetadata = append(streamMetadata, &logproto.StreamMetadata{
streamMetadata = append(streamMetadata, &proto.StreamMetadata{
StreamHash: stream.HashKeyNoShard,
EntriesSize: entriesSize,
StructuredMetadataSize: structuredMetadataSize,
})
}
return &logproto.ExceedsLimitsRequest{
return &proto.ExceedsLimitsRequest{
Tenant: tenant,
Streams: streamMetadata,
}, nil

@ -12,6 +12,7 @@ import (
"go.uber.org/atomic"
"github.com/grafana/loki/v3/pkg/limits"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/logproto"
)
@ -19,13 +20,13 @@ import (
type mockIngestLimitsFrontendClient struct {
t *testing.T
calls atomic.Uint64
expectedRequest *logproto.ExceedsLimitsRequest
response *logproto.ExceedsLimitsResponse
expectedRequest *proto.ExceedsLimitsRequest
response *proto.ExceedsLimitsResponse
responseErr error
}
// Implements the ingestLimitsFrontendClient interface.
func (c *mockIngestLimitsFrontendClient) exceedsLimits(_ context.Context, r *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error) {
func (c *mockIngestLimitsFrontendClient) exceedsLimits(_ context.Context, r *proto.ExceedsLimitsRequest) (*proto.ExceedsLimitsResponse, error) {
c.calls.Add(1)
if c.expectedRequest != nil {
require.Equal(c.t, c.expectedRequest, r)
@ -44,8 +45,8 @@ func TestIngestLimits_EnforceLimits(t *testing.T) {
name string
tenant string
streams []KeyedStream
expectedRequest *logproto.ExceedsLimitsRequest
response *logproto.ExceedsLimitsResponse
expectedRequest *proto.ExceedsLimitsRequest
response *proto.ExceedsLimitsResponse
responseErr error
expectedStreams []KeyedStream
expectedReasons map[uint64][]string
@ -83,9 +84,9 @@ func TestIngestLimits_EnforceLimits(t *testing.T) {
}},
},
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
EntriesSize: 0x3,
StructuredMetadataSize: 0x6,
@ -104,14 +105,14 @@ func TestIngestLimits_EnforceLimits(t *testing.T) {
HashKey: 1000, // Should not be used.
HashKeyNoShard: 1,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}},
},
response: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
response: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 1,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
@ -128,16 +129,16 @@ func TestIngestLimits_EnforceLimits(t *testing.T) {
HashKey: 2000, // Should not be used.
HashKeyNoShard: 2,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}, {
StreamHash: 2,
}},
},
response: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
response: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 1,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
@ -157,16 +158,16 @@ func TestIngestLimits_EnforceLimits(t *testing.T) {
HashKey: 2000, // Should not be used.
HashKeyNoShard: 2,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}, {
StreamHash: 2,
}},
},
response: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{},
response: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{},
},
expectedStreams: []KeyedStream{{
HashKey: 1000, // Should not be used.
@ -212,8 +213,8 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
name string
tenant string
streams []KeyedStream
expectedRequest *logproto.ExceedsLimitsRequest
response *logproto.ExceedsLimitsResponse
expectedRequest *proto.ExceedsLimitsRequest
response *proto.ExceedsLimitsResponse
responseErr error
expectedExceedsLimits bool
expectedReasons map[uint64][]string
@ -224,9 +225,9 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
streams: []KeyedStream{{
HashKeyNoShard: 1,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}},
},
@ -238,14 +239,14 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
streams: []KeyedStream{{
HashKeyNoShard: 1,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}},
},
response: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
response: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 1,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
@ -258,14 +259,14 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
streams: []KeyedStream{{
HashKeyNoShard: 1,
}},
expectedRequest: &logproto.ExceedsLimitsRequest{
expectedRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 1,
}},
},
response: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{},
response: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{},
},
expectedReasons: nil,
}}

@ -195,45 +195,6 @@ func sovPush(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}
// 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, tenantID string, streamHash, entriesSize, structuredMetadataSize uint64) (*kgo.Record, error) {
// Validate stream hash
if streamHash == 0 {
return nil, fmt.Errorf("invalid stream hash '%d'", streamHash)
}
metadata := logproto.StreamMetadata{
StreamHash: streamHash,
EntriesSize: entriesSize,
StructuredMetadataSize: structuredMetadataSize,
}
// Encode the metadata into a byte slice
value, err := metadata.Marshal()
if err != nil {
return nil, err
}
return &kgo.Record{
Key: []byte(tenantID),
Value: value,
Partition: partition,
Topic: MetadataTopicFor(topic),
}, nil
}
// DecodeStreamMetadata decodes a Kafka record into a StreamMetadata.
// It returns the decoded metadata and any error encountered.
func DecodeStreamMetadata(record *kgo.Record) (*logproto.StreamMetadata, error) {
var metadata logproto.StreamMetadata
if err := metadata.Unmarshal(record.Value); err != nil {
return nil, fmt.Errorf("failed to unmarshal stream metadata: %w", err)
}
return &metadata, nil
}
// MetadataTopicFor returns the metadata topic name for the given topic.
func MetadataTopicFor(topic string) string {
return topic + metadataTopicSuffix

@ -7,7 +7,6 @@ import (
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/twmb/franz-go/pkg/kgo"
"github.com/grafana/loki/v3/pkg/logproto"
)
@ -150,84 +149,3 @@ func generateRandomString(length int) string {
}
return string(b)
}
func TestEncodeDecodeStreamMetadata(t *testing.T) {
tests := []struct {
name string
hash uint64
partition int32
topic string
tenantID string
entriesSize uint64
structuredMetadataSize uint64
expectErr bool
}{
{
name: "Valid metadata",
hash: 12345,
partition: 1,
topic: "logs",
tenantID: "tenant-1",
entriesSize: 1024,
structuredMetadataSize: 512,
expectErr: false,
},
{
name: "Valid metadata with zero sizes",
hash: 67890,
partition: 2,
topic: "metrics",
tenantID: "tenant-2",
entriesSize: 0,
structuredMetadataSize: 0,
expectErr: false,
},
{
name: "Zero hash - should error",
hash: 0,
partition: 3,
topic: "traces",
tenantID: "tenant-3",
entriesSize: 2048,
structuredMetadataSize: 1024,
expectErr: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
// Encode metadata
record, err := EncodeStreamMetadata(tt.partition, tt.topic, tt.tenantID, tt.hash, tt.entriesSize, tt.structuredMetadataSize)
if tt.expectErr {
require.Error(t, err)
require.Nil(t, record)
return
}
require.NotNil(t, record)
require.NotNil(t, record.Value)
require.Equal(t, tt.topic+metadataTopicSuffix, record.Topic)
require.Equal(t, tt.partition, record.Partition)
require.Equal(t, []byte(tt.tenantID), record.Key)
// Decode metadata
metadata, err := DecodeStreamMetadata(record)
require.NoError(t, err)
require.NotNil(t, metadata)
// Verify decoded values
require.Equal(t, tt.hash, metadata.StreamHash)
require.Equal(t, tt.entriesSize, metadata.EntriesSize)
require.Equal(t, tt.structuredMetadataSize, metadata.StructuredMetadataSize)
})
}
t.Run("Decode invalid value", func(t *testing.T) {
record := &kgo.Record{
Value: []byte("invalid data"),
}
metadata, err := DecodeStreamMetadata(record)
require.Error(t, err)
require.Nil(t, metadata)
})
}

@ -18,7 +18,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/util/server"
)
@ -67,7 +67,7 @@ func (cfg *PoolConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
// Client is a gRPC client for the ingest-limits.
type Client struct {
logproto.IngestLimitsClient
proto.IngestLimitsClient
grpc_health_v1.HealthClient
io.Closer
}
@ -89,7 +89,7 @@ func NewClient(cfg Config, addr string) (*Client, error) {
return nil, err
}
return &Client{
IngestLimitsClient: logproto.NewIngestLimitsClient(conn),
IngestLimitsClient: proto.NewIngestLimitsClient(conn),
HealthClient: grpc_health_v1.NewHealthClient(conn),
Closer: conn,
}, nil

@ -21,7 +21,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/util/server"
)
@ -85,7 +85,7 @@ func (cfg *PoolConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
// Client is a gRPC client for the ingest-limits-frontend.
type Client struct {
logproto.IngestLimitsFrontendClient
proto.IngestLimitsFrontendClient
grpc_health_v1.HealthClient
io.Closer
}
@ -107,7 +107,7 @@ func NewClient(cfg Config, addr string) (*Client, error) {
return nil, err
}
return &Client{
IngestLimitsFrontendClient: logproto.NewIngestLimitsFrontendClient(conn),
IngestLimitsFrontendClient: proto.NewIngestLimitsFrontendClient(conn),
HealthClient: grpc_health_v1.NewHealthClient(conn),
Closer: conn,
}, nil

@ -16,7 +16,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
limits_client "github.com/grafana/loki/v3/pkg/limits/client"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
// Frontend is a frontend for the limits service. It is responsible for
@ -27,7 +27,7 @@ type Frontend struct {
cfg Config
logger log.Logger
gatherer ExceedsLimitsGatherer
assignedPartitionsCache Cache[string, *logproto.GetAssignedPartitionsResponse]
assignedPartitionsCache Cache[string, *proto.GetAssignedPartitionsResponse]
subservices *services.Manager
subservicesWatcher *services.FailureWatcher
lifecycler *ring.Lifecycler
@ -48,12 +48,12 @@ func New(cfg Config, ringName string, limitsRing ring.ReadRing, logger log.Logge
)
// Set up the assigned partitions cache.
var assignedPartitionsCache Cache[string, *logproto.GetAssignedPartitionsResponse]
var assignedPartitionsCache Cache[string, *proto.GetAssignedPartitionsResponse]
if cfg.AssignedPartitionsCacheTTL == 0 {
// When the TTL is 0, the cache is disabled.
assignedPartitionsCache = NewNopCache[string, *logproto.GetAssignedPartitionsResponse]()
assignedPartitionsCache = NewNopCache[string, *proto.GetAssignedPartitionsResponse]()
} else {
assignedPartitionsCache = NewTTLCache[string, *logproto.GetAssignedPartitionsResponse](cfg.AssignedPartitionsCacheTTL)
assignedPartitionsCache = NewTTLCache[string, *proto.GetAssignedPartitionsResponse](cfg.AssignedPartitionsCacheTTL)
}
gatherer := NewRingGatherer(limitsRing, clientPool, cfg.NumPartitions, assignedPartitionsCache, logger)
@ -87,17 +87,17 @@ func New(cfg Config, ringName string, limitsRing ring.ReadRing, logger log.Logge
return f, nil
}
// ExceedsLimits implements logproto.IngestLimitsFrontendClient.
func (f *Frontend) ExceedsLimits(ctx context.Context, req *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error) {
// ExceedsLimits implements proto.IngestLimitsFrontendClient.
func (f *Frontend) ExceedsLimits(ctx context.Context, req *proto.ExceedsLimitsRequest) (*proto.ExceedsLimitsResponse, error) {
resps, err := f.gatherer.ExceedsLimits(ctx, req)
if err != nil {
return nil, err
}
results := make([]*logproto.ExceedsLimitsResult, 0, len(req.Streams))
results := make([]*proto.ExceedsLimitsResult, 0, len(req.Streams))
for _, resp := range resps {
results = append(results, resp.Results...)
}
return &logproto.ExceedsLimitsResponse{results}, nil
return &proto.ExceedsLimitsResponse{results}, nil
}
func (f *Frontend) CheckReady(ctx context.Context) error {

@ -8,67 +8,67 @@ import (
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/limits"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func TestFrontend_ExceedsLimits(t *testing.T) {
tests := []struct {
name string
exceedsLimitsRequest *logproto.ExceedsLimitsRequest
exceedsLimitsResponses []*logproto.ExceedsLimitsResponse
expected *logproto.ExceedsLimitsResponse
exceedsLimitsRequest *proto.ExceedsLimitsRequest
exceedsLimitsResponses []*proto.ExceedsLimitsResponse
expected *proto.ExceedsLimitsResponse
}{{
name: "no streams",
exceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
exceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: nil,
},
expected: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{},
expected: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{},
},
}, {
name: "one stream",
exceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
exceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}},
expected: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
expected: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
},
}, {
name: "one stream, no responses",
exceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
exceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{},
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{},
}},
expected: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{},
expected: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{},
},
}, {
name: "two stream, one response",
exceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
exceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -78,8 +78,8 @@ func TestFrontend_ExceedsLimits(t *testing.T) {
StructuredMetadataSize: 0x6,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}, {
@ -87,8 +87,8 @@ func TestFrontend_ExceedsLimits(t *testing.T) {
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
}},
expected: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
expected: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}, {
@ -98,9 +98,9 @@ func TestFrontend_ExceedsLimits(t *testing.T) {
},
}, {
name: "two stream, two responses",
exceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
exceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -110,19 +110,19 @@ func TestFrontend_ExceedsLimits(t *testing.T) {
StructuredMetadataSize: 0x6,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}, {
Results: []*logproto.ExceedsLimitsResult{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x4,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
}},
expected: &logproto.ExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
expected: &proto.ExceedsLimitsResponse{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}, {

@ -3,12 +3,12 @@ package frontend
import (
"context"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
type ExceedsLimitsGatherer interface {
// ExceedsLimits checks if the streams in the request have exceeded their
// per-partition limits. It returns more than one response when the
// requested streams are sharded over two or more limits instances.
ExceedsLimits(context.Context, *logproto.ExceedsLimitsRequest) ([]*logproto.ExceedsLimitsResponse, error)
ExceedsLimits(context.Context, *proto.ExceedsLimitsRequest) ([]*proto.ExceedsLimitsResponse, error)
}

@ -7,17 +7,17 @@ import (
"github.com/go-kit/log/level"
"github.com/grafana/dskit/user"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/util"
)
type httpExceedsLimitsRequest struct {
Tenant string `json:"tenant"`
Streams []*logproto.StreamMetadata `json:"streams"`
Tenant string `json:"tenant"`
Streams []*proto.StreamMetadata `json:"streams"`
}
type httpExceedsLimitsResponse struct {
Results []*logproto.ExceedsLimitsResult `json:"results,omitempty"`
Results []*proto.ExceedsLimitsResult `json:"results,omitempty"`
}
// ServeHTTP implements http.Handler.
@ -39,7 +39,7 @@ func (f *Frontend) ServeHTTP(w http.ResponseWriter, r *http.Request) {
return
}
resp, err := f.ExceedsLimits(ctx, &logproto.ExceedsLimitsRequest{
resp, err := f.ExceedsLimits(ctx, &proto.ExceedsLimitsRequest{
Tenant: req.Tenant,
Streams: req.Streams,
})

@ -11,30 +11,30 @@ import (
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/limits"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func TestFrontend_ServeHTTP(t *testing.T) {
tests := []struct {
name string
expectedExceedsLimitsRequest *logproto.ExceedsLimitsRequest
exceedsLimitsResponses []*logproto.ExceedsLimitsResponse
expectedExceedsLimitsRequest *proto.ExceedsLimitsRequest
exceedsLimitsResponses []*proto.ExceedsLimitsResponse
request httpExceedsLimitsRequest
expected httpExceedsLimitsResponse
}{{
name: "within limits",
expectedExceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedExceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{}},
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{}},
request: httpExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -43,30 +43,30 @@ func TestFrontend_ServeHTTP(t *testing.T) {
// expected should be default value.
}, {
name: "exceeds limits",
expectedExceedsLimitsRequest: &logproto.ExceedsLimitsRequest{
expectedExceedsLimitsRequest: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},
}},
request: httpExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
},
expected: httpExceedsLimitsResponse{
Results: []*logproto.ExceedsLimitsResult{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsRateLimit),
}},

@ -13,7 +13,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
// mockExceedsLimitsGatherer mocks an ExeceedsLimitsGatherer. It avoids having to
@ -21,30 +21,30 @@ import (
type mockExceedsLimitsGatherer struct {
t *testing.T
expectedExceedsLimitsRequest *logproto.ExceedsLimitsRequest
exceedsLimitsResponses []*logproto.ExceedsLimitsResponse
expectedExceedsLimitsRequest *proto.ExceedsLimitsRequest
exceedsLimitsResponses []*proto.ExceedsLimitsResponse
}
func (g *mockExceedsLimitsGatherer) ExceedsLimits(_ context.Context, req *logproto.ExceedsLimitsRequest) ([]*logproto.ExceedsLimitsResponse, error) {
func (g *mockExceedsLimitsGatherer) ExceedsLimits(_ context.Context, req *proto.ExceedsLimitsRequest) ([]*proto.ExceedsLimitsResponse, error) {
if expected := g.expectedExceedsLimitsRequest; expected != nil {
require.Equal(g.t, expected, req)
}
return g.exceedsLimitsResponses, nil
}
// mockIngestLimitsClient mocks logproto.IngestLimitsClient.
// mockIngestLimitsClient mocks proto.IngestLimitsClient.
type mockIngestLimitsClient struct {
logproto.IngestLimitsClient
proto.IngestLimitsClient
t *testing.T
// The requests expected to be received.
expectedAssignedPartitionsRequest *logproto.GetAssignedPartitionsRequest
expectedExceedsLimitsRequest *logproto.ExceedsLimitsRequest
expectedAssignedPartitionsRequest *proto.GetAssignedPartitionsRequest
expectedExceedsLimitsRequest *proto.ExceedsLimitsRequest
// The mocked responses.
getAssignedPartitionsResponse *logproto.GetAssignedPartitionsResponse
getAssignedPartitionsResponse *proto.GetAssignedPartitionsResponse
getAssignedPartitionsResponseErr error
exceedsLimitsResponse *logproto.ExceedsLimitsResponse
exceedsLimitsResponse *proto.ExceedsLimitsResponse
exceedsLimitsResponseErr error
// The expected request counts.
@ -56,7 +56,7 @@ type mockIngestLimitsClient struct {
numExceedsLimitsRequests int
}
func (m *mockIngestLimitsClient) GetAssignedPartitions(_ context.Context, r *logproto.GetAssignedPartitionsRequest, _ ...grpc.CallOption) (*logproto.GetAssignedPartitionsResponse, error) {
func (m *mockIngestLimitsClient) GetAssignedPartitions(_ context.Context, r *proto.GetAssignedPartitionsRequest, _ ...grpc.CallOption) (*proto.GetAssignedPartitionsResponse, error) {
if expected := m.expectedAssignedPartitionsRequest; expected != nil {
require.Equal(m.t, expected, r)
}
@ -67,7 +67,7 @@ func (m *mockIngestLimitsClient) GetAssignedPartitions(_ context.Context, r *log
return m.getAssignedPartitionsResponse, nil
}
func (m *mockIngestLimitsClient) ExceedsLimits(_ context.Context, r *logproto.ExceedsLimitsRequest, _ ...grpc.CallOption) (*logproto.ExceedsLimitsResponse, error) {
func (m *mockIngestLimitsClient) ExceedsLimits(_ context.Context, r *proto.ExceedsLimitsRequest, _ ...grpc.CallOption) (*proto.ExceedsLimitsResponse, error) {
if expected := m.expectedExceedsLimitsRequest; expected != nil {
require.Equal(m.t, expected, r)
}
@ -98,9 +98,9 @@ func (m *mockIngestLimitsClient) Watch(_ context.Context, _ *grpc_health_v1.Heal
}
// mockFactory mocks ring_client.PoolFactory. It returns a mocked
// logproto.IngestLimitsClient.
// proto.IngestLimitsClient.
type mockFactory struct {
clients map[string]logproto.IngestLimitsClient
clients map[string]proto.IngestLimitsClient
}
func (f *mockFactory) FromInstance(inst ring.InstanceDesc) (ring_client.PoolClient, error) {
@ -130,7 +130,7 @@ func newMockRingWithClientPool(_ *testing.T, name string, clients []*mockIngestL
}
// Set up the factory that is used to create clients on demand.
factory := &mockFactory{
clients: make(map[string]logproto.IngestLimitsClient),
clients: make(map[string]proto.IngestLimitsClient),
}
for i := 0; i < len(clients); i++ {
factory.clients[instances[i].Addr] = clients[i]

@ -9,7 +9,7 @@ import (
ring_client "github.com/grafana/dskit/ring/client"
"golang.org/x/sync/errgroup"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
const (
@ -27,7 +27,7 @@ type RingGatherer struct {
ring ring.ReadRing
pool *ring_client.Pool
numPartitions int
assignedPartitionsCache Cache[string, *logproto.GetAssignedPartitionsResponse]
assignedPartitionsCache Cache[string, *proto.GetAssignedPartitionsResponse]
}
// NewRingGatherer returns a new RingGatherer.
@ -35,7 +35,7 @@ func NewRingGatherer(
ring ring.ReadRing,
pool *ring_client.Pool,
numPartitions int,
assignedPartitionsCache Cache[string, *logproto.GetAssignedPartitionsResponse],
assignedPartitionsCache Cache[string, *proto.GetAssignedPartitionsResponse],
logger log.Logger,
) *RingGatherer {
return &RingGatherer{
@ -48,7 +48,7 @@ func NewRingGatherer(
}
// ExceedsLimits implements ExceedsLimitsGatherer.
func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *logproto.ExceedsLimitsRequest) ([]*logproto.ExceedsLimitsResponse, error) {
func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *proto.ExceedsLimitsRequest) ([]*proto.ExceedsLimitsResponse, error) {
if len(req.Streams) == 0 {
return nil, nil
}
@ -60,7 +60,7 @@ func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *logproto.ExceedsL
if err != nil {
return nil, err
}
ownedStreams := make(map[string][]*logproto.StreamMetadata)
ownedStreams := make(map[string][]*proto.StreamMetadata)
for _, s := range req.Streams {
partitionID := int32(s.StreamHash % uint64(g.numPartitions))
addr, ok := partitionConsumers[partitionID]
@ -72,7 +72,7 @@ func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *logproto.ExceedsL
ownedStreams[addr] = append(ownedStreams[addr], s)
}
errg, ctx := errgroup.WithContext(ctx)
responseCh := make(chan *logproto.ExceedsLimitsResponse, len(ownedStreams))
responseCh := make(chan *proto.ExceedsLimitsResponse, len(ownedStreams))
for addr, streams := range ownedStreams {
errg.Go(func() error {
client, err := g.pool.GetClientFor(addr)
@ -80,7 +80,7 @@ func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *logproto.ExceedsL
level.Error(g.logger).Log("msg", "failed to get client for instance", "instance", addr, "err", err.Error())
return err
}
resp, err := client.(logproto.IngestLimitsClient).ExceedsLimits(ctx, &logproto.ExceedsLimitsRequest{
resp, err := client.(proto.IngestLimitsClient).ExceedsLimits(ctx, &proto.ExceedsLimitsRequest{
Tenant: req.Tenant,
Streams: streams,
})
@ -95,7 +95,7 @@ func (g *RingGatherer) ExceedsLimits(ctx context.Context, req *logproto.ExceedsL
return nil, err
}
close(responseCh)
responses := make([]*logproto.ExceedsLimitsResponse, 0, len(rs.Instances))
responses := make([]*proto.ExceedsLimitsResponse, 0, len(rs.Instances))
for resp := range responseCh {
responses = append(responses, resp)
}
@ -146,7 +146,7 @@ func (g *RingGatherer) getZoneAwarePartitionConsumers(ctx context.Context, insta
type getAssignedPartitionsResponse struct {
addr string
response *logproto.GetAssignedPartitionsResponse
response *proto.GetAssignedPartitionsResponse
}
// getPartitionConsumers returns the consumer for each partition.
@ -185,7 +185,7 @@ func (g *RingGatherer) getPartitionConsumers(ctx context.Context, instances []ri
level.Error(g.logger).Log("failed to get client for instance", "instance", instance.Addr, "err", err.Error())
return nil
}
resp, err := client.(logproto.IngestLimitsClient).GetAssignedPartitions(ctx, &logproto.GetAssignedPartitionsRequest{})
resp, err := client.(proto.IngestLimitsClient).GetAssignedPartitions(ctx, &proto.GetAssignedPartitionsRequest{})
if err != nil {
level.Error(g.logger).Log("failed to get assigned partitions for instance", "instance", instance.Addr, "err", err.Error())
return nil

@ -11,13 +11,13 @@ import (
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/limits"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func TestRingGatherer_ExceedsLimits(t *testing.T) {
tests := []struct {
name string
request *logproto.ExceedsLimitsRequest
request *proto.ExceedsLimitsRequest
// Instances contains the complete set of instances that should be mocked.
// For example, if a test case is expected to make RPC calls to one instance,
// then just one InstanceDesc is required.
@ -27,34 +27,34 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
// value contains the expected request/response for the instance at the
// same index in the instances slice. If a request/response is not expected,
// the value can be set to nil.
expectedAssignedPartitionsRequest []*logproto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*logproto.GetAssignedPartitionsResponse
expectedExceedsLimitsRequests []*logproto.ExceedsLimitsRequest
exceedsLimitsResponses []*logproto.ExceedsLimitsResponse
expectedAssignedPartitionsRequest []*proto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*proto.GetAssignedPartitionsResponse
expectedExceedsLimitsRequests []*proto.ExceedsLimitsRequest
exceedsLimitsResponses []*proto.ExceedsLimitsResponse
exceedsLimitsResponseErrs []error
expected []*logproto.ExceedsLimitsResponse
expected []*proto.ExceedsLimitsResponse
expectedErr string
}{{
// When there are no streams, no RPCs should be sent.
name: "no streams",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: nil,
},
instances: []ring.InstanceDesc{{Addr: "instance-0"}},
numPartitions: 1,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{nil},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{nil},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{nil},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{nil},
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{nil},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{nil},
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{nil},
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{nil},
exceedsLimitsResponseErrs: []error{nil},
}, {
// When there is one instance owning all partitions, that instance is
// responsible for enforcing limits of all streams.
name: "one stream one instance",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1, // 0x1 is assigned to partition 0.
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -64,29 +64,29 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
Addr: "instance-0",
}},
numPartitions: 1,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
}},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{{
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
}},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}},
exceedsLimitsResponseErrs: []error{nil},
expected: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
expected: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -97,9 +97,9 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
// partitions. But when we have one stream, just one instance
// should be called to enforce limits.
name: "one stream two instances",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1, // 0x1 is assigned to partition 1.
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -111,8 +111,8 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
Addr: "instance-1",
}},
numPartitions: 2,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -121,23 +121,23 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
1: time.Now().UnixNano(),
},
}},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{nil, {
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{nil, {
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
}},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{nil, {
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{nil, {
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}},
exceedsLimitsResponseErrs: []error{nil, nil},
expected: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
expected: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -147,9 +147,9 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
// shard to one partition, just the instance that consumes that
// partition should be called to enforce limits.
name: "two streams, two instances, all streams to one partition",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1, // 0x1 is assigned to partition 1.
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -165,8 +165,8 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
Addr: "instance-1",
}},
numPartitions: 2,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -175,9 +175,9 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
1: time.Now().UnixNano(),
},
}},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{nil, {
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{nil, {
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -187,15 +187,15 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
StructuredMetadataSize: 0x5,
}},
}},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{nil, {
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{nil, {
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}},
exceedsLimitsResponseErrs: []error{nil, nil},
expected: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
expected: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -205,9 +205,9 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
// shards to different partitions, all instances should be called
// called to enforce limits.
name: "two streams, two instances, one stream each",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1, // 0x1 is assigned to partition 1.
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -223,8 +223,8 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
Addr: "instance-1",
}},
numPartitions: 2,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -233,40 +233,40 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
1: time.Now().UnixNano(),
},
}},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{{
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x2,
EntriesSize: 0x4,
StructuredMetadataSize: 0x5,
}},
}, {
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
}},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x2,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}, {
Results: []*logproto.ExceedsLimitsResult{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}},
exceedsLimitsResponseErrs: []error{nil, nil},
expected: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
expected: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x1,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
}, {
Results: []*logproto.ExceedsLimitsResult{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x2,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -274,9 +274,9 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
}, {
// When one instance returns an error, the entire request is failed.
name: "two streams, two instances, one instance returns error",
request: &logproto.ExceedsLimitsRequest{
request: &proto.ExceedsLimitsRequest{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1, // 0x1 is assigned to partition 1.
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
@ -292,8 +292,8 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
Addr: "instance-1",
}},
numPartitions: 2,
expectedAssignedPartitionsRequest: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequest: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -302,23 +302,23 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
1: time.Now().UnixNano(),
},
}},
expectedExceedsLimitsRequests: []*logproto.ExceedsLimitsRequest{{
expectedExceedsLimitsRequests: []*proto.ExceedsLimitsRequest{{
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x2,
EntriesSize: 0x4,
StructuredMetadataSize: 0x5,
}},
}, {
Tenant: "test",
Streams: []*logproto.StreamMetadata{{
Streams: []*proto.StreamMetadata{{
StreamHash: 0x1,
EntriesSize: 0x2,
StructuredMetadataSize: 0x3,
}},
}},
exceedsLimitsResponses: []*logproto.ExceedsLimitsResponse{{
Results: []*logproto.ExceedsLimitsResult{{
exceedsLimitsResponses: []*proto.ExceedsLimitsResponse{{
Results: []*proto.ExceedsLimitsResult{{
StreamHash: 0x2,
Reason: uint32(limits.ReasonExceedsMaxStreams),
}},
@ -354,7 +354,7 @@ func TestRingGatherer_ExceedsLimits(t *testing.T) {
t.Cleanup(mockClients[i].AssertExpectedNumRequests)
}
readRing, clientPool := newMockRingWithClientPool(t, "test", mockClients, test.instances)
cache := NewNopCache[string, *logproto.GetAssignedPartitionsResponse]()
cache := NewNopCache[string, *proto.GetAssignedPartitionsResponse]()
g := NewRingGatherer(readRing, clientPool, test.numPartitions, cache, log.NewNopLogger())
// Set a maximum upper bound on the test execution time.
@ -377,8 +377,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
tests := []struct {
name string
instances []ring.InstanceDesc
expectedAssignedPartitionsRequests []*logproto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*logproto.GetAssignedPartitionsResponse
expectedAssignedPartitionsRequests []*proto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*proto.GetAssignedPartitionsResponse
getAssignedPartitionsResponseErrs []error
expected map[string]map[int32]string
}{{
@ -387,8 +387,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-a-0",
Zone: "a",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -404,8 +404,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-b-0",
Zone: "b",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -428,8 +428,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-b-0",
Zone: "b",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
1: time.Now().UnixNano(),
@ -453,8 +453,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-b-0",
Zone: "b",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
1: time.Now().UnixNano(),
@ -474,8 +474,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-b-0",
Zone: "b",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{}, {}},
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{}, {}},
getAssignedPartitionsResponseErrs: []error{nil, nil, nil},
expected: map[string]map[int32]string{"a": {}, "b": {}},
}, {
@ -490,8 +490,8 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
Addr: "instance-b-0",
Zone: "b",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -533,7 +533,7 @@ func TestRingStreamUsageGatherer_GetZoneAwarePartitionConsumers(t *testing.T) {
}
// Set up the mocked ring and client pool for the tests.
readRing, clientPool := newMockRingWithClientPool(t, "test", clients, test.instances)
cache := NewNopCache[string, *logproto.GetAssignedPartitionsResponse]()
cache := NewNopCache[string, *proto.GetAssignedPartitionsResponse]()
g := NewRingGatherer(readRing, clientPool, 2, cache, log.NewNopLogger())
// Set a maximum upper bound on the test execution time.
@ -558,8 +558,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
// value contains the expected request/response for the instance at the
// same index in the instances slice. If a request/response is not expected,
// the value can be set to nil.
expectedAssignedPartitionsRequests []*logproto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*logproto.GetAssignedPartitionsResponse
expectedAssignedPartitionsRequests []*proto.GetAssignedPartitionsRequest
getAssignedPartitionsResponses []*proto.GetAssignedPartitionsResponse
getAssignedPartitionsResponseErrs []error
// The expected result.
expected map[int32]string
@ -568,8 +568,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
instances: []ring.InstanceDesc{{
Addr: "instance-0",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -585,8 +585,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
}, {
Addr: "instance-1",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -607,8 +607,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
}, {
Addr: "instance-1",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().Add(-time.Second).UnixNano(),
},
@ -630,8 +630,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
}, {
Addr: "instance-1",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{{
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{{
AssignedPartitions: map[int32]int64{
0: time.Now().Add(-time.Second).UnixNano(),
},
@ -652,8 +652,8 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
}, {
Addr: "instance-1",
}},
expectedAssignedPartitionsRequests: []*logproto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*logproto.GetAssignedPartitionsResponse{nil, nil},
expectedAssignedPartitionsRequests: []*proto.GetAssignedPartitionsRequest{{}, {}},
getAssignedPartitionsResponses: []*proto.GetAssignedPartitionsResponse{nil, nil},
getAssignedPartitionsResponseErrs: []error{
errors.New("an unexpected error occurred"),
errors.New("an unexpected error occurred"),
@ -682,7 +682,7 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers(t *testing.T) {
}
// Set up the mocked ring and client pool for the tests.
readRing, clientPool := newMockRingWithClientPool(t, "test", mockClients, test.instances)
cache := NewNopCache[string, *logproto.GetAssignedPartitionsResponse]()
cache := NewNopCache[string, *proto.GetAssignedPartitionsResponse]()
g := NewRingGatherer(readRing, clientPool, 1, cache, log.NewNopLogger())
// Set a maximum upper bound on the test execution time.
@ -700,7 +700,7 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers_IsCached(t *testing.T) {
// Set up the mock clients, one for each pair of mock RPC responses.
client0 := mockIngestLimitsClient{
t: t,
getAssignedPartitionsResponse: &logproto.GetAssignedPartitionsResponse{
getAssignedPartitionsResponse: &proto.GetAssignedPartitionsResponse{
AssignedPartitions: map[int32]int64{
0: time.Now().UnixNano(),
},
@ -710,7 +710,7 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers_IsCached(t *testing.T) {
t.Cleanup(client0.AssertExpectedNumRequests)
client1 := mockIngestLimitsClient{
t: t,
getAssignedPartitionsResponse: &logproto.GetAssignedPartitionsResponse{
getAssignedPartitionsResponse: &proto.GetAssignedPartitionsResponse{
AssignedPartitions: map[int32]int64{
1: time.Now().UnixNano(),
},
@ -726,7 +726,7 @@ func TestRingStreamUsageGatherer_GetPartitionConsumers_IsCached(t *testing.T) {
// Set the cache TTL large enough that entries cannot expire (flake)
// during slow test runs.
cache := NewTTLCache[string, *logproto.GetAssignedPartitionsResponse](time.Minute)
cache := NewTTLCache[string, *proto.GetAssignedPartitionsResponse](time.Minute)
g := NewRingGatherer(readRing, clientPool, 2, cache, log.NewNopLogger())
// Set a maximum upper bound on the test execution time.

@ -17,7 +17,7 @@ import (
"github.com/grafana/loki/v3/pkg/kafka"
"github.com/grafana/loki/v3/pkg/kafka/client"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/util"
"github.com/grafana/loki/v3/pkg/util/constants"
)
@ -297,7 +297,7 @@ func (s *IngestLimits) evictOldStreamsPeriodic(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(rec *logproto.StreamMetadata, tenant string, partition int32, lastSeenAt time.Time) {
func (s *IngestLimits) updateMetadata(rec *proto.StreamMetadata, tenant string, partition int32, lastSeenAt time.Time) {
var (
// Use the provided lastSeenAt timestamp as the last seen time
recordTime = lastSeenAt.UnixNano()
@ -337,18 +337,18 @@ func (s *IngestLimits) stopping(failureCase error) error {
return allErrs.Err()
}
// GetAssignedPartitions implements the logproto.IngestLimitsServer interface.
// GetAssignedPartitions implements the proto.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) {
resp := logproto.GetAssignedPartitionsResponse{
func (s *IngestLimits) GetAssignedPartitions(_ context.Context, _ *proto.GetAssignedPartitionsRequest) (*proto.GetAssignedPartitionsResponse, error) {
resp := proto.GetAssignedPartitionsResponse{
AssignedPartitions: s.partitionManager.List(),
}
return &resp, nil
}
// ExceedsLimits implements the logproto.IngestLimitsServer interface.
// ExceedsLimits implements the proto.IngestLimitsServer interface.
// It returns the number of active streams for a tenant and the status of requested streams.
func (s *IngestLimits) ExceedsLimits(_ context.Context, req *logproto.ExceedsLimitsRequest) (*logproto.ExceedsLimitsResponse, error) {
func (s *IngestLimits) ExceedsLimits(_ context.Context, req *proto.ExceedsLimitsRequest) (*proto.ExceedsLimitsResponse, error) {
var (
lastSeenAt = s.clock.Now()
// Use the provided lastSeenAt timestamp as the last seen time
@ -386,10 +386,10 @@ func (s *IngestLimits) ExceedsLimits(_ context.Context, req *logproto.ExceedsLim
ingestedBytes := s.metadata.StoreCond(req.Tenant, streams, cutoff, bucketStart, bucketCutoff, cond)
var results []*logproto.ExceedsLimitsResult
var results []*proto.ExceedsLimitsResult
for reason, streamHashes := range storeRes {
for _, streamHash := range streamHashes {
results = append(results, &logproto.ExceedsLimitsResult{
results = append(results, &proto.ExceedsLimitsResult{
StreamHash: streamHash,
Reason: uint32(reason),
})
@ -398,5 +398,5 @@ func (s *IngestLimits) ExceedsLimits(_ context.Context, req *logproto.ExceedsLim
s.metrics.tenantIngestedBytesTotal.WithLabelValues(req.Tenant).Add(float64(ingestedBytes))
return &logproto.ExceedsLimitsResponse{results}, nil
return &proto.ExceedsLimitsResponse{results}, nil
}

@ -15,7 +15,7 @@ import (
"github.com/grafana/loki/v3/pkg/kafka"
"github.com/grafana/loki/v3/pkg/limits/internal/testutil"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func TestIngestLimits_ExceedsLimits(t *testing.T) {
@ -36,11 +36,11 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
// Request data for ExceedsLimits.
tenantID string
streams []*logproto.StreamMetadata
streams []*proto.StreamMetadata
// Expectations.
expectedIngestedBytes float64
expectedResults []*logproto.ExceedsLimitsResult
expectedResults []*proto.ExceedsLimitsResult
}{
{
name: "tenant not found",
@ -66,7 +66,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 10,
// request data
tenantID: "tenant2",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{
StreamHash: 0x2,
EntriesSize: 1000,
@ -102,7 +102,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
// request data
tenantID: "tenant1",
maxActiveStreams: 10,
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x1, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x3, EntriesSize: 1000, StructuredMetadataSize: 10},
@ -136,13 +136,13 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 3,
// request data
tenantID: "tenant1",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x4, EntriesSize: 1000, StructuredMetadataSize: 10},
},
// expect data
expectedIngestedBytes: 0,
expectedResults: []*logproto.ExceedsLimitsResult{
expectedResults: []*proto.ExceedsLimitsResult{
{StreamHash: 0x2, Reason: uint32(ReasonExceedsMaxStreams)},
{StreamHash: 0x4, Reason: uint32(ReasonExceedsMaxStreams)},
},
@ -172,7 +172,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 3,
// request data
tenantID: "tenant1",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x1, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x3, EntriesSize: 1000, StructuredMetadataSize: 10},
@ -181,7 +181,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
},
// expect data
expectedIngestedBytes: 3030,
expectedResults: []*logproto.ExceedsLimitsResult{
expectedResults: []*proto.ExceedsLimitsResult{
{StreamHash: 0x2, Reason: uint32(ReasonExceedsMaxStreams)},
{StreamHash: 0x4, Reason: uint32(ReasonExceedsMaxStreams)},
},
@ -213,7 +213,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 5,
// request data
tenantID: "tenant1",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x1, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x3, EntriesSize: 1000, StructuredMetadataSize: 10},
@ -241,7 +241,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 3,
// request data
tenantID: "tenant1",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x1, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10},
{StreamHash: 0x3, EntriesSize: 1000, StructuredMetadataSize: 10},
@ -249,7 +249,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
},
// expect data
expectedIngestedBytes: 2020,
expectedResults: []*logproto.ExceedsLimitsResult{
expectedResults: []*proto.ExceedsLimitsResult{
{StreamHash: 0x3, Reason: uint32(ReasonExceedsMaxStreams)},
{StreamHash: 0x4, Reason: uint32(ReasonExceedsMaxStreams)},
},
@ -272,7 +272,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
maxActiveStreams: 3,
// request data
tenantID: "tenant1",
streams: []*logproto.StreamMetadata{
streams: []*proto.StreamMetadata{
{StreamHash: 0x1, EntriesSize: 1000, StructuredMetadataSize: 10}, // Unassigned
{StreamHash: 0x2, EntriesSize: 1000, StructuredMetadataSize: 10}, // Assigned
{StreamHash: 0x3, EntriesSize: 1000, StructuredMetadataSize: 10}, // Unassigned
@ -280,7 +280,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
},
// expect data
expectedIngestedBytes: 1010,
expectedResults: []*logproto.ExceedsLimitsResult{
expectedResults: []*proto.ExceedsLimitsResult{
{StreamHash: 0x4, Reason: uint32(ReasonExceedsMaxStreams)},
},
},
@ -329,7 +329,7 @@ func TestIngestLimits_ExceedsLimits(t *testing.T) {
s.partitionManager.Assign(context.Background(), nil, partitions)
// Call ExceedsLimits.
req := &logproto.ExceedsLimitsRequest{
req := &proto.ExceedsLimitsRequest{
Tenant: tt.tenantID,
Streams: tt.streams,
}
@ -419,9 +419,9 @@ func TestIngestLimits_ExceedsLimits_Concurrent(t *testing.T) {
for range concurrency {
go func() {
defer wg.Done()
req := &logproto.ExceedsLimitsRequest{
req := &proto.ExceedsLimitsRequest{
Tenant: "tenant1",
Streams: []*logproto.StreamMetadata{{StreamHash: 1}, {StreamHash: 2}, {StreamHash: 3}, {StreamHash: 4}, {StreamHash: 5}},
Streams: []*proto.StreamMetadata{{StreamHash: 1}, {StreamHash: 2}, {StreamHash: 3}, {StreamHash: 4}, {StreamHash: 5}},
}
resp, err := s.ExceedsLimits(context.Background(), req)

File diff suppressed because it is too large Load Diff

@ -0,0 +1,38 @@
syntax = "proto3";
package proto;
service IngestLimitsFrontend {
rpc ExceedsLimits(ExceedsLimitsRequest) returns (ExceedsLimitsResponse) {}
}
service IngestLimits {
rpc ExceedsLimits(ExceedsLimitsRequest) returns (ExceedsLimitsResponse) {}
rpc GetAssignedPartitions(GetAssignedPartitionsRequest) returns (GetAssignedPartitionsResponse) {}
}
message ExceedsLimitsRequest {
string tenant = 1;
repeated StreamMetadata streams = 2;
}
message ExceedsLimitsResponse {
repeated ExceedsLimitsResult results = 1;
}
message ExceedsLimitsResult {
uint64 streamHash = 1;
uint32 reason = 2;
}
message GetAssignedPartitionsRequest {}
message GetAssignedPartitionsResponse {
map<int32, int64> assignedPartitions = 1;
}
message StreamMetadata {
uint64 streamHash = 1;
uint64 entriesSize = 2;
uint64 structuredMetadataSize = 3;
}

@ -6,7 +6,7 @@ import (
"testing"
"time"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func BenchmarkStreamMetadata_Store(b *testing.B) {
@ -66,7 +66,7 @@ func BenchmarkStreamMetadata_Store(b *testing.B) {
streamIdx := i % bm.streamsPerPartition
updateTime := now.Add(time.Duration(i) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -91,7 +91,7 @@ func BenchmarkStreamMetadata_Store(b *testing.B) {
streamIdx := i % bm.streamsPerPartition
updateTime := now.Add(time.Duration(i) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -119,7 +119,7 @@ func BenchmarkStreamMetadata_Store(b *testing.B) {
streamIdx := i % bm.streamsPerPartition
updateTime := now.Add(time.Duration(i) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -146,7 +146,7 @@ func BenchmarkStreamMetadata_Store(b *testing.B) {
streamIdx := i % bm.streamsPerPartition
updateTime := now.Add(time.Duration(i) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -227,7 +227,7 @@ func BenchmarkStreamMetadata_UsageAndStore(b *testing.B) {
streamIdx := partition % bm.streamsPerPartition
updateTime := now.Add(time.Duration(partition) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -261,7 +261,7 @@ func BenchmarkStreamMetadata_UsageAndStore(b *testing.B) {
streamIdx := i % bm.streamsPerPartition
updateTime := now.Add(time.Duration(i) * time.Second)
metadata := &logproto.StreamMetadata{
metadata := &proto.StreamMetadata{
StreamHash: uint64(streamIdx),
EntriesSize: 1000,
StructuredMetadataSize: 500,

@ -8,7 +8,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
)
func TestStreamMetadata_All(t *testing.T) {
@ -130,7 +130,7 @@ func TestStreamMetadata_Store(t *testing.T) {
tenantID string
partitionID int32
lastSeenAt time.Time
record *logproto.StreamMetadata
record *proto.StreamMetadata
// Expectations.
expected map[string]map[int32]map[uint64]Stream
@ -141,7 +141,7 @@ func TestStreamMetadata_Store(t *testing.T) {
tenantID: "tenant1",
partitionID: 0,
lastSeenAt: time.Unix(100, 0),
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 123,
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -185,7 +185,7 @@ func TestStreamMetadata_Store(t *testing.T) {
},
tenantID: "tenant1",
partitionID: 1,
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 456,
EntriesSize: 2000,
StructuredMetadataSize: 1000,
@ -239,7 +239,7 @@ func TestStreamMetadata_Store(t *testing.T) {
},
tenantID: "tenant1",
partitionID: 0,
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 123,
EntriesSize: 3000,
StructuredMetadataSize: 1500,
@ -264,7 +264,7 @@ func TestStreamMetadata_Store(t *testing.T) {
{
name: "update existing bucket",
tenantID: "tenant1",
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 888,
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -307,7 +307,7 @@ func TestStreamMetadata_Store(t *testing.T) {
{
name: "clean up buckets outside rate window",
tenantID: "tenant1",
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 999,
EntriesSize: 2000,
StructuredMetadataSize: 1000,
@ -353,7 +353,7 @@ func TestStreamMetadata_Store(t *testing.T) {
{
name: "update same minute bucket",
tenantID: "tenant1",
record: &logproto.StreamMetadata{
record: &proto.StreamMetadata{
StreamHash: 555,
EntriesSize: 1000,
StructuredMetadataSize: 500,
@ -439,7 +439,7 @@ func TestStreamMetadata_Store_Concurrent(t *testing.T) {
partitionID = 1
}
record := &logproto.StreamMetadata{
record := &proto.StreamMetadata{
StreamHash: uint64(i),
EntriesSize: 1000,
StructuredMetadataSize: 500,

File diff suppressed because it is too large Load Diff

@ -55,35 +55,6 @@ message StreamRatesResponse {
repeated StreamRate streamRates = 1;
}
message StreamMetadata {
uint64 streamHash = 1;
uint64 entriesSize = 2;
uint64 structuredMetadataSize = 3;
}
service IngestLimitsFrontend {
rpc ExceedsLimits(ExceedsLimitsRequest) returns (ExceedsLimitsResponse) {}
}
message ExceedsLimitsRequest {
string tenant = 1;
repeated StreamMetadata streams = 2;
}
message ExceedsLimitsResponse {
repeated ExceedsLimitsResult results = 1;
}
message ExceedsLimitsResult {
uint64 streamHash = 1;
uint32 reason = 2;
}
service IngestLimits {
rpc ExceedsLimits(ExceedsLimitsRequest) returns (ExceedsLimitsResponse) {}
rpc GetAssignedPartitions(GetAssignedPartitionsRequest) returns (GetAssignedPartitionsResponse) {}
}
message StreamRate {
uint64 streamHash = 1;
uint64 streamHashNoShard = 2;
@ -92,12 +63,6 @@ message StreamRate {
uint32 pushes = 5;
}
message GetAssignedPartitionsRequest {}
message GetAssignedPartitionsResponse {
map<int32, int64> assignedPartitions = 1;
}
message QueryRequest {
string selector = 1 [deprecated = true];
uint32 limit = 2;

@ -63,6 +63,7 @@ import (
"github.com/grafana/loki/v3/pkg/kafka/partition"
"github.com/grafana/loki/v3/pkg/limits"
limits_frontend "github.com/grafana/loki/v3/pkg/limits/frontend"
limitsproto "github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
"github.com/grafana/loki/v3/pkg/logqlmodel/stats"
@ -454,7 +455,7 @@ func (t *Loki) initIngestLimits() (services.Service, error) {
}
t.ingestLimits = ingestLimits
logproto.RegisterIngestLimitsServer(t.Server.GRPC, ingestLimits)
limitsproto.RegisterIngestLimitsServer(t.Server.GRPC, ingestLimits)
// Register HTTP handler for metadata
t.Server.HTTP.Path("/ingest-limits/usage/{tenant}").Methods("GET").Handler(ingestLimits)
@ -511,7 +512,7 @@ func (t *Loki) initIngestLimitsFrontend() (services.Service, error) {
return nil, err
}
t.ingestLimitsFrontend = ingestLimitsFrontend
logproto.RegisterIngestLimitsFrontendServer(t.Server.GRPC, ingestLimitsFrontend)
limitsproto.RegisterIngestLimitsFrontendServer(t.Server.GRPC, ingestLimitsFrontend)
// Register HTTP handler to check if a tenant exceeds limits
// Returns a JSON response for the frontend to display which

@ -15,7 +15,7 @@ import (
"github.com/grafana/loki/v3/pkg/kafka/client"
"github.com/grafana/loki/v3/pkg/limits"
frontend_client "github.com/grafana/loki/v3/pkg/limits/frontend/client"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/limits/proto"
"github.com/prometheus/client_golang/prometheus"
"github.com/twmb/franz-go/pkg/kgo"
)
@ -33,14 +33,14 @@ func (s *Generator) sendStreamMetadata(ctx context.Context, streamsBatch []distr
return
}
var streamMetadata []*logproto.StreamMetadata
var streamMetadata []*proto.StreamMetadata
for _, stream := range streamsBatch {
streamMetadata = append(streamMetadata, &logproto.StreamMetadata{
streamMetadata = append(streamMetadata, &proto.StreamMetadata{
StreamHash: stream.HashKeyNoShard,
})
}
req := &logproto.ExceedsLimitsRequest{
req := &proto.ExceedsLimitsRequest{
Tenant: tenant,
Streams: streamMetadata,
}
@ -93,13 +93,22 @@ func (s *Generator) sendStreamsToKafka(ctx context.Context, streams []distributo
logSize += uint64(len(entry.Line))
}
// Add metadata record
metadataRecord, err := kafka.EncodeStreamMetadata(partitionID, s.cfg.Kafka.Topic, tenant, stream.HashKeyNoShard, logSize, 0)
metadata := proto.StreamMetadata{
StreamHash: stream.HashKeyNoShard,
EntriesSize: logSize,
}
b, err := metadata.Marshal()
if err != nil {
errCh <- fmt.Errorf("failed to encode stream metadata: %w", err)
errCh <- fmt.Errorf("failed to marshal metadata: %w", err)
return
}
metadataRecord := &kgo.Record{
Key: []byte(tenant),
Value: b,
Partition: partitionID,
Topic: kafka.MetadataTopicFor(s.cfg.Kafka.Topic),
}
// Send to Kafka
produceResults := s.writer.ProduceSync(ctx, []*kgo.Record{metadataRecord})

Loading…
Cancel
Save