@ -8,6 +8,7 @@ import (
"net/http"
"strconv"
"strings"
"sync"
"testing"
"time"
@ -28,6 +29,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/pkg/ingester"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
@ -38,10 +40,6 @@ import (
"github.com/grafana/loki/pkg/validation"
)
const (
numIngesters = 5
)
var (
success = & logproto . PushResponse { }
ctx = user . InjectOrgID ( context . Background ( ) , "test" )
@ -78,7 +76,7 @@ func TestDistributor(t *testing.T) {
expectedError : httpgrpc . Errorf ( http . StatusBadRequest , validation . InvalidLabelsErrorMsg , "{ab\"" , "1:4: parse error: unterminated quoted string" ) ,
} ,
} {
t . Run ( fmt . Sprintf ( "[%d](samp les=%v)" , i , tc . lines ) , func ( t * testing . T ) {
t . Run ( fmt . Sprintf ( "[%d](lin es=%v)" , i , tc . lines ) , func ( t * testing . T ) {
limits := & validation . Limits { }
flagext . DefaultValues ( limits )
limits . EnforceMetricName = false
@ -86,8 +84,7 @@ func TestDistributor(t *testing.T) {
limits . IngestionBurstSizeMB = ingestionRateLimit
limits . MaxLineSize = fe . ByteSize ( tc . maxLineSize )
d := prepare ( t , limits , nil , nil )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
distributors , _ := prepare ( t , 1 , 5 , limits , nil )
request := makeWriteRequest ( tc . lines , 10 )
@ -95,7 +92,7 @@ func TestDistributor(t *testing.T) {
request . Streams [ 0 ] . Labels = ` { ab" `
}
response , err := d . Push ( ctx , request )
response , err := distributors [ i % len ( distributors ) ] . Push ( ctx , request )
assert . Equal ( t , tc . expectedResponse , response )
assert . Equal ( t , tc . expectedError , err )
} )
@ -327,27 +324,107 @@ func Test_IncrementTimestamp(t *testing.T) {
testData := testData
t . Run ( testName , func ( t * testing . T ) {
ingester := & mockIngester { }
d := prepare ( t , testData . limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
_ , err := d . Push ( ctx , testData . push )
ing := & mockIngester { }
distributors , _ := prepare ( t , 1 , 3 , testData . limits , func ( addr string ) ( ring_client . PoolClient , error ) { return ing , nil } )
_ , err := distributors [ 0 ] . Push ( ctx , testData . push )
assert . NoError ( t , err )
assert . Equal ( t , testData . expectedPush , ingester . pushed [ 0 ] )
assert . Equal ( t , testData . expectedPush , ing . pushed [ 0 ] )
} )
}
}
func TestDistributorPushConcurrently ( t * testing . T ) {
limits := & validation . Limits { }
flagext . DefaultValues ( limits )
distributors , ingesters := prepare ( t , 1 , 5 , limits , nil )
numReq := 1
var wg sync . WaitGroup
for i := 0 ; i < numReq ; i ++ {
wg . Add ( 1 )
go func ( n int ) {
defer wg . Done ( )
request := makeWriteRequestWithLabels ( 100 , 100 ,
[ ] string {
fmt . Sprintf ( ` { app="foo-%d"} ` , n ) ,
fmt . Sprintf ( ` { instance="bar-%d"} ` , n ) ,
} ,
)
response , err := distributors [ n % len ( distributors ) ] . Push ( ctx , request )
assert . NoError ( t , err )
assert . Equal ( t , & logproto . PushResponse { } , response )
} ( i )
}
wg . Wait ( )
// make sure the ingesters received the push requests
time . Sleep ( 10 * time . Millisecond )
counter := 0
labels := make ( map [ string ] int )
for i := range ingesters {
pushed := ingesters [ i ] . pushed
counter = counter + len ( pushed )
for _ , pr := range pushed {
for _ , st := range pr . Streams {
labels [ st . Labels ] = labels [ st . Labels ] + 1
}
}
}
assert . Equal ( t , numReq * 3 , counter ) // RF=3
// each stream is present 3 times
for i := 0 ; i < numReq ; i ++ {
l := fmt . Sprintf ( ` { instance="bar-%d"} ` , i )
assert . Equal ( t , 3 , labels [ l ] , "stream %s expected 3 times, got %d" , l , labels [ l ] )
l = fmt . Sprintf ( ` { app="foo-%d"} ` , i )
assert . Equal ( t , 3 , labels [ l ] , "stream %s expected 3 times, got %d" , l , labels [ l ] )
}
}
func TestDistributorPushErrors ( t * testing . T ) {
limits := & validation . Limits { }
flagext . DefaultValues ( limits )
t . Run ( "with RF=3 a single push can fail" , func ( t * testing . T ) {
distributors , ingesters := prepare ( t , 1 , 3 , limits , nil )
ingesters [ 0 ] . failAfter = 5 * time . Millisecond
ingesters [ 1 ] . succeedAfter = 10 * time . Millisecond
ingesters [ 2 ] . succeedAfter = 15 * time . Millisecond
request := makeWriteRequest ( 10 , 64 )
_ , err := distributors [ 0 ] . Push ( ctx , request )
require . NoError ( t , err )
require . Equal ( t , 0 , len ( ingesters [ 0 ] . pushed ) )
require . Equal ( t , 1 , len ( ingesters [ 1 ] . pushed ) )
require . Equal ( t , 1 , len ( ingesters [ 2 ] . pushed ) )
} )
t . Run ( "with RF=3 two push failures result in error" , func ( t * testing . T ) {
distributors , ingesters := prepare ( t , 1 , 3 , limits , nil )
ingesters [ 0 ] . failAfter = 5 * time . Millisecond
ingesters [ 1 ] . succeedAfter = 10 * time . Millisecond
ingesters [ 2 ] . failAfter = 15 * time . Millisecond
request := makeWriteRequest ( 10 , 64 )
_ , err := distributors [ 0 ] . Push ( ctx , request )
require . Error ( t , err )
require . Equal ( t , 0 , len ( ingesters [ 0 ] . pushed ) )
require . Equal ( t , 1 , len ( ingesters [ 1 ] . pushed ) )
require . Equal ( t , 0 , len ( ingesters [ 2 ] . pushed ) )
} )
}
func Test_SortLabelsOnPush ( t * testing . T ) {
limits := & validation . Limits { }
flagext . DefaultValues ( limits )
limits . EnforceMetricName = false
ingester := & mockIngester { }
d := prepare ( t , limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
distributors , _ := prepare ( t , 1 , 5 , limits , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
request := makeWriteRequest ( 10 , 10 )
request . Streams [ 0 ] . Labels = ` { buzz="f", a="b"} `
_ , err := d . Push ( ctx , request )
_ , err := distributors [ 0 ] . Push ( ctx , request )
require . NoError ( t , err )
require . Equal ( t , ` { a="b", buzz="f"} ` , ingester . pushed [ 0 ] . Streams [ 0 ] . Labels )
}
@ -365,11 +442,9 @@ func Test_TruncateLogLines(t *testing.T) {
t . Run ( "it truncates lines to MaxLineSize when MaxLineSizeTruncate is true" , func ( t * testing . T ) {
limits , ingester := setup ( )
distributors , _ := prepare ( t , 1 , 5 , limits , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
d := prepare ( t , limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
_ , err := d . Push ( ctx , makeWriteRequest ( 1 , 10 ) )
_ , err := distributors [ 0 ] . Push ( ctx , makeWriteRequest ( 1 , 10 ) )
require . NoError ( t , err )
require . Len ( t , ingester . pushed [ 0 ] . Streams [ 0 ] . Entries [ 0 ] . Line , 5 )
} )
@ -701,9 +776,8 @@ func Benchmark_SortLabelsOnPush(b *testing.B) {
limits := & validation . Limits { }
flagext . DefaultValues ( limits )
limits . EnforceMetricName = false
ingester := & mockIngester { }
d := prepare ( & testing . T { } , limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
distributors , _ := prepare ( & testing . T { } , 1 , 5 , limits , nil )
d := distributors [ 0 ]
request := makeWriteRequest ( 10 , 10 )
vCtx := d . validator . getValidationContextForTime ( testTime , "123" )
for n := 0 ; n < b . N ; n ++ {
@ -727,17 +801,14 @@ func Benchmark_Push(b *testing.B) {
limits . RejectOldSamples = true
limits . RejectOldSamplesMaxAge = model . Duration ( 24 * time . Hour )
limits . CreationGracePeriod = model . Duration ( 24 * time . Hour )
ingester := & mockIngester { }
d := prepare ( & testing . T { } , limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
distributors , _ := prepare ( & testing . T { } , 1 , 5 , limits , nil )
request := makeWriteRequest ( 100000 , 100 )
b . ResetTimer ( )
b . ReportAllocs ( )
for n := 0 ; n < b . N ; n ++ {
_ , err := d . Push ( ctx , request )
_ , err := distributors [ 0 ] . Push ( ctx , request )
if err != nil {
require . NoError ( b , err )
}
@ -752,9 +823,7 @@ func Benchmark_PushWithLineTruncation(b *testing.B) {
limits . MaxLineSizeTruncate = true
limits . MaxLineSize = 50
ingester := & mockIngester { }
d := prepare ( & testing . T { } , limits , nil , func ( addr string ) ( ring_client . PoolClient , error ) { return ingester , nil } )
defer services . StopAndAwaitTerminated ( context . Background ( ) , d ) //nolint:errcheck
distributors , _ := prepare ( & testing . T { } , 1 , 5 , limits , nil )
request := makeWriteRequest ( 100000 , 100 )
b . ResetTimer ( )
@ -762,7 +831,7 @@ func Benchmark_PushWithLineTruncation(b *testing.B) {
for n := 0 ; n < b . N ; n ++ {
_ , err := d . Push ( ctx , request )
_ , err := distributors [ 0 ] . Push ( ctx , request )
if err != nil {
require . NoError ( b , err )
}
@ -831,26 +900,7 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) {
limits . IngestionRateMB = testData . ingestionRateMB
limits . IngestionBurstSizeMB = testData . ingestionBurstSizeMB
// Init a shared KVStore
kvStore , closer := consul . NewInMemoryClient ( ring . GetCodec ( ) , log . NewNopLogger ( ) , nil )
t . Cleanup ( func ( ) { assert . NoError ( t , closer . Close ( ) ) } )
// Start all expected distributors
distributors := make ( [ ] * Distributor , testData . distributors )
for i := 0 ; i < testData . distributors ; i ++ {
distributors [ i ] = prepare ( t , limits , kvStore , nil )
defer services . StopAndAwaitTerminated ( context . Background ( ) , distributors [ i ] ) //nolint:errcheck
}
// If the distributors ring is setup, wait until the first distributor
// updates to the expected size
if distributors [ 0 ] . distributorsLifecycler != nil {
test . Poll ( t , time . Second , testData . distributors , func ( ) interface { } {
return distributors [ 0 ] . distributorsLifecycler . HealthyInstancesCount ( )
} )
}
// Push samples in multiple requests to the first distributor
distributors , _ := prepare ( t , testData . distributors , 5 , limits , nil )
for _ , push := range testData . pushes {
request := makeWriteRequest ( 1 , push . bytes )
response , err := distributors [ 0 ] . Push ( ctx , request )
@ -867,156 +917,156 @@ func TestDistributor_PushIngestionRateLimiter(t *testing.T) {
}
}
func prepare ( t * testing . T , limits * validation . Limits , kvStore kv . Client , factory func ( addr string ) ( ring_client . PoolClient , error ) ) * Distributor {
var (
distributorConfig Config
clientConfig client . Config
)
flagext . DefaultValues ( & distributorConfig , & clientConfig )
overrides , err := validation . NewOverrides ( * limits , nil )
require . NoError ( t , err )
func prepare ( t * testing . T , numDistributors , numIngesters int , limits * validation . Limits , factory func ( addr string ) ( ring_client . PoolClient , error ) ) ( [ ] * Distributor , [ ] mockIngester ) {
t . Helper ( )
// Mock the ingesters ring
ingesters := map [ string ] * mockIngester { }
ingesters := make ( [ ] mockIngester , numIngesters )
for i := 0 ; i < numIngesters ; i ++ {
ingesters [ fmt . Spr intf ( "ingester%d" , i ) ] = & mockIngester { }
ingesters [ i ] = mockIngester { }
}
ingestersRing := & mockRing {
replicationFactor : 3 ,
}
for addr := range ingesters {
ingestersRing . ingesters = append ( ingestersRing . ingesters , ring . InstanceDesc {
ingesterByAddr := map [ string ] * mockIngester { }
ingesterDescs := map [ string ] ring . InstanceDesc { }
for i := range ingesters {
addr := fmt . Sprintf ( "ingester-%d" , i )
ingesterDescs [ addr ] = ring . InstanceDesc {
Addr : addr ,
} )
State : ring . ACTIVE ,
Timestamp : time . Now ( ) . Unix ( ) ,
RegisteredTimestamp : time . Now ( ) . Add ( - 10 * time . Minute ) . Unix ( ) ,
Tokens : [ ] uint32 { uint32 ( ( math . MaxUint32 / numIngesters ) * i ) } ,
}
ingesterByAddr [ addr ] = & ingesters [ i ]
}
kvStore , closer := consul . NewInMemoryClient ( ring . GetCodec ( ) , log . NewNopLogger ( ) , nil )
err := kvStore . CAS ( context . Background ( ) , ingester . RingKey ,
func ( _ interface { } ) ( interface { } , bool , error ) {
return & ring . Desc {
Ingesters : ingesterDescs ,
} , true , nil
} ,
)
require . NoError ( t , err )
ingestersRing , err := ring . New ( ring . Config {
KVStore : kv . Config {
Mock : kvStore ,
} ,
HeartbeatTimeout : 60 * time . Minute ,
ReplicationFactor : 3 ,
} , ingester . RingKey , ingester . RingKey , nil , nil )
require . NoError ( t , err )
require . NoError ( t , services . StartAndAwaitRunning ( context . Background ( ) , ingestersRing ) )
test . Poll ( t , time . Second , numIngesters , func ( ) interface { } {
return ingestersRing . InstancesCount ( )
} )
loopbackName , err := loki_net . LoopbackInterfaceName ( )
require . NoError ( t , err )
distributors := make ( [ ] * Distributor , numDistributors )
for i := 0 ; i < numDistributors ; i ++ {
var distributorConfig Config
var clientConfig client . Config
flagext . DefaultValues ( & distributorConfig , & clientConfig )
distributorConfig . DistributorRing . HeartbeatPeriod = 100 * time . Millisecond
distributorConfig . DistributorRing . InstanceID = strconv . Itoa ( rand . Int ( ) )
distributorConfig . DistributorRing . KVStore . Mock = kvStore
distributorConfig . DistributorRing . KVStore . Store = "inmemory"
distributorConfig . DistributorRing . InstanceAddr = "127.0.0.1 "
distributorConfig . DistributorRing . InstanceInterfaceNames = [ ] string { loopbackName }
distributorConfig . factory = factory
if factory == nil {
distributorConfig . factory = func ( addr string ) ( ring_client . PoolClient , error ) {
return ingesters [ addr ] , nil
return ingesterByAddr [ addr ] , nil
}
}
d , err := New ( distributorConfig , clientConfig , runtime . DefaultTenantConfigs ( ) , ingestersRing , overrides , nil )
overri des , err := validation . NewOverrides ( * limit s, nil )
require . NoError ( t , err )
require . NoError ( t , services . StartAndAwaitRunning ( context . Background ( ) , d ) )
return d
}
func makeWriteRequest ( lines int , size int ) * logproto . PushRequest {
req := logproto . PushRequest {
Streams : [ ] logproto . Stream {
{
Labels : ` { foo="bar"} ` ,
} ,
} ,
d , err := New ( distributorConfig , clientConfig , runtime . DefaultTenantConfigs ( ) , ingestersRing , overrides , prometheus . NewPedanticRegistry ( ) )
require . NoError ( t , err )
require . NoError ( t , services . StartAndAwaitRunning ( context . Background ( ) , d ) )
distributors [ i ] = d
}
for i := 0 ; i < lines ; i ++ {
// Construct the log line, honoring the input size
line := strconv . Itoa ( i ) + strings . Repeat ( " " , size )
line = line [ : size ]
req . Streams [ 0 ] . Entries = append ( req . Streams [ 0 ] . Entries , logproto . Entry {
Timestamp : time . Now ( ) . Add ( time . Duration ( i ) * time . Millisecond ) ,
Line : line ,
if distributors [ 0 ] . distributorsLifecycler != nil {
test . Poll ( t , time . Second , numDistributors , func ( ) interface { } {
return distributors [ 0 ] . distributorsLifecycler . HealthyInstancesCount ( )
} )
}
return & req
}
type mockIngester struct {
grpc_health_v1 . HealthClient
logproto . PusherClient
pushed [ ] * logproto . PushRequest
t . Cleanup ( func ( ) {
assert . NoError ( t , closer . Close ( ) )
for _ , d := range distributors {
assert . NoError ( t , services . StopAndAwaitTerminated ( context . Background ( ) , d ) )
}
ingestersRing . StopAsync ( )
} )
func ( i * mockIngester ) Push ( ctx context . Context , in * logproto . PushRequest , opts ... grpc . CallOption ) ( * logproto . PushResponse , error ) {
i . pushed = append ( i . pushed , in )
return nil , nil
return distributors , ingesters
}
func ( i * mockIngester ) Close ( ) error {
return nil
}
func makeWriteRequestWithLabels ( lines , size int , labels [ ] string ) * logproto . PushRequest {
streams := make ( [ ] logproto . Stream , len ( labels ) )
for i := 0 ; i < len ( labels ) ; i ++ {
stream := logproto . Stream { Labels : labels [ i ] }
// Copied from Cortex; TODO(twilkie) - factor this our and share it.
// mockRing doesn't do virtual nodes, just returns mod(key) + replicationFactor
// ingesters.
type mockRing struct {
prometheus . Counter
ingesters [ ] ring . InstanceDesc
replicationFactor uint32
}
for j := 0 ; j < lines ; j ++ {
// Construct the log line, honoring the input size
line := strconv . Itoa ( j ) + strings . Repeat ( "0" , size )
line = line [ : size ]
func ( r mockRing ) Get ( key uint32 , op ring . Operation , buf [ ] ring . InstanceDesc , _ [ ] string , _ [ ] string ) ( ring . ReplicationSet , error ) {
result := ring . ReplicationSet {
MaxErrors : 1 ,
Instances : buf [ : 0 ] ,
}
for i := uint32 ( 0 ) ; i < r . replicationFactor ; i ++ {
n := ( key + i ) % uint32 ( len ( r . ingesters ) )
result . Instances = append ( result . Instances , r . ingesters [ n ] )
}
return result , nil
stream . Entries = append ( stream . Entries , logproto . Entry {
Timestamp : time . Now ( ) . Add ( time . Duration ( j ) * time . Millisecond ) ,
Line : line ,
} )
}
func ( r mockRing ) GetAllHealthy ( op ring . Operation ) ( ring . ReplicationSet , error ) {
return r . GetReplicationSetForOperation ( op )
streams [ i ] = stream
}
func ( r mockRing ) GetReplicationSetForOperation ( op ring . Operation ) ( ring . ReplicationSet , error ) {
return ring . ReplicationSet {
Instances : r . ingesters ,
MaxErrors : 1 ,
} , nil
return & logproto . PushRequest {
Streams : streams ,
}
func ( r mockRing ) ReplicationFactor ( ) int {
return int ( r . replicationFactor )
}
func ( r mockRing ) InstancesCount ( ) in t {
return len ( r . ingesters )
func makeWriteRequest ( lines , size int ) * logproto . PushRequest {
return makeWriteRequestWithLabels ( lines , size , [ ] string { ` { foo="bar"} ` } )
}
func ( r mockRing ) Subring ( key uint32 , n int ) ring . ReadRing {
return r
}
type mockIngester struct {
grpc_health_v1 . HealthClient
logproto . PusherClient
func ( r mockRing ) HasInstance ( instanceID string ) bool {
for _ , ing := range r . ingesters {
if ing . Addr != instanceID {
return true
failAfter time . Duration
succeedAfter time . Duration
mu sync . Mutex
pushed [ ] * logproto . PushRequest
}
func ( i * mockIngester ) Push ( ctx context . Context , in * logproto . PushRequest , opts ... grpc . CallOption ) ( * logproto . PushResponse , error ) {
if i . failAfter > 0 {
time . Sleep ( i . failAfter )
return nil , fmt . Errorf ( "push request failed" )
}
return false
if i . succeedAfter > 0 {
time . Sleep ( i . succeedAfter )
}
func ( r mockRing ) ShuffleShard ( identifier string , size int ) ring . ReadRing {
// take advantage of pass by value to bound to size:
r . ingesters = r . ingesters [ : size ]
return r
}
i . mu . Lock ( )
defer i . mu . Unlock ( )
func ( r mockRing ) ShuffleShardWithLookback ( identifier string , size int , lookbackPeriod time . Duration , now time . Time ) ring . ReadRing {
return r
i . pushed = append ( i . pushed , in )
return nil , nil
}
func ( r mockRing ) CleanupShuffleShardCache ( identifier string ) { }
func ( r mockRing ) GetInstanceState ( instanceID string ) ( ring . InstanceState , error ) {
return 0 , nil
func ( i * mockIngester ) Close ( ) error {
return nil
}