@ -6,70 +6,33 @@ import (
"fmt"
"io"
"math"
"math/rand"
"strings"
"sync"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
"github.com/grafana/dskit/grpcclient"
"github.com/grafana/dskit/instrument"
"github.com/grafana/dskit/ring"
ringclient "github.com/grafana/dskit/ring/client"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"golang.org/x/exp/slices"
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/v3/pkg/bloomutils"
"github.com/grafana/loki/v3/pkg/distributor/clientpool"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logqlmodel/stats"
"github.com/grafana/loki/v3/pkg/querier/plan"
"github.com/grafana/loki/v3/pkg/queue"
v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1"
"github.com/grafana/loki/v3/pkg/storage/chunk/cache"
"github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache"
"github.com/grafana/loki/v3/pkg/util/constants"
util_log "github.com/grafana/loki/v3/pkg/util/log "
"github.com/grafana/loki/v3/pkg/util/discovery "
)
var (
// BlocksOwnerRead is the operation used to check the authoritative owners of a block
// (replicas included) that are available for queries (a bloom gateway is available for
// queries only when ACTIVE).
BlocksOwnerRead = ring . NewOp ( [ ] ring . InstanceState { ring . ACTIVE } , nil )
// groupedChunksRefPool pooling slice of logproto.GroupedChunkRefs [64, 128, 256, ..., 65536]
groupedChunksRefPool = queue . NewSlicePool [ * logproto . GroupedChunkRefs ] ( 1 << 6 , 1 << 16 , 2 )
// ringGetBuffersPool pooling for ringGetBuffers to avoid calling ring.MakeBuffersForGet() for each request
ringGetBuffersPool = sync . Pool {
New : func ( ) interface { } {
descs , hosts , zones := ring . MakeBuffersForGet ( )
return & ringGetBuffers {
Descs : descs ,
Hosts : hosts ,
Zones : zones ,
}
} ,
}
)
type ringGetBuffers struct {
Descs [ ] ring . InstanceDesc
Hosts [ ] string
Zones [ ] string
}
func ( buf * ringGetBuffers ) Reset ( ) {
buf . Descs = buf . Descs [ : 0 ]
buf . Hosts = buf . Hosts [ : 0 ]
buf . Zones = buf . Zones [ : 0 ]
}
// GRPCPool represents a pool of gRPC connections to different bloom gateway instances.
// Interfaces are inlined for simplicity to automatically satisfy interface functions.
type GRPCPool struct {
@ -98,19 +61,17 @@ func NewBloomGatewayGRPCPool(address string, opts []grpc.DialOption) (*GRPCPool,
type ClientConfig struct {
// PoolConfig defines the behavior of the gRPC connection pool used to communicate
// with the Bloom Gateway.
// It is defined at the distributors YAML section and reused here.
PoolConfig clientpool . PoolConfig ` yaml:"pool_config,omitempty" doc:"description=Configures the behavior of the connection pool." `
PoolConfig PoolConfig ` yaml:"pool_config,omitempty" doc:"description=Configures the behavior of the connection pool." `
// GRPCClientConfig configures the gRPC connection between the Bloom Gateway client and the server.
GRPCClientConfig grpcclient . Config ` yaml:"grpc_client_config" `
// Ring is the Bloom Gateway ring used to find the appropriate Bloom Gateway instance
// this client should talk to.
Ring ring . ReadRing ` yaml:"-" `
// Cache configures the cache used to store the results of the Bloom Gateway server.
Cache CacheConfig ` yaml:"results_cache,omitempty" `
CacheResults bool ` yaml:"cache_results" `
// Client sharding using DNS disvovery and jumphash
Addresses string ` yaml:"addresses,omitempty" `
}
// RegisterFlags registers flags for the Bloom Gateway client configuration.
@ -122,7 +83,9 @@ func (i *ClientConfig) RegisterFlags(f *flag.FlagSet) {
func ( i * ClientConfig ) RegisterFlagsWithPrefix ( prefix string , f * flag . FlagSet ) {
i . GRPCClientConfig . RegisterFlagsWithPrefix ( prefix + "grpc" , f )
i . Cache . RegisterFlagsWithPrefix ( prefix + "cache." , f )
i . PoolConfig . RegisterFlagsWithPrefix ( prefix + "pool." , f )
f . BoolVar ( & i . CacheResults , prefix + "cache_results" , false , "Flag to control whether to cache bloom gateway client requests/responses." )
f . StringVar ( & i . Addresses , prefix + "addresses" , "" , "Comma separated addresses list in DNS Service Discovery format: https://grafana.com/docs/mimir/latest/configure/about-dns-service-discovery/#supported-discovery-modes" )
}
func ( i * ClientConfig ) Validate ( ) error {
@ -130,12 +93,20 @@ func (i *ClientConfig) Validate() error {
return errors . Wrap ( err , "grpc client config" )
}
if err := i . PoolConfig . Validate ( ) ; err != nil {
return errors . Wrap ( err , "pool config" )
}
if i . CacheResults {
if err := i . Cache . Validate ( ) ; err != nil {
return errors . Wrap ( err , "cache config" )
}
}
if i . Addresses == "" {
return errors . New ( "addresses requires a list of comma separated strings in DNS service discovery format with at least one item" )
}
return nil
}
@ -144,33 +115,25 @@ type Client interface {
}
type GatewayClient struct {
cfg ClientConfig
limits Limits
logger log . Logger
metrics * clientMetrics
pool * ringclient . Pool
ring ring . ReadRing
cfg ClientConfig
limits Limits
logger log . Logger
metrics * clientMetrics
pool * JumpHashClient Pool
dnsProvider * discovery . DNS
}
func NewClient (
cfg ClientConfig ,
readRing ring . ReadRing ,
limits Limits ,
registerer prometheus . Registerer ,
logger log . Logger ,
metricsNamespace string ,
cacheGen resultscache . CacheGenNumberLoader ,
retentionEnabled bool ,
) ( * GatewayClient , error ) {
latency := promauto . With ( registerer ) . NewHistogramVec ( prometheus . HistogramOpts {
Namespace : constants . Loki ,
Subsystem : "bloom_gateway" ,
Name : "request_duration_seconds" ,
Help : "Time (in seconds) spent serving requests when using the bloom gateway" ,
Buckets : instrument . DefBuckets ,
} , [ ] string { "operation" , "status_code" } )
dialOpts , err := cfg . GRPCClientConfig . DialOption ( grpcclient . Instrument ( latency ) )
metrics := newClientMetrics ( registerer )
dialOpts , err := cfg . GRPCClientConfig . DialOption ( grpcclient . Instrument ( metrics . requestLatency ) )
if err != nil {
return nil , err
}
@ -206,29 +169,35 @@ func NewClient(
return pool , nil
}
dnsProvider := discovery . NewDNS ( logger , cfg . PoolConfig . CheckInterval , cfg . Addresses , nil )
// Make an attempt to do one DNS lookup so we can start with addresses
dnsProvider . RunOnce ( )
clientPool := ringclient . NewPool (
"bloom-gateway" ,
ringclient . PoolConfig ( cfg . PoolConfig ) ,
func ( ) ( [ ] string , error ) { return dnsProvider . Addresses ( ) , nil } ,
ringclient . PoolAddrFunc ( poolFactory ) ,
metrics . clients ,
logger ,
)
pool := NewJumpHashClientPool ( clientPool , dnsProvider , cfg . PoolConfig . CheckInterval , logger )
pool . Start ( )
return & GatewayClient {
cfg : cfg ,
logger : logger ,
limits : limits ,
metrics : newClientMetrics ( registerer ) ,
pool : clientpool . NewPool ( "bloom-gateway" , cfg . PoolConfig , cfg . Ring , ringclient . PoolAddrFunc ( poolFactory ) , logger , metricsNamespace ) ,
ring : readRing ,
cfg : cfg ,
logger : logger ,
limits : limits ,
metrics : metrics ,
pool : p ool,
dnsProvider : dnsProvider , // keep reference so we can stop it when the client is closed
} , nil
}
func JoinFunc [ S ~ [ ] E , E any ] ( elems S , sep string , f func ( e E ) string ) string {
res := make ( [ ] string , len ( elems ) )
for i := range elems {
res [ i ] = f ( elems [ i ] )
}
return strings . Join ( res , sep )
}
func shuffleAddrs ( addrs [ ] string ) [ ] string {
rand . Shuffle ( len ( addrs ) , func ( i , j int ) {
addrs [ i ] , addrs [ j ] = addrs [ j ] , addrs [ i ]
} )
return addrs
func ( c * GatewayClient ) Close ( ) {
c . pool . Stop ( )
c . dnsProvider . Stop ( )
}
// FilterChunkRefs implements Client
@ -237,19 +206,23 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t
return groups , nil
}
subRing := GetShuffleShardingSubring ( c . ring , tenant , c . limits )
rs , err := subRing . GetAllHealthy ( BlocksOwnerRead )
if err != nil {
return nil , errors . Wrap ( err , "bloom gateway get healthy instances" )
clients := make ( map [ string ] [ ] * logproto . GroupedChunkRefs )
for _ , g := range groups {
addr , err := c . pool . AddrForFingerprint ( g . Fingerprint )
if err != nil {
return nil , errors . Wrap ( err , "server address for fingerprint" )
}
clients [ addr ] = append ( clients [ addr ] , g )
}
servers , err := replicationSetsWithBounds ( subRing , rs . Instances )
if err != nil {
return nil , errors . Wrap ( err , "bloom gateway get replication sets" )
servers := make ( [ ] addrWithGroups , 0 , len ( clients ) )
for k , v := range clients {
servers = append ( servers , addrWithGroups {
groups : v ,
addr : k ,
} )
}
servers = partitionByReplicationSet ( groups , servers )
if len ( servers ) > 0 {
// cache locality score (higher is better):
// `% keyspace / % instances`. Ideally converges to 1 (querying x% of keyspace requires x% of instances),
@ -258,23 +231,20 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t
// overlap on instances to the left and right of the range.
firstFp , lastFp := groups [ 0 ] . Fingerprint , groups [ len ( groups ) - 1 ] . Fingerprint
pctKeyspace := float64 ( lastFp - firstFp ) / float64 ( math . MaxUint64 )
pctInstances := float64 ( len ( servers ) ) / float64 ( len ( rs . Instances ) )
pctInstances := float64 ( len ( servers ) ) / float64 ( max ( 1 , len ( c . pool . Addrs ( ) ) ) )
cacheLocalityScore := pctKeyspace / pctInstances
c . metrics . cacheLocalityScore . Observe ( cacheLocalityScore )
}
results := make ( [ ] [ ] * logproto . GroupedChunkRefs , len ( servers ) )
count := 0
err = concurrency . ForEachJob ( ctx , len ( servers ) , len ( servers ) , func ( ctx context . Context , i int ) error {
err : = concurrency . ForEachJob ( ctx , len ( servers ) , len ( servers ) , func ( ctx context . Context , i int ) error {
rs := servers [ i ]
// randomize order of addresses so we don't hotspot the first server in the list
addrs := shuffleAddrs ( rs . rs . GetAddresses ( ) )
level . Info ( c . logger ) . Log (
"msg" , "do FilterChunkRefs for addresses" ,
"progress" , fmt . Sprintf ( "%d/%d" , i + 1 , len ( servers ) ) ,
"bounds" , JoinFunc ( rs . ranges , "," , func ( e v1 . FingerprintBounds ) string { return e . String ( ) } ) ,
"addrs" , strings . Join ( addrs , "," ) ,
"addr" , rs . addr ,
"from" , from . Time ( ) ,
"through" , through . Time ( ) ,
"num_refs" , len ( rs . groups ) ,
@ -282,7 +252,7 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t
"plan_hash" , plan . Hash ( ) ,
)
return c . doForAddrs ( addrs , func ( client logproto . BloomGatewayClient ) error {
return c . doForAddrs ( [ ] string { rs . addr } , func ( client logproto . BloomGatewayClient ) error {
req := & logproto . FilterChunkRefRequest {
From : from ,
Through : through ,
@ -316,7 +286,6 @@ func flatten(input [][]*logproto.GroupedChunkRefs, n int) []*logproto.GroupedChu
// doForAddrs sequetially calls the provided callback function fn for each
// address in given slice addrs until the callback function does not return an
// error.
// TODO(owen-d): parallelism
func ( c * GatewayClient ) doForAddrs ( addrs [ ] string , fn func ( logproto . BloomGatewayClient ) error ) error {
var err error
var poolClient ringclient . PoolClient
@ -337,116 +306,7 @@ func (c *GatewayClient) doForAddrs(addrs []string, fn func(logproto.BloomGateway
return err
}
func mapTokenRangeToFingerprintRange ( r bloomutils . Range [ uint32 ] ) v1 . FingerprintBounds {
minFp := uint64 ( r . Min ) << 32
maxFp := uint64 ( r . Max ) << 32
return v1 . NewBounds (
model . Fingerprint ( minFp ) ,
model . Fingerprint ( maxFp | math . MaxUint32 ) ,
)
}
type rsWithRanges struct {
rs ring . ReplicationSet
ranges [ ] v1 . FingerprintBounds
type addrWithGroups struct {
addr string
groups [ ] * logproto . GroupedChunkRefs
}
func replicationSetsWithBounds ( subRing ring . ReadRing , instances [ ] ring . InstanceDesc ) ( [ ] rsWithRanges , error ) {
bufDescs , bufHosts , bufZones := ring . MakeBuffersForGet ( )
servers := make ( [ ] rsWithRanges , 0 , len ( instances ) )
for _ , inst := range instances {
tr , err := bloomutils . TokenRangesForInstance ( inst . Id , instances )
if err != nil {
return nil , errors . Wrap ( err , "bloom gateway get ring" )
}
if len ( tr ) == 0 {
level . Warn ( util_log . Logger ) . Log (
"subroutine" , "replicationSetsWithBounds" ,
"msg" , "instance has no token ranges - should not be possible" ,
"instance" , inst . Id ,
"n_instances" , len ( instances ) ,
)
continue
}
// NB(owen-d): this will send requests to the wrong nodes if RF>1 since it only checks the
// first token when assigning replicasets
rs , err := subRing . Get ( tr [ 0 ] , BlocksOwnerRead , bufDescs , bufHosts , bufZones )
if err != nil {
return nil , errors . Wrap ( err , "bloom gateway get ring" )
}
bounds := make ( [ ] v1 . FingerprintBounds , 0 , len ( tr ) / 2 )
for i := 0 ; i < len ( tr ) ; i += 2 {
b := v1 . NewBounds (
model . Fingerprint ( uint64 ( tr [ i ] ) << 32 ) ,
model . Fingerprint ( uint64 ( tr [ i + 1 ] ) << 32 | math . MaxUint32 ) ,
)
bounds = append ( bounds , b )
}
servers = append ( servers , rsWithRanges {
rs : rs ,
ranges : bounds ,
} )
}
return servers , nil
}
func partitionByReplicationSet ( fingerprints [ ] * logproto . GroupedChunkRefs , rs [ ] rsWithRanges ) ( result [ ] rsWithRanges ) {
for _ , inst := range rs {
for _ , bounds := range inst . ranges {
min , _ := slices . BinarySearchFunc ( fingerprints , bounds , func ( g * logproto . GroupedChunkRefs , b v1 . FingerprintBounds ) int {
if g . Fingerprint < uint64 ( b . Min ) {
return - 1
} else if g . Fingerprint > uint64 ( b . Min ) {
return 1
}
return 0
} )
max , _ := slices . BinarySearchFunc ( fingerprints , bounds , func ( g * logproto . GroupedChunkRefs , b v1 . FingerprintBounds ) int {
if g . Fingerprint <= uint64 ( b . Max ) {
return - 1
} else if g . Fingerprint > uint64 ( b . Max ) {
return 1
}
return 0
} )
// fingerprint is out of boundaries
if min == len ( fingerprints ) || max == 0 {
continue
}
inst . groups = append ( inst . groups , fingerprints [ min : max ] ... )
}
if len ( inst . groups ) > 0 {
result = append ( result , inst )
}
}
return result
}
// GetShuffleShardingSubring returns the subring to be used for a given user.
// This function should be used both by index gateway servers and clients in
// order to guarantee the same logic is used.
func GetShuffleShardingSubring ( ring ring . ReadRing , tenantID string , limits Limits ) ring . ReadRing {
shardSize := limits . BloomGatewayShardSize ( tenantID )
// A shard size of 0 means shuffle sharding is disabled for this specific user,
// so we just return the full ring so that indexes will be sharded across all index gateways.
// Since we set the shard size to replication factor if shard size is 0, this
// can only happen if both the shard size and the replication factor are set
// to 0.
if shardSize <= 0 {
return ring
}
return ring . ShuffleShard ( tenantID , shardSize )
}