Add basic structure of bloom gateways (#10782)

### Summary

This pull requests adds the basic structure for the new bloom gateway component.

- Adds new `bloom-gateway` target that runs with multiple instances joined by a ring
- Adds a querier and client component on the index gateway to filter chunk refs
- Adds the gRPC protobuf definitions for commication between index gateways and bloom gateways
- Adds a store component used on the bloom gateways to query binary bloom files

```

			     Querier   Query Frontend
			        |           |
			................................... service boundary
			        |           |
			        +----+------+
			             |
			     indexgateway.Gateway**
			             |
			   bloomgateway.BloomQuerier
			             |
			   bloomgateway.GatewayClient
			             |
			  logproto.BloomGatewayClient
			             |
			................................... service boundary
			             |
			      bloomgateway.Gateway
			             |
			       bloomshipper.Store
			             |
			      bloomshipper.Shipper
			             |
                        bloomshipper.BloomFileClient**
			             |
			        ObjectClient**
			             |
			................................... service boundary
			             |
		         object storage

** not part of this PR
```

This PR still contains a lot of TODOs and possibilities for optimisations, which will be addressed in subsequent pull requests.

Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
pull/10916/head
Christian Haudum 2 years ago committed by GitHub
parent 048587f1a6
commit b49b3ce6e1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 131
      docs/sources/configure/_index.md
  2. 162
      pkg/bloomgateway/bloomgateway.go
  3. 193
      pkg/bloomgateway/bloomgateway_test.go
  4. 310
      pkg/bloomgateway/client.go
  5. 121
      pkg/bloomgateway/client_test.go
  6. 47
      pkg/bloomgateway/config.go
  7. 64
      pkg/bloomgateway/querier.go
  8. 86
      pkg/bloomgateway/querier_test.go
  9. 264
      pkg/bloomgateway/ringmanager.go
  10. 162
      pkg/bloomgateway/sharding.go
  11. 12
      pkg/distributor/clientpool/ingester_client_pool.go
  12. 3
      pkg/distributor/distributor.go
  13. 2
      pkg/ingester/client/client.go
  14. 1519
      pkg/logproto/bloomgateway.pb.go
  15. 47
      pkg/logproto/bloomgateway.proto
  16. 639
      pkg/logproto/logproto.pb.go
  17. 6
      pkg/logproto/logproto.proto
  18. 24
      pkg/loki/config_wrapper.go
  19. 26
      pkg/loki/loki.go
  20. 65
      pkg/loki/modules.go
  21. 2
      pkg/querier/ingester_querier.go
  22. 2
      pkg/storage/factory.go
  23. 9
      pkg/storage/stores/series/series_index_gateway_store.go
  24. 2
      pkg/storage/stores/shipper/bloomshipper/client.go
  25. 3
      pkg/storage/stores/shipper/bloomshipper/config/config.go
  26. 100
      pkg/storage/stores/shipper/bloomshipper/shipper.go
  27. 86
      pkg/storage/stores/shipper/bloomshipper/shipper_test.go
  28. 70
      pkg/storage/stores/shipper/bloomshipper/store.go
  29. 11
      pkg/storage/stores/shipper/bloomshipper/store_test.go
  30. 2
      pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go
  31. 26
      pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go
  32. 4
      pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go
  33. 2
      pkg/util/limiter/combined_limits.go
  34. 6
      pkg/validation/limits.go
  35. 6
      tools/doc-generator/parse/root_blocks.go

@ -161,6 +161,10 @@ Pass the `-config.expand-env` flag at the command line to enable this way of set
# object store.
[index_gateway: <index_gateway>]
# The bloom_gateway block configures the Loki bloom gateway server, responsible
# for serving queries for filtering chunks based on filter expressions.
[bloom_gateway: <bloom_gateway>]
# The storage_config block configures one of many possible stores for both the
# index and chunks. Which configuration to be picked should be defined in
# schema_config block.
@ -1687,6 +1691,125 @@ ring:
[replication_factor: <int> | default = 3]
```
### bloom_gateway
The `bloom_gateway` block configures the Loki bloom gateway server, responsible for serving queries for filtering chunks based on filter expressions.
```yaml
# Defines the ring to be used by the bloom gateway servers and clients. In case
# this isn't configured, this block supports inheriting configuration from the
# common ring section.
ring:
kvstore:
# Backend storage to use for the ring. Supported values are: consul, etcd,
# inmemory, memberlist, multi.
# CLI flag: -bloom-gateway.ring.store
[store: <string> | default = "consul"]
# The prefix for the keys in the store. Should end with a /.
# CLI flag: -bloom-gateway.ring.prefix
[prefix: <string> | default = "collectors/"]
# Configuration for a Consul client. Only applies if the selected kvstore is
# consul.
# The CLI flags prefix for this block configuration is: bloom-gateway.ring
[consul: <consul>]
# Configuration for an ETCD v3 client. Only applies if the selected kvstore
# is etcd.
# The CLI flags prefix for this block configuration is: bloom-gateway.ring
[etcd: <etcd>]
multi:
# Primary backend storage used by multi-client.
# CLI flag: -bloom-gateway.ring.multi.primary
[primary: <string> | default = ""]
# Secondary backend storage used by multi-client.
# CLI flag: -bloom-gateway.ring.multi.secondary
[secondary: <string> | default = ""]
# Mirror writes to secondary store.
# CLI flag: -bloom-gateway.ring.multi.mirror-enabled
[mirror_enabled: <boolean> | default = false]
# Timeout for storing value to secondary store.
# CLI flag: -bloom-gateway.ring.multi.mirror-timeout
[mirror_timeout: <duration> | default = 2s]
# Period at which to heartbeat to the ring. 0 = disabled.
# CLI flag: -bloom-gateway.ring.heartbeat-period
[heartbeat_period: <duration> | default = 15s]
# The heartbeat timeout after which compactors are considered unhealthy within
# the ring. 0 = never (timeout disabled).
# CLI flag: -bloom-gateway.ring.heartbeat-timeout
[heartbeat_timeout: <duration> | default = 1m]
# File path where tokens are stored. If empty, tokens are not stored at
# shutdown and restored at startup.
# CLI flag: -bloom-gateway.ring.tokens-file-path
[tokens_file_path: <string> | default = ""]
# True to enable zone-awareness and replicate blocks across different
# availability zones.
# CLI flag: -bloom-gateway.ring.zone-awareness-enabled
[zone_awareness_enabled: <boolean> | default = false]
# Instance ID to register in the ring.
# CLI flag: -bloom-gateway.ring.instance-id
[instance_id: <string> | default = "<hostname>"]
# Name of network interface to read address from.
# CLI flag: -bloom-gateway.ring.instance-interface-names
[instance_interface_names: <list of strings> | default = [<private network interfaces>]]
# Port to advertise in the ring (defaults to server.grpc-listen-port).
# CLI flag: -bloom-gateway.ring.instance-port
[instance_port: <int> | default = 0]
# IP address to advertise in the ring.
# CLI flag: -bloom-gateway.ring.instance-addr
[instance_addr: <string> | default = ""]
# The availability zone where this instance is running. Required if
# zone-awareness is enabled.
# CLI flag: -bloom-gateway.ring.instance-availability-zone
[instance_availability_zone: <string> | default = ""]
# Enable using a IPv6 instance address.
# CLI flag: -bloom-gateway.ring.instance-enable-ipv6
[instance_enable_ipv6: <boolean> | default = false]
# Factor for data replication on the bloom gateways.
# CLI flag: -bloom-gateway.replication-factor
[replication_factor: <int> | default = 3]
# Flag to enable or disable the usage of the bloom gatway component.
# CLI flag: -bloom-gateway.enabled
[enabled: <boolean> | default = false]
client:
# Configures the behavior of the connection pool.
pool_config:
[client_cleanup_period: <duration>]
[health_check_ingesters: <boolean>]
[remote_timeout: <duration>]
# The grpc_client block configures the gRPC client used to communicate between
# two Loki components.
# The CLI flags prefix for this block configuration is:
# bloom-gateway-client.grpc
[grpc_client_config: <grpc_client>]
# Flag to control whether requests sent to the gateway should be logged or
# not.
# CLI flag: -bloom-gateway-client.log-gateway-requests
[log_gateway_requests: <boolean> | default = false]
```
### storage_config
The `storage_config` block configures one of many possible stores for both the index and chunks. Which configuration to be picked should be defined in schema_config block.
@ -2722,6 +2845,11 @@ shard_streams:
# CLI flag: -index-gateway.shard-size
[index_gateway_shard_size: <int> | default = 0]
# The shard size defines how many bloom gateways should be used by a tenant for
# querying.
# CLI flag: -bloom-gateway.shard-size
[bloom_gateway_shard_size: <int> | default = 1]
# Allow user to send structured metadata in push payload.
# CLI flag: -validation.allow-structured-metadata
[allow_structured_metadata: <boolean> | default = false]
@ -3362,6 +3490,7 @@ ring:
Configuration for a Consul client. Only applies if the selected kvstore is `consul`. The supported CLI flags `<prefix>` used to reference this configuration block are:
- `bloom-gateway.ring`
- `common.storage.ring`
- `compactor.ring`
- `distributor.ring`
@ -3406,6 +3535,7 @@ Configuration for a Consul client. Only applies if the selected kvstore is `cons
Configuration for an ETCD v3 client. Only applies if the selected kvstore is `etcd`. The supported CLI flags `<prefix>` used to reference this configuration block are:
- `bloom-gateway.ring`
- `common.storage.ring`
- `compactor.ring`
- `distributor.ring`
@ -3707,6 +3837,7 @@ When a memberlist config with atleast 1 join_members is defined, kvstore of type
The `grpc_client` block configures the gRPC client used to communicate between two Loki components. The supported CLI flags `<prefix>` used to reference this configuration block are:
- `bigtable`
- `bloom-gateway-client.grpc`
- `boltdb.shipper.index-gateway-client.grpc`
- `frontend.grpc-client-config`
- `ingester.client`

@ -0,0 +1,162 @@
/*
Bloom Gateway package
The bloom gateway is a component that can be run as a standalone microserivce
target and provides capabilities for filtering ChunkRefs based on a given list
of line filter expressions.
Querier Query Frontend
| |
................................... service boundary
| |
+----+------+
|
indexgateway.Gateway
|
bloomgateway.BloomQuerier
|
bloomgateway.GatewayClient
|
logproto.BloomGatewayClient
|
................................... service boundary
|
bloomgateway.Gateway
|
bloomshipper.Store
|
bloomshipper.Shipper
|
bloomshipper.BloomFileClient
|
ObjectClient
|
................................... service boundary
|
object storage
*/
package bloomgateway
import (
"context"
"sort"
"github.com/go-kit/log"
"github.com/grafana/dskit/services"
"github.com/grafana/dskit/tenant"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
)
var errGatewayUnhealthy = errors.New("bloom-gateway is unhealthy in the ring")
var errInvalidTenant = errors.New("invalid tenant in chunk refs")
type metrics struct{}
func newMetrics(r prometheus.Registerer) *metrics {
return &metrics{}
}
type Gateway struct {
services.Service
cfg Config
logger log.Logger
metrics *metrics
bloomStore bloomshipper.Store
sharding ShardingStrategy
}
// New returns a new instance of the Bloom Gateway.
func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, shardingStrategy ShardingStrategy, cm storage.ClientMetrics, logger log.Logger, reg prometheus.Registerer) (*Gateway, error) {
g := &Gateway{
cfg: cfg,
logger: logger,
metrics: newMetrics(reg),
sharding: shardingStrategy,
}
client, err := bloomshipper.NewBloomClient(schemaCfg.Configs, storageCfg, cm)
if err != nil {
return nil, err
}
bloomShipper, err := bloomshipper.NewShipper(client, storageCfg.BloomShipperConfig, logger)
if err != nil {
return nil, err
}
bloomStore, err := bloomshipper.NewBloomStore(bloomShipper)
if err != nil {
return nil, err
}
g.bloomStore = bloomStore
g.Service = services.NewIdleService(g.starting, g.stopping)
return g, nil
}
func (g *Gateway) starting(ctx context.Context) error {
return nil
}
func (g *Gateway) stopping(_ error) error {
g.bloomStore.Stop()
return nil
}
// FilterChunkRefs implements BloomGatewayServer
func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunkRefRequest) (*logproto.FilterChunkRefResponse, error) {
tenantID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
for _, ref := range req.Refs {
if ref.UserID != tenantID {
return nil, errors.Wrapf(errInvalidTenant, "expected chunk refs from tenant %s, got tenant %s", tenantID, ref.UserID)
}
}
// Sort ChunkRefs by fingerprint in ascending order
sort.Slice(req.Refs, func(i, j int) bool {
return req.Refs[i].Fingerprint < req.Refs[j].Fingerprint
})
chunkRefs := req.Refs
// Only query bloom filters if filters are present
if len(req.Filters) > 0 {
chunkRefs, err = g.bloomStore.FilterChunkRefs(ctx, tenantID, req.From.Time(), req.Through.Time(), req.Refs, req.Filters...)
if err != nil {
return nil, err
}
}
// TODO(chaudum): Re-use buffers for response.
resp := make([]*logproto.GroupedChunkRefs, 0)
for idx, chunkRef := range chunkRefs {
fp := chunkRef.Fingerprint
shortRef := &logproto.ShortRef{From: chunkRef.From, Through: chunkRef.Through, Checksum: chunkRef.Checksum}
if idx == 0 || fp > resp[len(resp)-1].Fingerprint {
r := &logproto.GroupedChunkRefs{
Fingerprint: fp,
Tenant: tenantID,
Refs: []*logproto.ShortRef{shortRef},
}
resp = append(resp, r)
continue
}
resp[len(resp)-1].Refs = append(resp[len(resp)-1].Refs, shortRef)
}
return &logproto.FilterChunkRefResponse{ChunkRefs: resp}, nil
}

@ -0,0 +1,193 @@
package bloomgateway
import (
"context"
"testing"
"time"
"github.com/go-kit/log"
"github.com/grafana/dskit/kv"
"github.com/grafana/dskit/kv/consul"
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/services"
"github.com/grafana/dskit/user"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/util"
)
func parseDayTime(s string) config.DayTime {
t, err := time.Parse("2006-01-02", s)
if err != nil {
panic(err)
}
return config.DayTime{
Time: model.TimeFromUnix(t.Unix()),
}
}
func TestBloomGateway_StartStopService(t *testing.T) {
ss := NewNoopStrategy()
logger := log.NewNopLogger()
reg := prometheus.NewRegistry()
cm := storage.NewClientMetrics()
t.Cleanup(cm.Unregister)
p := config.PeriodConfig{
From: parseDayTime("2023-09-01"),
IndexType: config.TSDBType,
ObjectType: config.StorageTypeFileSystem,
Schema: "v13",
RowShards: 16,
}
schemaCfg := config.SchemaConfig{
Configs: []config.PeriodConfig{p},
}
storageCfg := storage.Config{
FSConfig: local.FSConfig{
Directory: t.TempDir(),
},
}
t.Run("start and stop bloom gateway", func(t *testing.T) {
kvStore, closer := consul.NewInMemoryClient(ring.GetCodec(), logger, reg)
t.Cleanup(func() {
closer.Close()
})
cfg := Config{
Enabled: true,
Ring: RingCfg{
RingConfig: util.RingConfig{
KVStore: kv.Config{
Mock: kvStore,
},
},
ReplicationFactor: 1,
},
}
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg)
require.NoError(t, err)
err = services.StartAndAwaitRunning(context.Background(), gw)
require.NoError(t, err)
err = services.StopAndAwaitTerminated(context.Background(), gw)
require.NoError(t, err)
})
}
func TestBloomGateway_FilterChunkRefs(t *testing.T) {
tenantID := "test"
ss := NewNoopStrategy()
logger := log.NewNopLogger()
reg := prometheus.NewRegistry()
cm := storage.NewClientMetrics()
t.Cleanup(cm.Unregister)
p := config.PeriodConfig{
From: parseDayTime("2023-09-01"),
IndexType: config.TSDBType,
ObjectType: config.StorageTypeFileSystem,
Schema: "v13",
RowShards: 16,
}
schemaCfg := config.SchemaConfig{
Configs: []config.PeriodConfig{p},
}
storageCfg := storage.Config{
FSConfig: local.FSConfig{
Directory: t.TempDir(),
},
}
kvStore, closer := consul.NewInMemoryClient(ring.GetCodec(), logger, reg)
t.Cleanup(func() {
closer.Close()
})
cfg := Config{
Enabled: true,
Ring: RingCfg{
RingConfig: util.RingConfig{
KVStore: kv.Config{
Mock: kvStore,
},
},
ReplicationFactor: 1,
},
}
t.Run("returns unfiltered chunk refs if no filters provided", func(t *testing.T) {
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg)
require.NoError(t, err)
ts, _ := time.Parse("2006-01-02 15:04", "2023-10-03 10:00")
now := model.TimeFromUnix(ts.Unix())
chunkRefs := []*logproto.ChunkRef{
{Fingerprint: 3000, UserID: tenantID, From: now.Add(-24 * time.Hour), Through: now.Add(-23 * time.Hour), Checksum: 1},
{Fingerprint: 1000, UserID: tenantID, From: now.Add(-22 * time.Hour), Through: now.Add(-21 * time.Hour), Checksum: 2},
{Fingerprint: 2000, UserID: tenantID, From: now.Add(-20 * time.Hour), Through: now.Add(-19 * time.Hour), Checksum: 3},
{Fingerprint: 1000, UserID: tenantID, From: now.Add(-23 * time.Hour), Through: now.Add(-22 * time.Hour), Checksum: 4},
}
req := &logproto.FilterChunkRefRequest{
From: now.Add(-24 * time.Hour),
Through: now,
Refs: chunkRefs,
}
ctx := user.InjectOrgID(context.Background(), tenantID)
res, err := gw.FilterChunkRefs(ctx, req)
require.NoError(t, err)
require.Equal(t, &logproto.FilterChunkRefResponse{
ChunkRefs: []*logproto.GroupedChunkRefs{
{Fingerprint: 1000, Tenant: tenantID, Refs: []*logproto.ShortRef{
{From: 1696248000000, Through: 1696251600000, Checksum: 2},
{From: 1696244400000, Through: 1696248000000, Checksum: 4},
}},
{Fingerprint: 2000, Tenant: tenantID, Refs: []*logproto.ShortRef{
{From: 1696255200000, Through: 1696258800000, Checksum: 3},
}},
{Fingerprint: 3000, Tenant: tenantID, Refs: []*logproto.ShortRef{
{From: 1696240800000, Through: 1696244400000, Checksum: 1},
}},
},
}, res)
})
t.Run("returns error if chunk refs do not belong to tenant", func(t *testing.T) {
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg)
require.NoError(t, err)
ts, _ := time.Parse("2006-01-02 15:04", "2023-10-03 10:00")
now := model.TimeFromUnix(ts.Unix())
chunkRefs := []*logproto.ChunkRef{
{Fingerprint: 1000, UserID: tenantID, From: now.Add(-22 * time.Hour), Through: now.Add(-21 * time.Hour), Checksum: 1},
{Fingerprint: 2000, UserID: "other", From: now.Add(-20 * time.Hour), Through: now.Add(-19 * time.Hour), Checksum: 2},
}
req := &logproto.FilterChunkRefRequest{
From: now.Add(-24 * time.Hour),
Through: now,
Refs: chunkRefs,
}
ctx := user.InjectOrgID(context.Background(), tenantID)
_, err = gw.FilterChunkRefs(ctx, req)
require.Error(t, err)
require.Equal(t, "expected chunk refs from tenant test, got tenant other: invalid tenant in chunk refs", err.Error())
})
}

@ -0,0 +1,310 @@
package bloomgateway
import (
"context"
"flag"
"fmt"
"io"
"math"
"math/rand"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"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"
"google.golang.org/grpc"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/pkg/distributor/clientpool"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/util"
)
// 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 {
grpc_health_v1.HealthClient
logproto.BloomGatewayClient
io.Closer
}
// NewBloomGatewayGRPCPool instantiates a new pool of GRPC connections for the Bloom Gateway
// Internally, it also instantiates a protobuf bloom gateway client and a health client.
func NewBloomGatewayGRPCPool(address string, opts []grpc.DialOption) (*GRPCPool, error) {
conn, err := grpc.Dial(address, opts...)
if err != nil {
return nil, errors.Wrap(err, "new grpc pool dial")
}
return &GRPCPool{
Closer: conn,
HealthClient: grpc_health_v1.NewHealthClient(conn),
BloomGatewayClient: logproto.NewBloomGatewayClient(conn),
}, nil
}
// IndexGatewayClientConfig configures the Index Gateway client used to
// communicate with the Index Gateway server.
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."`
// GRPCClientConfig configures the gRPC connection between the Bloom Gateway client and the server.
GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"`
// LogGatewayRequests configures if requests sent to the gateway should be logged or not.
// The log messages are of type debug and contain the address of the gateway and the relevant tenant.
LogGatewayRequests bool `yaml:"log_gateway_requests"`
// Ring is the Bloom Gateway ring used to find the appropriate Bloom Gateway instance
// this client should talk to.
Ring ring.ReadRing `yaml:"-"`
}
// RegisterFlags registers flags for the Bloom Gateway client configuration.
func (i *ClientConfig) RegisterFlags(f *flag.FlagSet) {
i.RegisterFlagsWithPrefix("bloom-gateway-client.", f)
}
// RegisterFlagsWithPrefix registers flags for the Bloom Gateway client configuration with a common prefix.
func (i *ClientConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
i.GRPCClientConfig.RegisterFlagsWithPrefix(prefix+"grpc", f)
f.BoolVar(&i.LogGatewayRequests, prefix+"log-gateway-requests", false, "Flag to control whether requests sent to the gateway should be logged or not.")
}
type Client interface {
FilterChunks(ctx context.Context, tenant string, from, through model.Time, fingerprints []uint64, chunkRefs [][]*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]uint64, [][]*logproto.ChunkRef, error)
}
type GatewayClient struct {
cfg ClientConfig
limits Limits
logger log.Logger
pool *ringclient.Pool
ring ring.ReadRing
}
func NewGatewayClient(cfg ClientConfig, limits Limits, registerer prometheus.Registerer, logger log.Logger) (*GatewayClient, error) {
latency := promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{
Namespace: "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))
if err != nil {
return nil, err
}
poolFactory := func(addr string) (ringclient.PoolClient, error) {
pool, err := NewBloomGatewayGRPCPool(addr, dialOpts)
if err != nil {
return nil, errors.Wrap(err, "new bloom gateway grpc pool")
}
return pool, nil
}
c := &GatewayClient{
cfg: cfg,
logger: logger,
limits: limits,
pool: clientpool.NewPool("bloom-gateway", cfg.PoolConfig, cfg.Ring, ringclient.PoolAddrFunc(poolFactory), logger),
}
return c, nil
}
func shuffleAddrs(addrs []string) []string {
rand.Shuffle(len(addrs), func(i, j int) {
addrs[i], addrs[j] = addrs[j], addrs[i]
})
return addrs
}
// FilterChunkRefs implements Client
func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, through model.Time, fingerprints []uint64, chunkRefs [][]*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]uint64, [][]*logproto.ChunkRef, error) {
// Get the addresses of corresponding bloom gateways for each series.
_, addrs, err := c.serverAddrsForFingerprints(tenant, fingerprints)
if err != nil {
return nil, nil, err
}
// Group chunk refs by addresses of one or more bloom gateways.
// All chunk refs of series that belong to one and the same bloom gateway are set in one batch.
streamsByAddr := c.groupStreamsByAddr(fingerprints, chunkRefs, addrs)
// TODO(chaudum): We might over-allocate for the filtered responses here?
filteredChunkRefs := make([][]*logproto.ChunkRef, 0, len(fingerprints))
filteredFingerprints := make([]uint64, 0, len(fingerprints))
for _, item := range streamsByAddr {
// randomize order of addresses so we don't hotspot the first server in the list
addrs := shuffleAddrs(item.addrs)
err := c.doForAddrs(addrs, func(client logproto.BloomGatewayClient) error {
req := &logproto.FilterChunkRefRequest{
From: from,
Through: through,
Refs: item.refs,
Filters: filters,
}
resp, err := client.FilterChunkRefs(ctx, req)
if err != nil {
return err
}
for _, refGroup := range resp.ChunkRefs {
chunkRefs := make([]*logproto.ChunkRef, 0, len(refGroup.Refs))
for _, shortRef := range refGroup.Refs {
chunkRefs = append(chunkRefs,
&logproto.ChunkRef{
Fingerprint: refGroup.Fingerprint,
UserID: refGroup.Tenant,
From: shortRef.From,
Through: shortRef.Through,
Checksum: shortRef.Checksum,
},
)
}
filteredFingerprints = append(filteredFingerprints, refGroup.Fingerprint)
filteredChunkRefs = append(filteredChunkRefs, chunkRefs)
}
return nil
})
if err != nil {
return nil, nil, err
}
}
return fingerprints, filteredChunkRefs, nil
}
// isEqualStringElements checks if two string slices contain the same elements.
// The order of the elements is ignored.
func isEqualStringElements(a, b []string) bool {
if len(a) != len(b) {
return false
}
for _, s := range a {
if !util.StringsContain(b, s) {
return false
}
}
return true
}
// listContainsAddrs checks if a slice of chunkRefAddrs contains an element
// whos field addrs contains the same addresses as the given slice of
// addresses.
// It returns the index of the element, if found, and a boolean whether the
// given list contains the given addrs.
func listContainsAddrs(list []chunkRefsByAddrs, addrs []string) (int, bool) {
for i, r := range list {
if isEqualStringElements(r.addrs, addrs) {
return i, true
}
}
return -1, false
}
type chunkRefsByAddrs struct {
addrs []string
refs []*logproto.ChunkRef
streams []uint64
}
// groupStreamsByAddr takes a slice of stream fingerprints, a slices of chunkRef slices, and a slice of address slices
// and groups them into a slice of chunkRefsByAddrs.
// streams is a slice of uint64 stream fingerprints
// chunks is a slice of chunk ref slices
// addresses is a slice of string slices containing server addresses
// It is necessary that len(streams) == len(chunks) == len(addresses), but the
// function implementation does not validate the precondition and would fail silently.
func (c *GatewayClient) groupStreamsByAddr(streams []uint64, chunks [][]*logproto.ChunkRef, addresses [][]string) []chunkRefsByAddrs {
res := make([]chunkRefsByAddrs, 0, len(addresses))
for i := 0; i < len(addresses); i++ {
addrs := addresses[i]
refs := chunks[i]
fp := streams[i]
if idx, ok := listContainsAddrs(res, addrs); ok {
res[idx].refs = append(res[idx].refs, refs...)
res[idx].streams = append(res[idx].streams, fp)
} else {
res = append(res, chunkRefsByAddrs{addrs: addrs, refs: refs, streams: []uint64{fp}})
}
}
return res
}
// doForAddrs sequetially calls the provided callback function fn for each
// address in given slice addrs until the callback function does not return an
// error.
func (c *GatewayClient) doForAddrs(addrs []string, fn func(logproto.BloomGatewayClient) error) error {
var err error
var poolClient ringclient.PoolClient
for _, addr := range addrs {
poolClient, err = c.pool.GetClientFor(addr)
if err != nil {
level.Error(c.logger).Log("msg", fmt.Sprintf("failed to get client for instance %s", addr), "err", err)
continue
}
err = fn(poolClient.(logproto.BloomGatewayClient))
if err != nil {
level.Error(c.logger).Log("msg", fmt.Sprintf("client do failed for instance %s", addr), "err", err)
continue
}
return nil
}
return err
}
// serverAddrsForFingerprints returns a slices of server address slices for
// each fingerprint of given fingerprints.
// The indexes of the returned slices correspond to each other.
// Returns an error in case the bloom gateway ring could not get the
// corresponding replica set for a given fingerprint.
// Warning: This function becomes inefficient when the number of fingerprints is very large.
func (c *GatewayClient) serverAddrsForFingerprints(tenantID string, fingerprints []uint64) ([]uint64, [][]string, error) {
subRing := GetShuffleShardingSubring(c.ring, tenantID, c.limits)
rs, err := subRing.GetAllHealthy(BlocksRead)
if err != nil {
return nil, nil, errors.Wrap(err, "bloom gateway get healthy instances")
}
var numTokens int
for _, instanceDesc := range rs.Instances {
numTokens += len(instanceDesc.Tokens)
}
numFingerprints := len(fingerprints)
if numFingerprints > int(float64(numTokens)*math.Log2(float64(numFingerprints))) {
// TODO(chaudum): Implement algorithm in O(n * m * log(k) + n) instead of O(k) by iterating over ring tokens
// and finding corresponding fingerprint ranges using binary search.
// n .. number of instances
// m .. number of tokens per instance
// k .. number of fingerprints
level.Warn(c.logger).Log("msg", "using an inefficient algorithm to determin server addresses for fingerprints", "fingerprints", numFingerprints, "tokens", numTokens)
}
addresses := make([][]string, numFingerprints)
bufDescs, bufHosts, bufZones := ring.MakeBuffersForGet()
for idx, key := range fingerprints {
rs, err = subRing.Get(uint32(key), BlocksRead, bufDescs, bufHosts, bufZones)
if err != nil {
return nil, nil, errors.Wrap(err, "bloom gateway get ring")
}
addresses[idx] = rs.GetAddresses()
}
return fingerprints, addresses, nil
}

@ -0,0 +1,121 @@
package bloomgateway
import (
"testing"
"github.com/go-kit/log"
"github.com/grafana/dskit/flagext"
"github.com/prometheus/client_golang/prometheus"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/validation"
)
func TestBloomGatewayClient(t *testing.T) {
logger := log.NewNopLogger()
reg := prometheus.NewRegistry()
l, err := validation.NewOverrides(validation.Limits{BloomGatewayShardSize: 1}, nil)
require.NoError(t, err)
cfg := ClientConfig{}
flagext.DefaultValues(&cfg)
t.Run("", func(t *testing.T) {
_, err := NewGatewayClient(cfg, l, reg, logger)
require.NoError(t, err)
})
}
func TestBloomGatewayClient_GroupStreamsByAddresses(t *testing.T) {
logger := log.NewNopLogger()
reg := prometheus.NewRegistry()
l, err := validation.NewOverrides(validation.Limits{BloomGatewayShardSize: 1}, nil)
require.NoError(t, err)
cfg := ClientConfig{}
flagext.DefaultValues(&cfg)
c, err := NewGatewayClient(cfg, l, reg, logger)
require.NoError(t, err)
testCases := []struct {
name string
streams []uint64
chunks [][]*logproto.ChunkRef
addresses [][]string
expected []chunkRefsByAddrs
}{
{
name: "empty input yields empty result",
streams: []uint64{},
chunks: [][]*logproto.ChunkRef{},
addresses: [][]string{},
expected: []chunkRefsByAddrs{},
},
{
name: "addresses with same elements are grouped into single item",
streams: []uint64{1, 2, 3},
chunks: [][]*logproto.ChunkRef{
{{Fingerprint: 1, Checksum: 1}},
{{Fingerprint: 2, Checksum: 2}},
{{Fingerprint: 3, Checksum: 3}},
},
addresses: [][]string{
{"10.0.0.1", "10.0.0.2", "10.0.0.3"},
{"10.0.0.2", "10.0.0.3", "10.0.0.1"},
{"10.0.0.3", "10.0.0.1", "10.0.0.2"},
},
expected: []chunkRefsByAddrs{
{
addrs: []string{"10.0.0.1", "10.0.0.2", "10.0.0.3"},
refs: []*logproto.ChunkRef{
{Fingerprint: 1, Checksum: 1},
{Fingerprint: 2, Checksum: 2},
{Fingerprint: 3, Checksum: 3},
},
streams: []uint64{1, 2, 3},
},
},
},
{
name: "partially overlapping addresses are not grouped together",
streams: []uint64{1, 2},
chunks: [][]*logproto.ChunkRef{
{{Fingerprint: 1, Checksum: 1}},
{{Fingerprint: 2, Checksum: 2}},
},
addresses: [][]string{
{"10.0.0.1", "10.0.0.2"},
{"10.0.0.2", "10.0.0.3"},
},
expected: []chunkRefsByAddrs{
{
addrs: []string{"10.0.0.1", "10.0.0.2"},
refs: []*logproto.ChunkRef{
{Fingerprint: 1, Checksum: 1},
},
streams: []uint64{1},
},
{
addrs: []string{"10.0.0.2", "10.0.0.3"},
refs: []*logproto.ChunkRef{
{Fingerprint: 2, Checksum: 2},
},
streams: []uint64{2},
},
},
},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
res := c.groupStreamsByAddr(tc.streams, tc.chunks, tc.addresses)
require.Equal(t, tc.expected, res)
})
}
}

@ -0,0 +1,47 @@
package bloomgateway
import (
"flag"
"github.com/grafana/loki/pkg/util"
)
// RingCfg is a wrapper for our internally used ring configuration plus the replication factor.
type RingCfg struct {
// RingConfig configures the Bloom Gateway ring.
util.RingConfig `yaml:",inline"`
// ReplicationFactor defines how many replicas of the Bloom Gateway store a single data shard.
ReplicationFactor int `yaml:"replication_factor"`
}
// RegisterFlagsWithPrefix registers all Bloom Gateway CLI flags.
func (cfg *RingCfg) RegisterFlagsWithPrefix(prefix, storePrefix string, f *flag.FlagSet) {
cfg.RingConfig.RegisterFlagsWithPrefix(prefix, storePrefix, f)
f.IntVar(&cfg.ReplicationFactor, prefix+"replication-factor", 3, "Factor for data replication on the bloom gateways.")
}
// Config configures the Bloom Gateway component.
type Config struct {
// Ring configures the ring store used to save and retrieve the different Bloom Gateway instances.
// In case it isn't explicitly set, it follows the same behavior of the other rings (ex: using the common configuration
// section and the ingester configuration by default).
Ring RingCfg `yaml:"ring,omitempty" doc:"description=Defines the ring to be used by the bloom gateway servers and clients. In case this isn't configured, this block supports inheriting configuration from the common ring section."`
// Enabled is the global switch to configures whether Bloom Gateways should be used to filter chunks.
Enabled bool `yaml:"enabled"`
// Client configures the Bloom Gateway client
Client ClientConfig `yaml:"client,omitempty" doc:""`
}
// RegisterFlags registers flags for the Bloom Gateway configuration.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.RegisterFlagsWithPrefix("bloom-gateway.", f)
}
// RegisterFlagsWithPrefix registers flags for the Bloom Gateway configuration with a common prefix.
func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
cfg.Ring.RegisterFlagsWithPrefix(prefix, "collectors/", f)
f.BoolVar(&cfg.Enabled, prefix+"enabled", false, "Flag to enable or disable the usage of the bloom gatway component.")
// TODO(chaudum): Figure out what the better place is for registering flags
// -bloom-gateway.client.* or -bloom-gateway-client.*
cfg.Client.RegisterFlags(f)
}

@ -0,0 +1,64 @@
package bloomgateway
import (
"context"
"sort"
"github.com/go-kit/log"
"github.com/prometheus/common/model"
"github.com/grafana/loki/pkg/logproto"
)
// BloomQuerier is a store-level abstraction on top of Client
// It is used by the index gateway to filter ChunkRefs based on given line fiter expression.
type BloomQuerier struct {
c Client
logger log.Logger
}
func NewBloomQuerier(c Client, logger log.Logger) *BloomQuerier {
return &BloomQuerier{c: c, logger: logger}
}
func (bq *BloomQuerier) FilterChunkRefs(ctx context.Context, tenant string, from, through model.Time, chunkRefs []*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]*logproto.ChunkRef, error) {
// Shortcut that does not require any filtering
if len(chunkRefs) == 0 || len(filters) == 0 {
return chunkRefs, nil
}
// TODO(chaudum): Make buffer pool to reduce allocations.
// The indexes of the chunks slice correspond to the indexes of the fingerprint slice.
fingerprints := make([]uint64, 0, len(chunkRefs))
chunks := make([][]*logproto.ChunkRef, 0, len(chunkRefs))
// Sort the chunkRefs by their stream fingerprint
// so we can easily append them to the target slice by iterating over them.
sort.Slice(chunkRefs, func(i, j int) bool {
return chunkRefs[i].Fingerprint < chunkRefs[j].Fingerprint
})
for _, chunkRef := range chunkRefs {
idx := len(fingerprints) - 1
if idx == -1 || fingerprints[idx] < chunkRef.Fingerprint {
fingerprints = append(fingerprints, chunkRef.Fingerprint)
chunks = append(chunks, []*logproto.ChunkRef{chunkRef})
continue
}
chunks[idx] = append(chunks[idx], chunkRef)
}
// Drop series fingerprints, because they are not used (yet).
_, refs, err := bq.c.FilterChunks(ctx, tenant, from, through, fingerprints, chunks, filters...)
if err != nil {
return nil, err
}
// TODO(chaudum): Cache response
// Flatten response from client and return
result := make([]*logproto.ChunkRef, 0, len(chunkRefs))
for i := range refs {
result = append(result, refs[i]...)
}
return result, nil
}

@ -0,0 +1,86 @@
package bloomgateway
import (
"context"
"testing"
"time"
"github.com/go-kit/log"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logproto"
)
type noopClient struct {
err error // error to return
callCount int
}
// FilterChunks implements Client.
func (c *noopClient) FilterChunks(ctx context.Context, tenant string, from model.Time, through model.Time, fingerprints []uint64, chunkRefs [][]*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]uint64, [][]*logproto.ChunkRef, error) {
c.callCount++
return fingerprints, chunkRefs, c.err
}
func TestBloomQuerier(t *testing.T) {
logger := log.NewNopLogger()
tenant := "fake"
t.Run("client not called when filters are empty", func(t *testing.T) {
c := &noopClient{}
bq := NewBloomQuerier(c, logger)
ctx := context.Background()
through := model.Now()
from := through.Add(-12 * time.Hour)
chunkRefs := []*logproto.ChunkRef{
{Fingerprint: 3000, UserID: tenant, Checksum: 1},
{Fingerprint: 1000, UserID: tenant, Checksum: 2},
{Fingerprint: 2000, UserID: tenant, Checksum: 3},
}
filters := []*logproto.LineFilterExpression{}
res, err := bq.FilterChunkRefs(ctx, tenant, from, through, chunkRefs, filters...)
require.NoError(t, err)
require.Equal(t, chunkRefs, res)
require.Equal(t, 0, c.callCount)
})
t.Run("client not called when chunkRefs are empty", func(t *testing.T) {
c := &noopClient{}
bq := NewBloomQuerier(c, logger)
ctx := context.Background()
through := model.Now()
from := through.Add(-12 * time.Hour)
chunkRefs := []*logproto.ChunkRef{}
filters := []*logproto.LineFilterExpression{
{Operator: 0, Match: "uuid"},
}
res, err := bq.FilterChunkRefs(ctx, tenant, from, through, chunkRefs, filters...)
require.NoError(t, err)
require.Equal(t, chunkRefs, res)
require.Equal(t, 0, c.callCount)
})
t.Run("querier propagates error from client", func(t *testing.T) {
c := &noopClient{err: errors.New("something went wrong")}
bq := NewBloomQuerier(c, logger)
ctx := context.Background()
through := model.Now()
from := through.Add(-12 * time.Hour)
chunkRefs := []*logproto.ChunkRef{
{Fingerprint: 3000, UserID: tenant, Checksum: 1},
{Fingerprint: 1000, UserID: tenant, Checksum: 2},
{Fingerprint: 2000, UserID: tenant, Checksum: 3},
}
filters := []*logproto.LineFilterExpression{
{Operator: 0, Match: "uuid"},
}
res, err := bq.FilterChunkRefs(ctx, tenant, from, through, chunkRefs, filters...)
require.Error(t, err)
require.Nil(t, res)
})
}

@ -0,0 +1,264 @@
package bloomgateway
import (
"context"
"fmt"
"net/http"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/kv"
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/services"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
)
const (
ringAutoForgetUnhealthyPeriods = 10
ringNameForServer = "bloom-gateway"
ringNumTokens = 128
ringCheckPeriod = 3 * time.Second
// RingIdentifier is used as a unique name to register the Bloom Gateway ring.
RingIdentifier = "bloom-gateway"
// RingKey is the name of the key used to register the different Bloom Gateway instances in the key-value store.
RingKey = "bloom-gateway"
)
// ManagerMode defines the different modes for the RingManager to execute.
//
// The RingManager and its modes are only relevant if the Bloom Gateway is running in ring mode.
type ManagerMode int
const (
// ClientMode is the RingManager mode executed by Loki components that are clients of the Bloom Gateway.
// The RingManager in client will have its own ring key-value store but it won't try to register itself in the ring.
ClientMode ManagerMode = iota
// ServerMode is the RingManager mode execute by the Bloom Gateway.
// The RingManager in server mode will register itself in the ring.
ServerMode
)
// RingManager is a component instantiated before all the others and is responsible for the ring setup.
//
// All Loki components that are involved with the Bloom Gateway (including the Bloom Gateway itself) will
// require a RingManager. However, the components that are clients of the Bloom Gateway will ran it in client
// mode while the Bloom Gateway itself will ran the manager in server mode.
type RingManager struct {
services.Service
cfg Config
logger log.Logger
subservices *services.Manager
subservicesWatcher *services.FailureWatcher
RingLifecycler *ring.BasicLifecycler
Ring *ring.Ring
Mode ManagerMode
}
// NewRingManager instantiates a new RingManager instance.
// The other functions will assume the RingManager was instantiated through this function.
func NewRingManager(mode ManagerMode, cfg Config, logger log.Logger, registerer prometheus.Registerer) (*RingManager, error) {
rm := &RingManager{
cfg: cfg, logger: logger, Mode: mode,
}
// instantiate kv store for both modes.
ringStore, err := kv.NewClient(
rm.cfg.Ring.KVStore,
ring.GetCodec(),
kv.RegistererWithKVName(prometheus.WrapRegistererWithPrefix("loki_", registerer), "bloom-gateway-ring-manager"),
rm.logger,
)
if err != nil {
return nil, errors.Wrap(err, "bloom gateway ring manager create KV store client")
}
// instantiate ring for both mode modes.
ringCfg := rm.cfg.Ring.ToRingConfig(rm.cfg.Ring.ReplicationFactor)
rm.Ring, err = ring.NewWithStoreClientAndStrategy(
ringCfg,
ringNameForServer,
RingKey,
ringStore,
ring.NewIgnoreUnhealthyInstancesReplicationStrategy(),
prometheus.WrapRegistererWithPrefix("loki_", registerer),
rm.logger,
)
if err != nil {
return nil, errors.Wrap(err, "bloom gateway ring manager create ring client")
}
switch mode {
case ServerMode:
if err := rm.startServerMode(ringStore, registerer); err != nil {
return nil, err
}
case ClientMode:
if err := rm.startClientMode(); err != nil {
return nil, err
}
default:
return nil, fmt.Errorf("starting bloom gateway in unsupported mode %v", mode)
}
return rm, nil
}
func (rm *RingManager) startServerMode(ringStore kv.Client, registerer prometheus.Registerer) error {
lifecyclerCfg, err := rm.cfg.Ring.ToLifecyclerConfig(ringNumTokens, rm.logger)
if err != nil {
return errors.Wrap(err, "invalid ring lifecycler config")
}
delegate := ring.BasicLifecyclerDelegate(rm)
delegate = ring.NewLeaveOnStoppingDelegate(delegate, rm.logger)
delegate = ring.NewTokensPersistencyDelegate(rm.cfg.Ring.TokensFilePath, ring.JOINING, delegate, rm.logger)
delegate = ring.NewAutoForgetDelegate(ringAutoForgetUnhealthyPeriods*rm.cfg.Ring.HeartbeatTimeout, delegate, rm.logger)
rm.RingLifecycler, err = ring.NewBasicLifecycler(lifecyclerCfg, ringNameForServer, RingKey, ringStore, delegate, rm.logger, registerer)
if err != nil {
return errors.Wrap(err, "bloom gateway ring manager create ring lifecycler")
}
svcs := []services.Service{rm.RingLifecycler, rm.Ring}
rm.subservices, err = services.NewManager(svcs...)
if err != nil {
return errors.Wrap(err, "new bloom gateway services manager in server mode")
}
rm.subservicesWatcher = services.NewFailureWatcher()
rm.subservicesWatcher.WatchManager(rm.subservices)
rm.Service = services.NewBasicService(rm.starting, rm.running, rm.stopping)
return nil
}
func (rm *RingManager) startClientMode() error {
var err error
svcs := []services.Service{rm.Ring}
rm.subservices, err = services.NewManager(svcs...)
if err != nil {
return errors.Wrap(err, "new bloom gateway services manager in client mode")
}
rm.subservicesWatcher = services.NewFailureWatcher()
rm.subservicesWatcher.WatchManager(rm.subservices)
rm.Service = services.NewIdleService(func(ctx context.Context) error {
return services.StartManagerAndAwaitHealthy(ctx, rm.subservices)
}, func(failureCase error) error {
return services.StopManagerAndAwaitStopped(context.Background(), rm.subservices)
})
return nil
}
// starting implements the Lifecycler interface and is one of the lifecycle hooks.
func (rm *RingManager) starting(ctx context.Context) (err error) {
// In case this function will return error we want to unregister the instance
// from the ring. We do it ensuring dependencies are gracefully stopped if they
// were already started.
defer func() {
if err == nil || rm.subservices == nil {
return
}
if stopErr := services.StopManagerAndAwaitStopped(context.Background(), rm.subservices); stopErr != nil {
level.Error(rm.logger).Log("msg", "failed to gracefully stop bloom gateway ring manager dependencies", "err", stopErr)
}
}()
if err := services.StartManagerAndAwaitHealthy(ctx, rm.subservices); err != nil {
return errors.Wrap(err, "unable to start bloom gateway ring manager subservices")
}
// The BasicLifecycler does not automatically move state to ACTIVE such that any additional work that
// someone wants to do can be done before becoming ACTIVE. For the bloom gateway we don't currently
// have any additional work so we can become ACTIVE right away.
// Wait until the ring client detected this instance in the JOINING
// state to make sure that when we'll run the initial sync we already
// know the tokens assigned to this instance.
level.Info(rm.logger).Log("msg", "waiting until bloom gateway is JOINING in the ring")
if err := ring.WaitInstanceState(ctx, rm.Ring, rm.RingLifecycler.GetInstanceID(), ring.JOINING); err != nil {
return err
}
level.Info(rm.logger).Log("msg", "bloom gateway is JOINING in the ring")
if err = rm.RingLifecycler.ChangeState(ctx, ring.ACTIVE); err != nil {
return errors.Wrapf(err, "switch instance to %s in the ring", ring.ACTIVE)
}
// Wait until the ring client detected this instance in the ACTIVE state to
// make sure that when we'll run the loop it won't be detected as a ring
// topology change.
level.Info(rm.logger).Log("msg", "waiting until bloom gateway is ACTIVE in the ring")
if err := ring.WaitInstanceState(ctx, rm.Ring, rm.RingLifecycler.GetInstanceID(), ring.ACTIVE); err != nil {
return err
}
level.Info(rm.logger).Log("msg", "bloom gateway is ACTIVE in the ring")
return nil
}
// running implements the Lifecycler interface and is one of the lifecycle hooks.
func (rm *RingManager) running(ctx context.Context) error {
t := time.NewTicker(ringCheckPeriod)
defer t.Stop()
for {
select {
case <-ctx.Done():
return nil
case err := <-rm.subservicesWatcher.Chan():
return errors.Wrap(err, "running bloom gateway ring manager subservice failed")
case <-t.C:
continue
}
}
}
// stopping implements the Lifecycler interface and is one of the lifecycle hooks.
func (rm *RingManager) stopping(_ error) error {
level.Debug(rm.logger).Log("msg", "stopping bloom gateway ring manager")
return services.StopManagerAndAwaitStopped(context.Background(), rm.subservices)
}
// ServeHTTP serves the HTTP route /bloomgateway/ring.
func (rm *RingManager) ServeHTTP(w http.ResponseWriter, req *http.Request) {
rm.Ring.ServeHTTP(w, req)
}
func (rm *RingManager) OnRingInstanceRegister(_ *ring.BasicLifecycler, ringDesc ring.Desc, instanceExists bool, _ string, instanceDesc ring.InstanceDesc) (ring.InstanceState, ring.Tokens) {
// When we initialize the index gateway instance in the ring we want to start from
// a clean situation, so whatever is the state we set it JOINING, while we keep existing
// tokens (if any) or the ones loaded from file.
var tokens []uint32
if instanceExists {
tokens = instanceDesc.GetTokens()
}
takenTokens := ringDesc.GetTokens()
gen := ring.NewRandomTokenGenerator()
newTokens := gen.GenerateTokens(ringNumTokens-len(tokens), takenTokens)
// Tokens sorting will be enforced by the parent caller.
tokens = append(tokens, newTokens...)
return ring.JOINING, tokens
}
func (rm *RingManager) OnRingInstanceTokens(_ *ring.BasicLifecycler, _ ring.Tokens) {
}
func (rm *RingManager) OnRingInstanceStopping(_ *ring.BasicLifecycler) {
}
func (rm *RingManager) OnRingInstanceHeartbeat(_ *ring.BasicLifecycler, _ *ring.Desc, _ *ring.InstanceDesc) {
}

@ -0,0 +1,162 @@
package bloomgateway
import (
"context"
"github.com/go-kit/log"
"github.com/grafana/dskit/ring"
)
// TODO(chaudum): Replace this placeholder with actual BlockRef struct.
type BlockRef struct {
FromFp, ThroughFp uint64
FromTs, ThroughTs int64
}
var (
// BlocksOwnerSync is the operation used to check the authoritative owners of a block
// (replicas included).
BlocksOwnerSync = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE, ring.LEAVING}, nil)
// 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)
// BlocksRead is the operation run by the querier to query blocks via the bloom gateway.
BlocksRead = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, func(s ring.InstanceState) bool {
// Blocks can only be queried from ACTIVE instances. However, if the block belongs to
// a non-active instance, then we should extend the replication set and try to query it
// from the next ACTIVE instance in the ring (which is expected to have it because a
// bloom gateway keeps their previously owned blocks until new owners are ACTIVE).
return s != ring.ACTIVE
})
)
type Limits interface {
BloomGatewayShardSize(tenantID string) int
}
type ShardingStrategy interface {
// FilterTenants whose indexes should be loaded by the index gateway.
// Returns the list of user IDs that should be synced by the index gateway.
FilterTenants(ctx context.Context, tenantIDs []string) ([]string, error)
FilterBlocks(ctx context.Context, tenantID string, blockRefs []BlockRef) ([]BlockRef, error)
}
type ShuffleShardingStrategy struct {
r ring.ReadRing
limits Limits
instanceAddr string
instanceID string
logger log.Logger
}
func NewShuffleShardingStrategy(r ring.ReadRing, l Limits, instanceAddr, instanceID string, logger log.Logger) *ShuffleShardingStrategy {
return &ShuffleShardingStrategy{
r: r,
limits: l,
instanceAddr: instanceAddr,
instanceID: instanceID,
logger: logger,
}
}
// FilterTenants implements ShardingStrategy.
func (s *ShuffleShardingStrategy) FilterTenants(_ context.Context, tenantIDs []string) ([]string, error) {
// As a protection, ensure the bloom gateway instance is healthy in the ring. It could also be missing
// in the ring if it was failing to heartbeat the ring and it got remove from another healthy bloom gateway
// instance, because of the auto-forget feature.
if set, err := s.r.GetAllHealthy(BlocksOwnerSync); err != nil {
return nil, err
} else if !set.Includes(s.instanceAddr) {
return nil, errGatewayUnhealthy
}
var filteredIDs []string
for _, tenantID := range tenantIDs {
subRing := GetShuffleShardingSubring(s.r, tenantID, s.limits)
// Include the user only if it belongs to this bloom gateway shard.
if subRing.HasInstance(s.instanceID) {
filteredIDs = append(filteredIDs, tenantID)
}
}
return filteredIDs, nil
}
func getBucket(rangeMin, rangeMax, pos uint64) int {
return 0
}
// FilterBlocks implements ShardingStrategy.
func (s *ShuffleShardingStrategy) FilterBlocks(ctx context.Context, tenantID string, blockRefs []BlockRef) ([]BlockRef, error) {
filteredBlockRefs := make([]BlockRef, 0, len(blockRefs))
subRing := GetShuffleShardingSubring(s.r, tenantID, s.limits)
bufDescs, bufHosts, bufZones := ring.MakeBuffersForGet()
var rs ring.ReplicationSet
var err error
for _, blockRef := range blockRefs {
rs, err = subRing.Get(uint32(blockRef.FromFp), BlocksOwnerSync, bufDescs, bufHosts, bufZones)
if err != nil {
return nil, err
}
// Include the block only if it belongs to this bloom gateway shard.
if rs.Includes(s.instanceID) {
filteredBlockRefs = append(filteredBlockRefs, blockRef)
continue
}
rs, err = subRing.Get(uint32(blockRef.ThroughFp), BlocksOwnerSync, bufDescs, bufHosts, bufZones)
if err != nil {
return nil, err
}
// Include the block only if it belongs to this bloom gateway shard.
if rs.Includes(s.instanceID) {
filteredBlockRefs = append(filteredBlockRefs, blockRef)
continue
}
}
return filteredBlockRefs, nil
}
// 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)
}
// NoopStrategy is an implementation of the ShardingStrategy that does not
// filter anything.
type NoopStrategy struct{}
func NewNoopStrategy() *NoopStrategy {
return &NoopStrategy{}
}
// FilterTenants implements ShardingStrategy.
func (s *NoopStrategy) FilterTenants(ctx context.Context, tenantIDs []string) ([]string, error) {
return tenantIDs, nil
}
// FilterBlocks implements ShardingStrategy.
func (s *NoopStrategy) FilterBlocks(ctx context.Context, tenantID string, blockRefs []BlockRef) ([]BlockRef, error) {
return blockRefs, nil
}

@ -25,17 +25,19 @@ type PoolConfig struct {
}
// RegisterFlags adds the flags required to config this to the given FlagSet.
func (cfg *PoolConfig) RegisterFlags(f *flag.FlagSet) {
f.DurationVar(&cfg.ClientCleanupPeriod, "distributor.client-cleanup-period", 15*time.Second, "How frequently to clean up clients for ingesters that have gone away.")
f.BoolVar(&cfg.HealthCheckIngesters, "distributor.health-check-ingesters", true, "Run a health check on each ingester client during periodic cleanup.")
func (cfg *PoolConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
f.DurationVar(&cfg.ClientCleanupPeriod, prefix+"client-cleanup-period", 15*time.Second, "How frequently to clean up clients for ingesters that have gone away.")
f.BoolVar(&cfg.HealthCheckIngesters, prefix+"health-check-ingesters", true, "Run a health check on each ingester client during periodic cleanup.")
f.DurationVar(&cfg.RemoteTimeout, prefix+"remote-timeout", 1*time.Second, "Timeout for the health check.")
}
func NewPool(cfg PoolConfig, ring ring.ReadRing, factory ring_client.PoolFactory, logger log.Logger) *ring_client.Pool {
func NewPool(name string, cfg PoolConfig, ring ring.ReadRing, factory ring_client.PoolFactory, logger log.Logger) *ring_client.Pool {
poolCfg := ring_client.PoolConfig{
CheckInterval: cfg.ClientCleanupPeriod,
HealthCheckEnabled: cfg.HealthCheckIngesters,
HealthCheckTimeout: cfg.RemoteTimeout,
}
return ring_client.NewPool("ingester", poolCfg, ring_client.NewRingServiceDiscovery(ring), factory, clients, logger)
// TODO(chaudum): Allow cofiguration of metric name by the caller.
return ring_client.NewPool(name, poolCfg, ring_client.NewRingServiceDiscovery(ring), factory, clients, logger)
}

@ -171,7 +171,7 @@ func New(
tenantsRetention: retention.NewTenantsRetention(overrides),
ingestersRing: ingestersRing,
validator: validator,
pool: clientpool.NewPool(clientCfg.PoolConfig, ingestersRing, factory, util_log.Logger),
pool: clientpool.NewPool("ingester", clientCfg.PoolConfig, ingestersRing, factory, util_log.Logger),
labelCache: labelCache,
shardTracker: NewShardTracker(),
healthyInstancesCount: atomic.NewUint32(0),
@ -225,6 +225,7 @@ func New(
d.cfg.RateStore,
ingestersRing,
clientpool.NewPool(
"rate-store",
clientCfg.PoolConfig,
ingestersRing,
ring_client.PoolAddrFunc(internalFactory),

@ -54,7 +54,7 @@ type Config struct {
// RegisterFlags registers flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.GRPCClientConfig.RegisterFlagsWithPrefix("ingester.client", f)
cfg.PoolConfig.RegisterFlags(f)
cfg.PoolConfig.RegisterFlagsWithPrefix("distributor.", f)
f.DurationVar(&cfg.PoolConfig.RemoteTimeout, "ingester.client.healthcheck-timeout", 1*time.Second, "How quickly a dead client will be removed after it has been detected to disappear. Set this to a value to allow time for a secondary health check to recover the missing client.")
f.DurationVar(&cfg.RemoteTimeout, "ingester.client.timeout", 5*time.Second, "The remote request timeout on the client side.")

File diff suppressed because it is too large Load Diff

@ -0,0 +1,47 @@
syntax = "proto3";
package logproto;
import "gogoproto/gogo.proto";
import "pkg/logproto/logproto.proto";
option go_package = "github.com/grafana/loki/pkg/logproto";
message FilterChunkRefRequest {
int64 from = 1 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
(gogoproto.nullable) = false
];
int64 through = 2 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
(gogoproto.nullable) = false
];
repeated logproto.ChunkRef refs = 3;
repeated logproto.LineFilterExpression filters = 4;
}
message FilterChunkRefResponse {
repeated GroupedChunkRefs chunkRefs = 1;
}
message ShortRef {
int64 from = 1 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
(gogoproto.nullable) = false
];
int64 through = 2 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
(gogoproto.nullable) = false
];
uint32 checksum = 3;
}
message GroupedChunkRefs {
uint64 fingerprint = 1;
string tenant = 2;
repeated ShortRef refs = 3;
}
service BloomGateway {
rpc FilterChunkRefs(FilterChunkRefRequest) returns (FilterChunkRefResponse) {}
}

@ -1639,16 +1639,68 @@ func (m *LabelNamesForMetricNameRequest) GetMetricName() string {
return ""
}
type LineFilterExpression struct {
Operator int64 `protobuf:"varint,1,opt,name=operator,proto3" json:"operator,omitempty"`
Match string `protobuf:"bytes,2,opt,name=match,proto3" json:"match,omitempty"`
}
func (m *LineFilterExpression) Reset() { *m = LineFilterExpression{} }
func (*LineFilterExpression) ProtoMessage() {}
func (*LineFilterExpression) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{29}
}
func (m *LineFilterExpression) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LineFilterExpression) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LineFilterExpression.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *LineFilterExpression) XXX_Merge(src proto.Message) {
xxx_messageInfo_LineFilterExpression.Merge(m, src)
}
func (m *LineFilterExpression) XXX_Size() int {
return m.Size()
}
func (m *LineFilterExpression) XXX_DiscardUnknown() {
xxx_messageInfo_LineFilterExpression.DiscardUnknown(m)
}
var xxx_messageInfo_LineFilterExpression proto.InternalMessageInfo
func (m *LineFilterExpression) GetOperator() int64 {
if m != nil {
return m.Operator
}
return 0
}
func (m *LineFilterExpression) GetMatch() string {
if m != nil {
return m.Match
}
return ""
}
type GetChunkRefRequest struct {
From github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=from,proto3,customtype=github.com/prometheus/common/model.Time" json:"from"`
Through github_com_prometheus_common_model.Time `protobuf:"varint,2,opt,name=through,proto3,customtype=github.com/prometheus/common/model.Time" json:"through"`
Matchers string `protobuf:"bytes,3,opt,name=matchers,proto3" json:"matchers,omitempty"`
Filters []*LineFilterExpression `protobuf:"bytes,4,rep,name=filters,proto3" json:"filters,omitempty"`
}
func (m *GetChunkRefRequest) Reset() { *m = GetChunkRefRequest{} }
func (*GetChunkRefRequest) ProtoMessage() {}
func (*GetChunkRefRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{29}
return fileDescriptor_c28a5f14f1f4c79a, []int{30}
}
func (m *GetChunkRefRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1684,6 +1736,13 @@ func (m *GetChunkRefRequest) GetMatchers() string {
return ""
}
func (m *GetChunkRefRequest) GetFilters() []*LineFilterExpression {
if m != nil {
return m.Filters
}
return nil
}
type GetChunkRefResponse struct {
Refs []*ChunkRef `protobuf:"bytes,1,rep,name=refs,proto3" json:"refs,omitempty"`
}
@ -1691,7 +1750,7 @@ type GetChunkRefResponse struct {
func (m *GetChunkRefResponse) Reset() { *m = GetChunkRefResponse{} }
func (*GetChunkRefResponse) ProtoMessage() {}
func (*GetChunkRefResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{30}
return fileDescriptor_c28a5f14f1f4c79a, []int{31}
}
func (m *GetChunkRefResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1736,7 +1795,7 @@ type GetSeriesRequest struct {
func (m *GetSeriesRequest) Reset() { *m = GetSeriesRequest{} }
func (*GetSeriesRequest) ProtoMessage() {}
func (*GetSeriesRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{31}
return fileDescriptor_c28a5f14f1f4c79a, []int{32}
}
func (m *GetSeriesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1779,7 +1838,7 @@ type GetSeriesResponse struct {
func (m *GetSeriesResponse) Reset() { *m = GetSeriesResponse{} }
func (*GetSeriesResponse) ProtoMessage() {}
func (*GetSeriesResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{32}
return fileDescriptor_c28a5f14f1f4c79a, []int{33}
}
func (m *GetSeriesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1823,7 +1882,7 @@ type IndexSeries struct {
func (m *IndexSeries) Reset() { *m = IndexSeries{} }
func (*IndexSeries) ProtoMessage() {}
func (*IndexSeries) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{33}
return fileDescriptor_c28a5f14f1f4c79a, []int{34}
}
func (m *IndexSeries) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1860,7 +1919,7 @@ type QueryIndexResponse struct {
func (m *QueryIndexResponse) Reset() { *m = QueryIndexResponse{} }
func (*QueryIndexResponse) ProtoMessage() {}
func (*QueryIndexResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{34}
return fileDescriptor_c28a5f14f1f4c79a, []int{35}
}
func (m *QueryIndexResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1911,7 +1970,7 @@ type Row struct {
func (m *Row) Reset() { *m = Row{} }
func (*Row) ProtoMessage() {}
func (*Row) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{35}
return fileDescriptor_c28a5f14f1f4c79a, []int{36}
}
func (m *Row) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -1961,7 +2020,7 @@ type QueryIndexRequest struct {
func (m *QueryIndexRequest) Reset() { *m = QueryIndexRequest{} }
func (*QueryIndexRequest) ProtoMessage() {}
func (*QueryIndexRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{36}
return fileDescriptor_c28a5f14f1f4c79a, []int{37}
}
func (m *QueryIndexRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2008,7 +2067,7 @@ type IndexQuery struct {
func (m *IndexQuery) Reset() { *m = IndexQuery{} }
func (*IndexQuery) ProtoMessage() {}
func (*IndexQuery) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{37}
return fileDescriptor_c28a5f14f1f4c79a, []int{38}
}
func (m *IndexQuery) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2081,7 +2140,7 @@ type IndexStatsRequest struct {
func (m *IndexStatsRequest) Reset() { *m = IndexStatsRequest{} }
func (*IndexStatsRequest) ProtoMessage() {}
func (*IndexStatsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{38}
return fileDescriptor_c28a5f14f1f4c79a, []int{39}
}
func (m *IndexStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2127,7 +2186,7 @@ type IndexStatsResponse struct {
func (m *IndexStatsResponse) Reset() { *m = IndexStatsResponse{} }
func (*IndexStatsResponse) ProtoMessage() {}
func (*IndexStatsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{39}
return fileDescriptor_c28a5f14f1f4c79a, []int{40}
}
func (m *IndexStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2197,7 +2256,7 @@ type VolumeRequest struct {
func (m *VolumeRequest) Reset() { *m = VolumeRequest{} }
func (*VolumeRequest) ProtoMessage() {}
func (*VolumeRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{40}
return fileDescriptor_c28a5f14f1f4c79a, []int{41}
}
func (m *VolumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2269,7 +2328,7 @@ type VolumeResponse struct {
func (m *VolumeResponse) Reset() { *m = VolumeResponse{} }
func (*VolumeResponse) ProtoMessage() {}
func (*VolumeResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{41}
return fileDescriptor_c28a5f14f1f4c79a, []int{42}
}
func (m *VolumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2320,7 +2379,7 @@ type Volume struct {
func (m *Volume) Reset() { *m = Volume{} }
func (*Volume) ProtoMessage() {}
func (*Volume) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{42}
return fileDescriptor_c28a5f14f1f4c79a, []int{43}
}
func (m *Volume) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -2395,6 +2454,7 @@ func init() {
proto.RegisterType((*ChunkRef)(nil), "logproto.ChunkRef")
proto.RegisterType((*LabelValuesForMetricNameRequest)(nil), "logproto.LabelValuesForMetricNameRequest")
proto.RegisterType((*LabelNamesForMetricNameRequest)(nil), "logproto.LabelNamesForMetricNameRequest")
proto.RegisterType((*LineFilterExpression)(nil), "logproto.LineFilterExpression")
proto.RegisterType((*GetChunkRefRequest)(nil), "logproto.GetChunkRefRequest")
proto.RegisterType((*GetChunkRefResponse)(nil), "logproto.GetChunkRefResponse")
proto.RegisterType((*GetSeriesRequest)(nil), "logproto.GetSeriesRequest")
@ -2414,141 +2474,145 @@ func init() {
func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) }
var fileDescriptor_c28a5f14f1f4c79a = []byte{
// 2141 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0xcf, 0x8f, 0x5b, 0x47,
0xd9, 0x63, 0x3f, 0xff, 0xfa, 0x6c, 0x6f, 0x36, 0xb3, 0x6e, 0x62, 0xb9, 0x89, 0xbd, 0x19, 0x95,
0x74, 0x95, 0xa6, 0x76, 0x93, 0x42, 0x49, 0x13, 0x0a, 0xc4, 0xbb, 0x4d, 0xba, 0xf9, 0xcd, 0x6c,
0x08, 0xa8, 0x02, 0x45, 0x6f, 0xed, 0x59, 0xdb, 0x8a, 0x9f, 0x9f, 0xf3, 0xde, 0xb8, 0xe9, 0x4a,
0x1c, 0xf8, 0x07, 0x2a, 0xf5, 0x86, 0xb8, 0x20, 0x0e, 0x48, 0x20, 0x21, 0x2e, 0xfc, 0x01, 0xc0,
0x01, 0x89, 0x70, 0x0b, 0xb7, 0x8a, 0x83, 0x21, 0x9b, 0x0b, 0xda, 0x53, 0x6f, 0xdc, 0x10, 0x9a,
0x5f, 0xef, 0x8d, 0xbd, 0xde, 0xb6, 0x0e, 0x91, 0x50, 0x2e, 0x7e, 0x33, 0xdf, 0x7c, 0xf3, 0xcd,
0xf7, 0xfb, 0x9b, 0x6f, 0x0c, 0xaf, 0x8e, 0x1e, 0x74, 0x9b, 0x03, 0xbf, 0x3b, 0x0a, 0x7c, 0xee,
0x47, 0x83, 0x86, 0xfc, 0xc5, 0x39, 0x33, 0xaf, 0x96, 0xbb, 0x7e, 0xd7, 0x57, 0x38, 0x62, 0xa4,
0xd6, 0xab, 0xf5, 0xae, 0xef, 0x77, 0x07, 0xac, 0x29, 0x67, 0xdb, 0xe3, 0x9d, 0x26, 0xef, 0x7b,
0x2c, 0xe4, 0xae, 0x37, 0xd2, 0x08, 0xab, 0x9a, 0xfa, 0xc3, 0x81, 0xe7, 0x77, 0xd8, 0xa0, 0x19,
0x72, 0x97, 0x87, 0xea, 0x57, 0x63, 0xac, 0x08, 0x8c, 0xd1, 0x38, 0xec, 0xc9, 0x1f, 0x05, 0x24,
0x65, 0xc0, 0x5b, 0x3c, 0x60, 0xae, 0x47, 0x5d, 0xce, 0x42, 0xca, 0x1e, 0x8e, 0x59, 0xc8, 0xc9,
0x4d, 0x58, 0x99, 0x82, 0x86, 0x23, 0x7f, 0x18, 0x32, 0xfc, 0x0e, 0x14, 0xc2, 0x18, 0x5c, 0x41,
0xab, 0xa9, 0xb5, 0xc2, 0xf9, 0x72, 0x23, 0x12, 0x25, 0xde, 0x43, 0x6d, 0x44, 0xf2, 0x0b, 0x04,
0x10, 0xaf, 0xe1, 0x1a, 0x80, 0x5a, 0xfd, 0xc0, 0x0d, 0x7b, 0x15, 0xb4, 0x8a, 0xd6, 0x1c, 0x6a,
0x41, 0xf0, 0x59, 0x38, 0x1a, 0xcf, 0x6e, 0xf9, 0x5b, 0x3d, 0x37, 0xe8, 0x54, 0x92, 0x12, 0xed,
0xe0, 0x02, 0xc6, 0xe0, 0x04, 0x2e, 0x67, 0x95, 0xd4, 0x2a, 0x5a, 0x4b, 0x51, 0x39, 0xc6, 0xc7,
0x20, 0xc3, 0xd9, 0xd0, 0x1d, 0xf2, 0x8a, 0xb3, 0x8a, 0xd6, 0xf2, 0x54, 0xcf, 0x04, 0x5c, 0xc8,
0xce, 0xc2, 0x4a, 0x7a, 0x15, 0xad, 0x95, 0xa8, 0x9e, 0x91, 0xbf, 0x24, 0xa1, 0xf8, 0xbd, 0x31,
0x0b, 0x76, 0xb5, 0x02, 0x70, 0x15, 0x72, 0x21, 0x1b, 0xb0, 0x36, 0xf7, 0x03, 0xc9, 0x60, 0x9e,
0x46, 0x73, 0x5c, 0x86, 0xf4, 0xa0, 0xef, 0xf5, 0xb9, 0x64, 0xa9, 0x44, 0xd5, 0x04, 0x5f, 0x84,
0x74, 0xc8, 0xdd, 0x80, 0x4b, 0x3e, 0x0a, 0xe7, 0xab, 0x0d, 0x65, 0xb0, 0x86, 0x31, 0x58, 0xe3,
0xae, 0x31, 0x58, 0x2b, 0xf7, 0x78, 0x52, 0x4f, 0x7c, 0xfa, 0x8f, 0x3a, 0xa2, 0x6a, 0x0b, 0x7e,
0x07, 0x52, 0x6c, 0xd8, 0x91, 0xbc, 0x7e, 0xd5, 0x9d, 0x62, 0x03, 0x3e, 0x07, 0xf9, 0x4e, 0x3f,
0x60, 0x6d, 0xde, 0xf7, 0x87, 0x52, 0xa2, 0xa5, 0xf3, 0x2b, 0xb1, 0x35, 0x36, 0xcc, 0x12, 0x8d,
0xb1, 0xf0, 0x59, 0xc8, 0x84, 0x42, 0x6d, 0x61, 0x25, 0xbb, 0x9a, 0x5a, 0xcb, 0xb7, 0xca, 0xfb,
0x93, 0xfa, 0xb2, 0x82, 0x9c, 0xf5, 0xbd, 0x3e, 0x67, 0xde, 0x88, 0xef, 0x52, 0x8d, 0x83, 0xcf,
0x40, 0xb6, 0xc3, 0x06, 0x4c, 0x18, 0x3b, 0x27, 0x8d, 0xbd, 0x6c, 0x91, 0x97, 0x0b, 0xd4, 0x20,
0x5c, 0x73, 0x72, 0x99, 0xe5, 0x2c, 0xf9, 0x0f, 0x02, 0xbc, 0xe5, 0x7a, 0xa3, 0x01, 0xfb, 0xca,
0xfa, 0x8c, 0x34, 0x97, 0x7c, 0x6e, 0xcd, 0xa5, 0x16, 0xd5, 0x5c, 0xac, 0x06, 0x67, 0x31, 0x35,
0xa4, 0xbf, 0x44, 0x0d, 0xe4, 0x06, 0x64, 0x14, 0xe8, 0xcb, 0x7c, 0x28, 0x96, 0x39, 0x65, 0xa4,
0x59, 0x8e, 0xa5, 0x49, 0x49, 0x3e, 0xc9, 0x2f, 0x11, 0x94, 0xb4, 0x22, 0x75, 0x0c, 0x6e, 0x43,
0x56, 0xc5, 0x80, 0x89, 0xbf, 0xe3, 0xb3, 0xf1, 0x77, 0xb9, 0xe3, 0x8e, 0x38, 0x0b, 0x5a, 0xcd,
0xc7, 0x93, 0x3a, 0xfa, 0xfb, 0xa4, 0xfe, 0x7a, 0xb7, 0xcf, 0x7b, 0xe3, 0xed, 0x46, 0xdb, 0xf7,
0x9a, 0xdd, 0xc0, 0xdd, 0x71, 0x87, 0x6e, 0x73, 0xe0, 0x3f, 0xe8, 0x37, 0x4d, 0x3e, 0x30, 0x71,
0x6b, 0x08, 0xe3, 0x37, 0x24, 0x77, 0x3c, 0xd4, 0x16, 0x39, 0xd2, 0x50, 0x69, 0x64, 0x73, 0xd8,
0x65, 0xa1, 0xa0, 0xec, 0x08, 0x65, 0x52, 0x85, 0x43, 0x7e, 0x02, 0x2b, 0x53, 0x06, 0xd7, 0x7c,
0x5e, 0x80, 0x4c, 0xc8, 0x82, 0x7e, 0x94, 0x26, 0x2c, 0x95, 0x6d, 0x49, 0x78, 0x6b, 0x49, 0xf3,
0x97, 0x51, 0x73, 0xaa, 0xf1, 0x17, 0x3b, 0xfd, 0xcf, 0x08, 0x8a, 0x37, 0xdc, 0x6d, 0x36, 0x30,
0x9e, 0x86, 0xc1, 0x19, 0xba, 0x1e, 0xd3, 0x1a, 0x97, 0x63, 0x11, 0xf6, 0x1f, 0xb9, 0x83, 0x31,
0x53, 0x24, 0x73, 0x54, 0xcf, 0x16, 0x8d, 0x59, 0xf4, 0xdc, 0x31, 0x8b, 0x62, 0xcf, 0x2b, 0x43,
0xfa, 0xa1, 0x50, 0x94, 0x8c, 0xd7, 0x3c, 0x55, 0x13, 0xf2, 0x3a, 0x94, 0xb4, 0x14, 0x5a, 0x7d,
0x31, 0xcb, 0x42, 0x7d, 0x79, 0xc3, 0x32, 0xf1, 0x20, 0xa3, 0xb4, 0x8d, 0x5f, 0x83, 0x7c, 0x54,
0x03, 0xa4, 0xb4, 0xa9, 0x56, 0x66, 0x7f, 0x52, 0x4f, 0xf2, 0x90, 0xc6, 0x0b, 0xb8, 0x0e, 0x69,
0xb9, 0x53, 0x4a, 0x8e, 0x5a, 0xf9, 0xfd, 0x49, 0x5d, 0x01, 0xa8, 0xfa, 0xe0, 0x13, 0xe0, 0xf4,
0x44, 0x1a, 0x16, 0x2a, 0x70, 0x5a, 0xb9, 0xfd, 0x49, 0x5d, 0xce, 0xa9, 0xfc, 0x25, 0x57, 0xa1,
0x78, 0x83, 0x75, 0xdd, 0xf6, 0xae, 0x3e, 0xb4, 0x6c, 0xc8, 0x89, 0x03, 0x91, 0xa1, 0x71, 0x0a,
0x8a, 0xd1, 0x89, 0xf7, 0xbd, 0x50, 0x3b, 0x75, 0x21, 0x82, 0xdd, 0x0c, 0xc9, 0xcf, 0x11, 0x68,
0x3b, 0x63, 0x02, 0x99, 0x81, 0x90, 0x35, 0x54, 0x36, 0x6a, 0xc1, 0xfe, 0xa4, 0xae, 0x21, 0x54,
0x7f, 0xf1, 0x25, 0xc8, 0x86, 0xf2, 0x44, 0x41, 0x6c, 0xd6, 0x7d, 0xe4, 0x42, 0xeb, 0x88, 0x70,
0x83, 0xfd, 0x49, 0xdd, 0x20, 0x52, 0x33, 0xc0, 0x8d, 0xa9, 0xfa, 0xa2, 0x04, 0x5b, 0xda, 0x9f,
0xd4, 0x2d, 0xa8, 0x5d, 0x6f, 0xc8, 0xcf, 0x10, 0x14, 0xee, 0xba, 0xfd, 0xc8, 0x85, 0x22, 0x13,
0x21, 0xcb, 0x44, 0x22, 0x9c, 0x3b, 0x6c, 0xe0, 0xee, 0x5e, 0xf1, 0x03, 0x49, 0xb3, 0x44, 0xa3,
0x79, 0x5c, 0x12, 0x9c, 0xb9, 0x25, 0x21, 0xbd, 0x70, 0x62, 0xbb, 0xe6, 0xe4, 0x92, 0xcb, 0x29,
0xf2, 0x3b, 0x04, 0x45, 0xc5, 0x99, 0x76, 0x8b, 0x1f, 0x41, 0x46, 0x31, 0x2e, 0x79, 0xfb, 0x82,
0xe0, 0x7f, 0x63, 0x91, 0xc0, 0xd7, 0x34, 0xf1, 0x77, 0x60, 0xa9, 0x13, 0xf8, 0xa3, 0x11, 0xeb,
0x6c, 0xe9, 0x14, 0x93, 0x9c, 0x4d, 0x31, 0x1b, 0xf6, 0x3a, 0x9d, 0x41, 0x27, 0x7f, 0x45, 0x50,
0xd2, 0xd1, 0xac, 0x75, 0x19, 0xe9, 0x00, 0x3d, 0x77, 0x72, 0x4f, 0x2e, 0x9a, 0xdc, 0x8f, 0x41,
0xa6, 0x1b, 0xf8, 0xe3, 0x51, 0x58, 0x49, 0xa9, 0xd8, 0x51, 0xb3, 0xc5, 0x92, 0x3e, 0xb9, 0x06,
0x4b, 0x46, 0x94, 0x43, 0x52, 0x5a, 0x75, 0x36, 0xa5, 0x6d, 0x76, 0xd8, 0x90, 0xf7, 0x77, 0xfa,
0x51, 0x92, 0xd2, 0xf8, 0xe4, 0x13, 0x04, 0xcb, 0xb3, 0x28, 0xf8, 0xdb, 0x56, 0x1c, 0x08, 0x72,
0xa7, 0x0f, 0x27, 0xd7, 0x90, 0xc9, 0x21, 0x7c, 0x7f, 0xc8, 0x83, 0x5d, 0x13, 0x23, 0xd5, 0x77,
0xa1, 0x60, 0x81, 0x45, 0xf1, 0x78, 0xc0, 0x8c, 0xcf, 0x8a, 0x61, 0x1c, 0xac, 0x49, 0xe5, 0xc7,
0x72, 0x72, 0x31, 0x79, 0x01, 0x09, 0x8f, 0x2f, 0x4d, 0x59, 0x12, 0x5f, 0x00, 0x67, 0x27, 0xf0,
0xbd, 0x85, 0xcc, 0x24, 0x77, 0xe0, 0xaf, 0x43, 0x92, 0xfb, 0x0b, 0x19, 0x29, 0xc9, 0x7d, 0x61,
0x23, 0x2d, 0x7c, 0x4a, 0xdd, 0xd0, 0xd4, 0x8c, 0x7c, 0x03, 0xf2, 0x52, 0xa8, 0x3b, 0x6e, 0x3f,
0x98, 0x9b, 0xcb, 0xe7, 0x0a, 0x45, 0x2e, 0xc1, 0x11, 0x95, 0xa7, 0xe6, 0x6f, 0x2e, 0xce, 0xdb,
0x5c, 0x34, 0x9b, 0x5f, 0x85, 0xf4, 0x7a, 0x6f, 0x3c, 0x7c, 0x20, 0xb6, 0x74, 0x5c, 0xee, 0x9a,
0x2d, 0x62, 0x4c, 0x5e, 0x81, 0x15, 0x11, 0x81, 0x2c, 0x08, 0xd7, 0xfd, 0xf1, 0x90, 0x9b, 0x1b,
0xf2, 0x59, 0x28, 0x4f, 0x83, 0xb5, 0x8f, 0x94, 0x21, 0xdd, 0x16, 0x00, 0x49, 0xa3, 0x44, 0xd5,
0x84, 0xfc, 0x0a, 0x01, 0xbe, 0xca, 0xb8, 0x3c, 0x65, 0x73, 0x23, 0xb4, 0x6e, 0x45, 0x9e, 0xcb,
0xdb, 0x3d, 0x16, 0x84, 0xe6, 0x86, 0x60, 0xe6, 0xff, 0x8f, 0x5b, 0x11, 0x39, 0x07, 0x2b, 0x53,
0x5c, 0x6a, 0x99, 0xaa, 0x90, 0x6b, 0x6b, 0x98, 0xae, 0x46, 0xd1, 0x9c, 0xfc, 0x3e, 0x09, 0x39,
0xb9, 0x81, 0xb2, 0x1d, 0x7c, 0x0e, 0x0a, 0x3b, 0xfd, 0x61, 0x97, 0x05, 0xa3, 0xa0, 0xaf, 0x55,
0xe0, 0xb4, 0x8e, 0xec, 0x4f, 0xea, 0x36, 0x98, 0xda, 0x13, 0xfc, 0x26, 0x64, 0xc7, 0x21, 0x0b,
0xee, 0xf7, 0x55, 0x9c, 0xe7, 0x5b, 0xe5, 0xbd, 0x49, 0x3d, 0xf3, 0xfd, 0x90, 0x05, 0x9b, 0x1b,
0xa2, 0x2e, 0x8c, 0xe5, 0x88, 0xaa, 0x6f, 0x07, 0x5f, 0xd7, 0x6e, 0x2a, 0xaf, 0x48, 0xad, 0x6f,
0x0a, 0xf6, 0x67, 0x12, 0xdd, 0x28, 0xf0, 0x3d, 0xc6, 0x7b, 0x6c, 0x1c, 0x36, 0xdb, 0xbe, 0xe7,
0xf9, 0xc3, 0xa6, 0xec, 0x87, 0xa4, 0xd0, 0xa2, 0xb8, 0x89, 0xed, 0xda, 0x73, 0xef, 0x42, 0x96,
0xf7, 0x02, 0x7f, 0xdc, 0xed, 0xc9, 0xbc, 0x9d, 0x6a, 0x5d, 0x5c, 0x9c, 0x9e, 0xa1, 0x40, 0xcd,
0x00, 0x9f, 0x12, 0xda, 0x62, 0xed, 0x07, 0xe1, 0xd8, 0x53, 0x5d, 0x46, 0x2b, 0xbd, 0x3f, 0xa9,
0xa3, 0x37, 0x69, 0x04, 0x26, 0x9f, 0x24, 0xa1, 0x2e, 0x1d, 0xf5, 0x9e, 0x2c, 0xea, 0x57, 0xfc,
0xe0, 0x26, 0xe3, 0x41, 0xbf, 0x7d, 0xcb, 0xf5, 0x98, 0xf1, 0x8d, 0x3a, 0x14, 0x3c, 0x09, 0xbc,
0x6f, 0x85, 0x00, 0x78, 0x11, 0x1e, 0x3e, 0x09, 0x20, 0x63, 0x46, 0xad, 0xab, 0x68, 0xc8, 0x4b,
0x88, 0x5c, 0x5e, 0x9f, 0xd2, 0x54, 0x73, 0x41, 0xc9, 0xb4, 0x86, 0x36, 0x67, 0x35, 0xb4, 0x30,
0x9d, 0x48, 0x2d, 0xb6, 0xaf, 0xa7, 0xa7, 0x7d, 0x9d, 0xfc, 0x0d, 0x41, 0xed, 0x86, 0xe1, 0xfc,
0x39, 0xd5, 0x61, 0xe4, 0x4d, 0xbe, 0x20, 0x79, 0x53, 0xff, 0x9b, 0xbc, 0xe4, 0x4f, 0x56, 0xc8,
0x53, 0xb6, 0x63, 0xe4, 0x58, 0xb7, 0xf2, 0xec, 0x8b, 0x60, 0x33, 0xf9, 0x02, 0xcd, 0x92, 0x9a,
0x31, 0xcb, 0x7b, 0x71, 0x3a, 0x90, 0x12, 0xe8, 0x74, 0x70, 0x1a, 0x9c, 0x80, 0xed, 0x98, 0xaa,
0x85, 0xe3, 0xaa, 0x15, 0x61, 0xca, 0x75, 0xf2, 0x07, 0x04, 0xcb, 0x57, 0x19, 0x9f, 0xbe, 0x0f,
0xbc, 0x4c, 0xf2, 0x7f, 0x00, 0x47, 0x2d, 0xfe, 0xb5, 0xf4, 0x6f, 0xcf, 0x5c, 0x02, 0x5e, 0x89,
0xe5, 0xdf, 0x1c, 0x76, 0xd8, 0xc7, 0xba, 0xb9, 0x99, 0xae, 0xff, 0x77, 0xa0, 0x60, 0x2d, 0xe2,
0xcb, 0x33, 0x95, 0xdf, 0x6a, 0xda, 0xa3, 0xfa, 0xd5, 0x2a, 0x6b, 0x99, 0x54, 0x7b, 0xa3, 0xef,
0x75, 0x51, 0x9d, 0xdc, 0x02, 0x2c, 0xfb, 0x2d, 0x49, 0xd6, 0xce, 0xd4, 0x12, 0x7a, 0x3d, 0xba,
0x08, 0x44, 0x73, 0x7c, 0x0a, 0x9c, 0xc0, 0x7f, 0x64, 0xae, 0x74, 0xa5, 0xf8, 0x48, 0xea, 0x3f,
0xa2, 0x72, 0x89, 0x5c, 0x82, 0x14, 0xf5, 0x1f, 0xe1, 0x1a, 0x40, 0xe0, 0x0e, 0xbb, 0xec, 0x5e,
0x74, 0xd3, 0x2f, 0x52, 0x0b, 0x72, 0x48, 0x15, 0x5d, 0x87, 0xa3, 0x36, 0x47, 0xca, 0xdc, 0x0d,
0xc8, 0x0a, 0x60, 0x7f, 0xde, 0x6b, 0x91, 0x44, 0x54, 0x4d, 0xa3, 0x41, 0x12, 0x3e, 0x03, 0x31,
0x1c, 0x9f, 0x80, 0x3c, 0x77, 0xb7, 0x07, 0xec, 0x56, 0x1c, 0xf3, 0x31, 0x40, 0xac, 0x8a, 0x26,
0xe5, 0x9e, 0x75, 0x1d, 0x88, 0x01, 0xf8, 0x0c, 0x2c, 0xc7, 0x3c, 0xdf, 0x09, 0xd8, 0x4e, 0xff,
0x63, 0x69, 0xe1, 0x22, 0x3d, 0x00, 0xc7, 0x6b, 0x70, 0x24, 0x86, 0x6d, 0xc9, 0xb2, 0xeb, 0x48,
0xd4, 0x59, 0xb0, 0xd0, 0x8d, 0x14, 0xf7, 0xfd, 0x87, 0x63, 0x77, 0x20, 0x13, 0x59, 0x91, 0x5a,
0x10, 0xf2, 0x47, 0x04, 0x47, 0x95, 0xa9, 0x45, 0x7b, 0xfa, 0x32, 0x7a, 0xfd, 0xaf, 0x11, 0x60,
0x5b, 0x02, 0xed, 0x5a, 0x5f, 0xb3, 0xdf, 0x1d, 0x44, 0x5d, 0x2f, 0xc8, 0xde, 0x4b, 0x81, 0xe2,
0xa7, 0x03, 0x02, 0x19, 0x79, 0x37, 0x50, 0x4d, 0xa0, 0xa3, 0x9a, 0x3b, 0x05, 0xa1, 0xfa, 0x2b,
0x7a, 0xd2, 0xed, 0x5d, 0xce, 0x42, 0xdd, 0x9a, 0xc9, 0x9e, 0x54, 0x02, 0xa8, 0xfa, 0x88, 0xb3,
0xd8, 0x90, 0x4b, 0xaf, 0x71, 0xe2, 0xb3, 0x34, 0x88, 0x9a, 0x01, 0xf9, 0x6d, 0x12, 0x4a, 0xf7,
0xfc, 0xc1, 0x38, 0xae, 0x12, 0x2f, 0x91, 0x9e, 0xa7, 0x7b, 0xc6, 0xb4, 0xe9, 0x19, 0x31, 0x38,
0x21, 0x67, 0x23, 0xe9, 0x59, 0x29, 0x2a, 0xc7, 0x98, 0x40, 0x91, 0xbb, 0x41, 0x97, 0x71, 0x75,
0xdd, 0xaf, 0x64, 0xe4, 0x1d, 0x6c, 0x0a, 0x86, 0x57, 0xa1, 0xe0, 0x76, 0xbb, 0x01, 0xeb, 0xba,
0x9c, 0xb5, 0x76, 0x2b, 0x59, 0x79, 0x98, 0x0d, 0x22, 0x3f, 0x84, 0x25, 0xa3, 0x2c, 0x6d, 0xd2,
0xb7, 0x20, 0xfb, 0x91, 0x84, 0xcc, 0x79, 0xa3, 0x51, 0xa8, 0x3a, 0x8d, 0x19, 0xb4, 0xe9, 0xa7,
0x4f, 0xc3, 0x33, 0xb9, 0x06, 0x19, 0x85, 0x8e, 0x4f, 0xd8, 0x17, 0x76, 0xf5, 0x98, 0x20, 0xe6,
0xfa, 0xf6, 0x4d, 0x20, 0xa3, 0x08, 0x69, 0xc3, 0x4b, 0xdf, 0x50, 0x10, 0xaa, 0xbf, 0x67, 0x4e,
0x43, 0x3e, 0x7a, 0xb7, 0xc4, 0x05, 0xc8, 0x5e, 0xb9, 0x4d, 0x7f, 0x70, 0x99, 0x6e, 0x2c, 0x27,
0x70, 0x11, 0x72, 0xad, 0xcb, 0xeb, 0xd7, 0xe5, 0x0c, 0x9d, 0xff, 0xb7, 0x63, 0x32, 0x4b, 0x80,
0xbf, 0x05, 0x69, 0x95, 0x2e, 0x8e, 0xc5, 0xfc, 0xdb, 0xaf, 0x8f, 0xd5, 0xe3, 0x07, 0xe0, 0x4a,
0x03, 0x24, 0xf1, 0x16, 0xc2, 0xb7, 0xa0, 0x20, 0x81, 0xfa, 0x85, 0xe3, 0xc4, 0xec, 0x43, 0xc3,
0x14, 0xa5, 0x93, 0x87, 0xac, 0x5a, 0xf4, 0x2e, 0x42, 0x5a, 0xda, 0xc4, 0xe6, 0xc6, 0x7e, 0xa1,
0xb2, 0xb9, 0x99, 0x7a, 0xf3, 0x21, 0x09, 0xfc, 0x2e, 0x38, 0xa2, 0xab, 0xc0, 0x56, 0x51, 0xb1,
0x1e, 0x26, 0xaa, 0xc7, 0x66, 0xc1, 0xd6, 0xb1, 0xef, 0x45, 0xef, 0x2b, 0xc7, 0x67, 0xfb, 0x48,
0xb3, 0xbd, 0x72, 0x70, 0x21, 0x3a, 0xf9, 0xb6, 0x7a, 0x68, 0x30, 0xfd, 0x0c, 0x3e, 0x39, 0x7d,
0xd4, 0x4c, 0xfb, 0x53, 0xad, 0x1d, 0xb6, 0x1c, 0x11, 0xbc, 0x01, 0x05, 0xab, 0x97, 0xb0, 0xd5,
0x7a, 0xb0, 0x11, 0xb2, 0xd5, 0x3a, 0xa7, 0x01, 0x21, 0x09, 0x7c, 0x15, 0x72, 0xa2, 0x14, 0x8b,
0x8c, 0x84, 0x5f, 0x9d, 0xad, 0xb8, 0x56, 0xa6, 0xad, 0x9e, 0x98, 0xbf, 0x18, 0x11, 0xfa, 0x2e,
0xe4, 0xaf, 0x32, 0xae, 0xdd, 0xf5, 0xf8, 0xac, 0xbf, 0xcf, 0xd1, 0xd4, 0x74, 0xcc, 0x90, 0xc4,
0xf9, 0x1f, 0x9b, 0xff, 0x32, 0x36, 0x5c, 0xee, 0xe2, 0xdb, 0xb0, 0x24, 0x19, 0x8b, 0xfe, 0xec,
0x98, 0x72, 0xa0, 0x03, 0xff, 0xac, 0x4c, 0x39, 0xd0, 0xc1, 0x7f, 0x58, 0x48, 0xa2, 0xf5, 0xe1,
0x93, 0xa7, 0xb5, 0xc4, 0x67, 0x4f, 0x6b, 0x89, 0xcf, 0x9f, 0xd6, 0xd0, 0x4f, 0xf7, 0x6a, 0xe8,
0x37, 0x7b, 0x35, 0xf4, 0x78, 0xaf, 0x86, 0x9e, 0xec, 0xd5, 0xd0, 0x3f, 0xf7, 0x6a, 0xe8, 0x5f,
0x7b, 0xb5, 0xc4, 0xe7, 0x7b, 0x35, 0xf4, 0xe9, 0xb3, 0x5a, 0xe2, 0xc9, 0xb3, 0x5a, 0xe2, 0xb3,
0x67, 0xb5, 0xc4, 0x87, 0xaf, 0x7d, 0xd1, 0x53, 0x8f, 0x39, 0x71, 0x3b, 0x23, 0x3f, 0x6f, 0xff,
0x37, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x41, 0xfe, 0xef, 0x8a, 0x1a, 0x00, 0x00,
// 2202 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0x4d, 0x8f, 0x1b, 0x49,
0xd5, 0x6d, 0xb7, 0xbf, 0x9e, 0x3d, 0x93, 0x49, 0x8d, 0x37, 0xb1, 0x9c, 0xc4, 0x9e, 0x94, 0x96,
0xec, 0x28, 0x9b, 0xb5, 0x37, 0xb3, 0xb0, 0x64, 0x13, 0x16, 0x88, 0x67, 0xf2, 0x31, 0xc9, 0xe4,
0x83, 0x9a, 0x10, 0xd0, 0x0a, 0x14, 0xf5, 0xd8, 0x65, 0xbb, 0x15, 0xb7, 0xdb, 0xe9, 0x2e, 0x6f,
0x32, 0x12, 0x07, 0xfe, 0xc0, 0x4a, 0x7b, 0x43, 0x5c, 0x10, 0x07, 0x24, 0x90, 0x10, 0x17, 0x7e,
0x00, 0x5c, 0x90, 0x08, 0xb7, 0x70, 0x5b, 0x71, 0x30, 0x64, 0x72, 0x41, 0x73, 0xda, 0x1b, 0x12,
0x07, 0x84, 0xea, 0xab, 0xbb, 0xec, 0xf1, 0xec, 0xae, 0x43, 0x24, 0x94, 0x8b, 0xbb, 0xde, 0xab,
0x57, 0xaf, 0xde, 0x77, 0xd5, 0x2b, 0xc3, 0x89, 0xe1, 0xc3, 0x6e, 0xa3, 0xef, 0x77, 0x87, 0x81,
0xcf, 0xfc, 0x68, 0x50, 0x17, 0xbf, 0x28, 0xa7, 0xe1, 0x4a, 0xa9, 0xeb, 0x77, 0x7d, 0x49, 0xc3,
0x47, 0x72, 0xbe, 0x52, 0xeb, 0xfa, 0x7e, 0xb7, 0x4f, 0x1b, 0x02, 0xda, 0x19, 0x75, 0x1a, 0xcc,
0xf5, 0x68, 0xc8, 0x1c, 0x6f, 0xa8, 0x08, 0x56, 0x14, 0xf7, 0x47, 0x7d, 0xcf, 0x6f, 0xd3, 0x7e,
0x23, 0x64, 0x0e, 0x0b, 0xe5, 0xaf, 0xa2, 0x58, 0xe6, 0x14, 0xc3, 0x51, 0xd8, 0x13, 0x3f, 0x12,
0x89, 0x4b, 0x80, 0xb6, 0x59, 0x40, 0x1d, 0x8f, 0x38, 0x8c, 0x86, 0x84, 0x3e, 0x1a, 0xd1, 0x90,
0xe1, 0x5b, 0xb0, 0x3c, 0x81, 0x0d, 0x87, 0xfe, 0x20, 0xa4, 0xe8, 0x7d, 0x28, 0x84, 0x31, 0xba,
0x6c, 0xad, 0xa4, 0x56, 0x0b, 0x6b, 0xa5, 0x7a, 0xa4, 0x4a, 0xbc, 0x86, 0x98, 0x84, 0xf8, 0x17,
0x16, 0x40, 0x3c, 0x87, 0xaa, 0x00, 0x72, 0xf6, 0xba, 0x13, 0xf6, 0xca, 0xd6, 0x8a, 0xb5, 0x6a,
0x13, 0x03, 0x83, 0xce, 0xc1, 0xd1, 0x18, 0xba, 0xed, 0x6f, 0xf7, 0x9c, 0xa0, 0x5d, 0x4e, 0x0a,
0xb2, 0x83, 0x13, 0x08, 0x81, 0x1d, 0x38, 0x8c, 0x96, 0x53, 0x2b, 0xd6, 0x6a, 0x8a, 0x88, 0x31,
0x3a, 0x06, 0x19, 0x46, 0x07, 0xce, 0x80, 0x95, 0xed, 0x15, 0x6b, 0x35, 0x4f, 0x14, 0xc4, 0xf1,
0x5c, 0x77, 0x1a, 0x96, 0xd3, 0x2b, 0xd6, 0xea, 0x02, 0x51, 0x10, 0xfe, 0x73, 0x12, 0x8a, 0xdf,
0x1b, 0xd1, 0x60, 0x57, 0x19, 0x00, 0x55, 0x20, 0x17, 0xd2, 0x3e, 0x6d, 0x31, 0x3f, 0x10, 0x02,
0xe6, 0x49, 0x04, 0xa3, 0x12, 0xa4, 0xfb, 0xae, 0xe7, 0x32, 0x21, 0xd2, 0x02, 0x91, 0x00, 0xba,
0x08, 0xe9, 0x90, 0x39, 0x01, 0x13, 0x72, 0x14, 0xd6, 0x2a, 0x75, 0xe9, 0xb0, 0xba, 0x76, 0x58,
0xfd, 0x9e, 0x76, 0x58, 0x33, 0xf7, 0x74, 0x5c, 0x4b, 0x7c, 0xfa, 0xf7, 0x9a, 0x45, 0xe4, 0x12,
0xf4, 0x3e, 0xa4, 0xe8, 0xa0, 0x2d, 0x64, 0xfd, 0xaa, 0x2b, 0xf9, 0x02, 0x74, 0x1e, 0xf2, 0x6d,
0x37, 0xa0, 0x2d, 0xe6, 0xfa, 0x03, 0xa1, 0xd1, 0xe2, 0xda, 0x72, 0xec, 0x8d, 0x0d, 0x3d, 0x45,
0x62, 0x2a, 0x74, 0x0e, 0x32, 0x21, 0x37, 0x5b, 0x58, 0xce, 0xae, 0xa4, 0x56, 0xf3, 0xcd, 0xd2,
0xfe, 0xb8, 0xb6, 0x24, 0x31, 0xe7, 0x7c, 0xcf, 0x65, 0xd4, 0x1b, 0xb2, 0x5d, 0xa2, 0x68, 0xd0,
0x59, 0xc8, 0xb6, 0x69, 0x9f, 0x72, 0x67, 0xe7, 0x84, 0xb3, 0x97, 0x0c, 0xf6, 0x62, 0x82, 0x68,
0x82, 0x1b, 0x76, 0x2e, 0xb3, 0x94, 0xc5, 0xff, 0xb1, 0x00, 0x6d, 0x3b, 0xde, 0xb0, 0x4f, 0xbf,
0xb2, 0x3d, 0x23, 0xcb, 0x25, 0x5f, 0xda, 0x72, 0xa9, 0x79, 0x2d, 0x17, 0x9b, 0xc1, 0x9e, 0xcf,
0x0c, 0xe9, 0x2f, 0x31, 0x03, 0xde, 0x82, 0x8c, 0x44, 0x7d, 0x59, 0x0c, 0xc5, 0x3a, 0xa7, 0xb4,
0x36, 0x4b, 0xb1, 0x36, 0x29, 0x21, 0x27, 0xfe, 0xa5, 0x05, 0x0b, 0xca, 0x90, 0x2a, 0x07, 0x77,
0x20, 0x2b, 0x73, 0x40, 0xe7, 0xdf, 0xf1, 0xe9, 0xfc, 0xbb, 0xdc, 0x76, 0x86, 0x8c, 0x06, 0xcd,
0xc6, 0xd3, 0x71, 0xcd, 0xfa, 0xdb, 0xb8, 0xf6, 0x56, 0xd7, 0x65, 0xbd, 0xd1, 0x4e, 0xbd, 0xe5,
0x7b, 0x8d, 0x6e, 0xe0, 0x74, 0x9c, 0x81, 0xd3, 0xe8, 0xfb, 0x0f, 0xdd, 0x86, 0xae, 0x07, 0x3a,
0x6f, 0x35, 0x63, 0xf4, 0xb6, 0x90, 0x8e, 0x85, 0xca, 0x23, 0x47, 0xea, 0xb2, 0x8c, 0x6c, 0x0e,
0xba, 0x34, 0xe4, 0x9c, 0x6d, 0x6e, 0x4c, 0x22, 0x69, 0xf0, 0x4f, 0x60, 0x79, 0xc2, 0xe1, 0x4a,
0xce, 0x0b, 0x90, 0x09, 0x69, 0xe0, 0x46, 0x65, 0xc2, 0x30, 0xd9, 0xb6, 0xc0, 0x37, 0x17, 0x95,
0x7c, 0x19, 0x09, 0x13, 0x45, 0x3f, 0xdf, 0xee, 0x7f, 0xb2, 0xa0, 0xb8, 0xe5, 0xec, 0xd0, 0xbe,
0x8e, 0x34, 0x04, 0xf6, 0xc0, 0xf1, 0xa8, 0xb2, 0xb8, 0x18, 0xf3, 0xb4, 0xff, 0xd8, 0xe9, 0x8f,
0xa8, 0x64, 0x99, 0x23, 0x0a, 0x9a, 0x37, 0x67, 0xad, 0x97, 0xce, 0x59, 0x2b, 0x8e, 0xbc, 0x12,
0xa4, 0x1f, 0x71, 0x43, 0x89, 0x7c, 0xcd, 0x13, 0x09, 0xe0, 0xb7, 0x60, 0x41, 0x69, 0xa1, 0xcc,
0x17, 0x8b, 0xcc, 0xcd, 0x97, 0xd7, 0x22, 0x63, 0x0f, 0x32, 0xd2, 0xda, 0xe8, 0x4d, 0xc8, 0x47,
0x67, 0x80, 0xd0, 0x36, 0xd5, 0xcc, 0xec, 0x8f, 0x6b, 0x49, 0x16, 0x92, 0x78, 0x02, 0xd5, 0x20,
0x2d, 0x56, 0x0a, 0xcd, 0xad, 0x66, 0x7e, 0x7f, 0x5c, 0x93, 0x08, 0x22, 0x3f, 0xe8, 0x24, 0xd8,
0x3d, 0x5e, 0x86, 0xb9, 0x09, 0xec, 0x66, 0x6e, 0x7f, 0x5c, 0x13, 0x30, 0x11, 0xbf, 0xf8, 0x1a,
0x14, 0xb7, 0x68, 0xd7, 0x69, 0xed, 0xaa, 0x4d, 0x4b, 0x9a, 0x1d, 0xdf, 0xd0, 0xd2, 0x3c, 0x4e,
0x43, 0x31, 0xda, 0xf1, 0x81, 0x17, 0xaa, 0xa0, 0x2e, 0x44, 0xb8, 0x5b, 0x21, 0xfe, 0xb9, 0x05,
0xca, 0xcf, 0x08, 0x43, 0xa6, 0xcf, 0x75, 0x0d, 0xa5, 0x8f, 0x9a, 0xb0, 0x3f, 0xae, 0x29, 0x0c,
0x51, 0x5f, 0x74, 0x09, 0xb2, 0xa1, 0xd8, 0x91, 0x33, 0x9b, 0x0e, 0x1f, 0x31, 0xd1, 0x3c, 0xc2,
0xc3, 0x60, 0x7f, 0x5c, 0xd3, 0x84, 0x44, 0x0f, 0x50, 0x7d, 0xe2, 0x7c, 0x91, 0x8a, 0x2d, 0xee,
0x8f, 0x6b, 0x06, 0xd6, 0x3c, 0x6f, 0xf0, 0xcf, 0x2c, 0x28, 0xdc, 0x73, 0xdc, 0x28, 0x84, 0x22,
0x17, 0x59, 0x86, 0x8b, 0x78, 0x3a, 0xb7, 0x69, 0xdf, 0xd9, 0xbd, 0xea, 0x07, 0x82, 0xe7, 0x02,
0x89, 0xe0, 0xf8, 0x48, 0xb0, 0x67, 0x1e, 0x09, 0xe9, 0xb9, 0x0b, 0xdb, 0x0d, 0x3b, 0x97, 0x5c,
0x4a, 0xe1, 0xdf, 0x59, 0x50, 0x94, 0x92, 0xa9, 0xb0, 0xf8, 0x11, 0x64, 0xa4, 0xe0, 0x42, 0xb6,
0x2f, 0x48, 0xfe, 0xb7, 0xe7, 0x49, 0x7c, 0xc5, 0x13, 0x7d, 0x07, 0x16, 0xdb, 0x81, 0x3f, 0x1c,
0xd2, 0xf6, 0xb6, 0x2a, 0x31, 0xc9, 0xe9, 0x12, 0xb3, 0x61, 0xce, 0x93, 0x29, 0x72, 0xfc, 0x17,
0x0b, 0x16, 0x54, 0x36, 0x2b, 0x5b, 0x46, 0x36, 0xb0, 0x5e, 0xba, 0xb8, 0x27, 0xe7, 0x2d, 0xee,
0xc7, 0x20, 0xd3, 0x0d, 0xfc, 0xd1, 0x30, 0x2c, 0xa7, 0x64, 0xee, 0x48, 0x68, 0xbe, 0xa2, 0x8f,
0x6f, 0xc0, 0xa2, 0x56, 0xe5, 0x90, 0x92, 0x56, 0x99, 0x2e, 0x69, 0x9b, 0x6d, 0x3a, 0x60, 0x6e,
0xc7, 0x8d, 0x8a, 0x94, 0xa2, 0xc7, 0x9f, 0x58, 0xb0, 0x34, 0x4d, 0x82, 0xbe, 0x6d, 0xe4, 0x01,
0x67, 0x77, 0xe6, 0x70, 0x76, 0x75, 0x51, 0x1c, 0xc2, 0x2b, 0x03, 0x16, 0xec, 0xea, 0x1c, 0xa9,
0x7c, 0x00, 0x05, 0x03, 0xcd, 0x0f, 0x8f, 0x87, 0x54, 0xc7, 0x2c, 0x1f, 0xc6, 0xc9, 0x9a, 0x94,
0x71, 0x2c, 0x80, 0x8b, 0xc9, 0x0b, 0x16, 0x8f, 0xf8, 0x85, 0x09, 0x4f, 0xa2, 0x0b, 0x60, 0x77,
0x02, 0xdf, 0x9b, 0xcb, 0x4d, 0x62, 0x05, 0xfa, 0x3a, 0x24, 0x99, 0x3f, 0x97, 0x93, 0x92, 0xcc,
0xe7, 0x3e, 0x52, 0xca, 0xa7, 0xe4, 0x0d, 0x4d, 0x42, 0xf8, 0x1b, 0x90, 0x17, 0x4a, 0xdd, 0x75,
0xdc, 0x60, 0x66, 0x2d, 0x9f, 0xa9, 0x14, 0xbe, 0x04, 0x47, 0x64, 0x9d, 0x9a, 0xbd, 0xb8, 0x38,
0x6b, 0x71, 0x51, 0x2f, 0x3e, 0x01, 0xe9, 0xf5, 0xde, 0x68, 0xf0, 0x90, 0x2f, 0x69, 0x3b, 0xcc,
0xd1, 0x4b, 0xf8, 0x18, 0xbf, 0x01, 0xcb, 0x3c, 0x03, 0x69, 0x10, 0xae, 0xfb, 0xa3, 0x01, 0xd3,
0x37, 0xe4, 0x73, 0x50, 0x9a, 0x44, 0xab, 0x18, 0x29, 0x41, 0xba, 0xc5, 0x11, 0x82, 0xc7, 0x02,
0x91, 0x00, 0xfe, 0x95, 0x05, 0xe8, 0x1a, 0x65, 0x62, 0x97, 0xcd, 0x8d, 0xd0, 0xb8, 0x15, 0x79,
0x0e, 0x6b, 0xf5, 0x68, 0x10, 0xea, 0x1b, 0x82, 0x86, 0xff, 0x1f, 0xb7, 0x22, 0x7c, 0x1e, 0x96,
0x27, 0xa4, 0x54, 0x3a, 0x55, 0x20, 0xd7, 0x52, 0x38, 0x75, 0x1a, 0x45, 0x30, 0xfe, 0x7d, 0x12,
0x72, 0x62, 0x01, 0xa1, 0x1d, 0x74, 0x1e, 0x0a, 0x1d, 0x77, 0xd0, 0xa5, 0xc1, 0x30, 0x70, 0x95,
0x09, 0xec, 0xe6, 0x91, 0xfd, 0x71, 0xcd, 0x44, 0x13, 0x13, 0x40, 0xef, 0x40, 0x76, 0x14, 0xd2,
0xe0, 0x81, 0x2b, 0xf3, 0x3c, 0xdf, 0x2c, 0xed, 0x8d, 0x6b, 0x99, 0xef, 0x87, 0x34, 0xd8, 0xdc,
0xe0, 0xe7, 0xc2, 0x48, 0x8c, 0x88, 0xfc, 0xb6, 0xd1, 0x4d, 0x15, 0xa6, 0xe2, 0x8a, 0xd4, 0xfc,
0x26, 0x17, 0x7f, 0xaa, 0xd0, 0x0d, 0x03, 0xdf, 0xa3, 0xac, 0x47, 0x47, 0x61, 0xa3, 0xe5, 0x7b,
0x9e, 0x3f, 0x68, 0x88, 0x7e, 0x48, 0x28, 0xcd, 0x0f, 0x37, 0xbe, 0x5c, 0x45, 0xee, 0x3d, 0xc8,
0xb2, 0x5e, 0xe0, 0x8f, 0xba, 0x3d, 0x51, 0xb7, 0x53, 0xcd, 0x8b, 0xf3, 0xf3, 0xd3, 0x1c, 0x88,
0x1e, 0xa0, 0xd3, 0xdc, 0x5a, 0xb4, 0xf5, 0x30, 0x1c, 0x79, 0xb2, 0xcb, 0x68, 0xa6, 0xf7, 0xc7,
0x35, 0xeb, 0x1d, 0x12, 0xa1, 0xf1, 0x27, 0x49, 0xa8, 0x89, 0x40, 0xbd, 0x2f, 0x0e, 0xf5, 0xab,
0x7e, 0x70, 0x8b, 0xb2, 0xc0, 0x6d, 0xdd, 0x76, 0x3c, 0xaa, 0x63, 0xa3, 0x06, 0x05, 0x4f, 0x20,
0x1f, 0x18, 0x29, 0x00, 0x5e, 0x44, 0x87, 0x4e, 0x01, 0x88, 0x9c, 0x91, 0xf3, 0x32, 0x1b, 0xf2,
0x02, 0x23, 0xa6, 0xd7, 0x27, 0x2c, 0xd5, 0x98, 0x53, 0x33, 0x65, 0xa1, 0xcd, 0x69, 0x0b, 0xcd,
0xcd, 0x27, 0x32, 0x8b, 0x19, 0xeb, 0xe9, 0xc9, 0x58, 0xc7, 0x7f, 0xb5, 0xa0, 0xba, 0xa5, 0x25,
0x7f, 0x49, 0x73, 0x68, 0x7d, 0x93, 0xaf, 0x48, 0xdf, 0xd4, 0xff, 0xa6, 0x2f, 0xbe, 0x0e, 0xa5,
0x2d, 0x77, 0x40, 0xaf, 0xba, 0x7d, 0x46, 0x83, 0x2b, 0x4f, 0x86, 0x01, 0x0d, 0x43, 0xde, 0x80,
0x55, 0x20, 0xe7, 0x0f, 0x69, 0xe0, 0xe8, 0xae, 0x20, 0x45, 0x22, 0x98, 0x17, 0x0f, 0x61, 0x13,
0x5d, 0xdb, 0x04, 0x80, 0xff, 0x6d, 0x14, 0x0f, 0x42, 0x3b, 0xda, 0x22, 0xeb, 0x46, 0xc5, 0x7e,
0x15, 0x0a, 0x27, 0x5f, 0xa1, 0x83, 0x53, 0x53, 0xc5, 0xec, 0x02, 0x64, 0x3b, 0xc2, 0x10, 0xf2,
0xe8, 0x2d, 0xac, 0x55, 0xe3, 0xb3, 0x6e, 0x96, 0x95, 0x88, 0x26, 0xc7, 0x1f, 0xc6, 0x25, 0x49,
0xe8, 0xae, 0x4a, 0xd2, 0x19, 0xb0, 0x03, 0xda, 0xd1, 0x27, 0x27, 0x8a, 0xb9, 0x45, 0x94, 0x62,
0x1e, 0xff, 0xc1, 0x82, 0xa5, 0x6b, 0x94, 0x4d, 0xde, 0x49, 0x5e, 0x23, 0xcb, 0xe1, 0xeb, 0x70,
0xd4, 0x90, 0x5f, 0x69, 0xff, 0xde, 0xd4, 0x45, 0xe4, 0x8d, 0x58, 0xff, 0xcd, 0x41, 0x9b, 0x3e,
0x51, 0x0d, 0xd6, 0xe4, 0x1d, 0xe4, 0x2e, 0x14, 0x8c, 0x49, 0x74, 0x79, 0xea, 0xf6, 0x61, 0x3c,
0x1c, 0x44, 0x67, 0x68, 0xb3, 0xa4, 0x74, 0x92, 0x2d, 0x96, 0xba, 0x5b, 0x46, 0x67, 0xf5, 0x36,
0x20, 0xd1, 0xf3, 0x09, 0xb6, 0xe6, 0x69, 0x21, 0xb0, 0x37, 0xa3, 0xcb, 0x48, 0x04, 0xa3, 0xd3,
0x60, 0x07, 0xfe, 0x63, 0x7d, 0xad, 0x5c, 0x88, 0xb7, 0x24, 0xfe, 0x63, 0x22, 0xa6, 0xf0, 0x25,
0x48, 0x11, 0xff, 0x31, 0xaa, 0x02, 0x04, 0xce, 0xa0, 0x4b, 0xef, 0x47, 0xdd, 0x46, 0x91, 0x18,
0x98, 0x43, 0x4e, 0xf2, 0x75, 0x38, 0x6a, 0x4a, 0x24, 0xdd, 0x5d, 0x87, 0x2c, 0x47, 0xba, 0xb3,
0x5e, 0xac, 0x04, 0xa1, 0x6c, 0x5c, 0x35, 0x11, 0x8f, 0x19, 0x88, 0xf1, 0xe8, 0x24, 0xe4, 0x99,
0xb3, 0xd3, 0xa7, 0xb7, 0xe3, 0xba, 0x13, 0x23, 0xf8, 0x2c, 0x6f, 0x94, 0xee, 0x1b, 0x57, 0x92,
0x18, 0x81, 0xce, 0xc2, 0x52, 0x2c, 0xf3, 0xdd, 0x80, 0x76, 0xdc, 0x27, 0xc2, 0xc3, 0x45, 0x72,
0x00, 0x8f, 0x56, 0xe1, 0x48, 0x8c, 0xdb, 0x16, 0x47, 0xbf, 0x2d, 0x48, 0xa7, 0xd1, 0xdc, 0x36,
0x42, 0xdd, 0x2b, 0x8f, 0x46, 0x4e, 0x5f, 0x14, 0xd3, 0x22, 0x31, 0x30, 0xf8, 0x8f, 0x16, 0x1c,
0x95, 0xae, 0xe6, 0x2d, 0xf2, 0xeb, 0x18, 0xf5, 0xbf, 0xb6, 0x00, 0x99, 0x1a, 0xa8, 0xd0, 0xfa,
0x9a, 0xf9, 0xf6, 0xc1, 0xef, 0x16, 0x05, 0xd1, 0xff, 0x49, 0x54, 0xfc, 0x7c, 0x81, 0x21, 0x23,
0xee, 0x27, 0xb2, 0x11, 0xb5, 0x65, 0x83, 0x29, 0x31, 0x44, 0x7d, 0x79, 0x5f, 0xbc, 0xb3, 0xcb,
0x68, 0xa8, 0xda, 0x43, 0xd1, 0x17, 0x0b, 0x04, 0x91, 0x1f, 0xbe, 0x17, 0x1d, 0x30, 0x11, 0x35,
0x76, 0xbc, 0x97, 0x42, 0x11, 0x3d, 0xc0, 0xbf, 0x4d, 0xc2, 0xc2, 0x7d, 0xbf, 0x3f, 0x8a, 0x4f,
0xaa, 0xd7, 0xa9, 0x2e, 0x4f, 0xf4, 0xad, 0x69, 0xdd, 0xb7, 0x22, 0xb0, 0x43, 0x46, 0x87, 0x22,
0xb2, 0x52, 0x44, 0x8c, 0x11, 0x86, 0x22, 0x73, 0x82, 0x2e, 0x65, 0xb2, 0xe5, 0x28, 0x67, 0xc4,
0x3d, 0x70, 0x02, 0x87, 0x56, 0xa0, 0xe0, 0x74, 0xbb, 0x01, 0xed, 0x3a, 0x8c, 0x36, 0x77, 0xcb,
0x59, 0xb1, 0x99, 0x89, 0xc2, 0x3f, 0x84, 0x45, 0x6d, 0x2c, 0xe5, 0xd2, 0x77, 0x21, 0xfb, 0xb1,
0xc0, 0xcc, 0x78, 0x27, 0x92, 0xa4, 0xaa, 0x8c, 0x69, 0xb2, 0xc9, 0xe7, 0x57, 0x2d, 0x33, 0xbe,
0x01, 0x19, 0x49, 0x8e, 0x4e, 0x9a, 0x4d, 0x83, 0x7c, 0xd0, 0xe0, 0xb0, 0xea, 0x00, 0x30, 0x64,
0x24, 0x23, 0xe5, 0x78, 0x11, 0x1b, 0x12, 0x43, 0xd4, 0xf7, 0xec, 0x19, 0xc8, 0x47, 0x6f, 0xa7,
0xa8, 0x00, 0xd9, 0xab, 0x77, 0xc8, 0x0f, 0x2e, 0x93, 0x8d, 0xa5, 0x04, 0x2a, 0x42, 0xae, 0x79,
0x79, 0xfd, 0xa6, 0x80, 0xac, 0xb5, 0x7f, 0xd9, 0xba, 0xb2, 0x04, 0xe8, 0x5b, 0x90, 0x96, 0xe5,
0xe2, 0x58, 0x2c, 0xbf, 0xf9, 0x02, 0x5a, 0x39, 0x7e, 0x00, 0x2f, 0x2d, 0x80, 0x13, 0xef, 0x5a,
0xe8, 0x36, 0x14, 0x04, 0x52, 0xbd, 0xb2, 0x9c, 0x9c, 0x7e, 0xec, 0x98, 0xe0, 0x74, 0xea, 0x90,
0x59, 0x83, 0xdf, 0x45, 0x48, 0x0b, 0x9f, 0x98, 0xd2, 0x98, 0xaf, 0x64, 0xa6, 0x34, 0x13, 0xef,
0x4e, 0x38, 0x81, 0x3e, 0x00, 0x9b, 0x77, 0x36, 0xc8, 0x38, 0x54, 0x8c, 0xc7, 0x91, 0xca, 0xb1,
0x69, 0xb4, 0xb1, 0xed, 0x87, 0xd1, 0x1b, 0xcf, 0xf1, 0xe9, 0x5e, 0x56, 0x2f, 0x2f, 0x1f, 0x9c,
0x88, 0x76, 0xbe, 0x23, 0x1f, 0x3b, 0x74, 0x4f, 0x85, 0x4e, 0x4d, 0x6e, 0x35, 0xd5, 0x82, 0x55,
0xaa, 0x87, 0x4d, 0x47, 0x0c, 0xb7, 0xa0, 0x60, 0xf4, 0x33, 0xa6, 0x59, 0x0f, 0x36, 0x63, 0xa6,
0x59, 0x67, 0x34, 0x41, 0x38, 0x81, 0xae, 0x41, 0x8e, 0x1f, 0xc5, 0xbc, 0x22, 0xa1, 0x13, 0xd3,
0x27, 0xae, 0x51, 0x69, 0x2b, 0x27, 0x67, 0x4f, 0x46, 0x8c, 0xbe, 0x0b, 0xf9, 0x6b, 0x94, 0xa9,
0x70, 0x3d, 0x3e, 0x1d, 0xef, 0x33, 0x2c, 0x35, 0x99, 0x33, 0x38, 0xb1, 0xf6, 0x63, 0xfd, 0x7f,
0xca, 0x86, 0xc3, 0x1c, 0x74, 0x07, 0x16, 0x85, 0x60, 0xd1, 0x1f, 0x2e, 0x13, 0x01, 0x74, 0xe0,
0xdf, 0x9d, 0x89, 0x00, 0x3a, 0xf8, 0x2f, 0x0f, 0x4e, 0x34, 0x3f, 0x7a, 0xf6, 0xbc, 0x9a, 0xf8,
0xec, 0x79, 0x35, 0xf1, 0xf9, 0xf3, 0xaa, 0xf5, 0xd3, 0xbd, 0xaa, 0xf5, 0x9b, 0xbd, 0xaa, 0xf5,
0x74, 0xaf, 0x6a, 0x3d, 0xdb, 0xab, 0x5a, 0xff, 0xd8, 0xab, 0x5a, 0xff, 0xdc, 0xab, 0x26, 0x3e,
0xdf, 0xab, 0x5a, 0x9f, 0xbe, 0xa8, 0x26, 0x9e, 0xbd, 0xa8, 0x26, 0x3e, 0x7b, 0x51, 0x4d, 0x7c,
0xf4, 0xe6, 0x17, 0x3d, 0x37, 0xe9, 0x1d, 0x77, 0x32, 0xe2, 0xf3, 0xde, 0x7f, 0x03, 0x00, 0x00,
0xff, 0xff, 0xc7, 0xff, 0x87, 0xf1, 0x0e, 0x1b, 0x00, 0x00,
}
func (x Direction) String() string {
@ -3485,6 +3549,33 @@ func (this *LabelNamesForMetricNameRequest) Equal(that interface{}) bool {
}
return true
}
func (this *LineFilterExpression) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LineFilterExpression)
if !ok {
that2, ok := that.(LineFilterExpression)
if ok {
that1 = &that2
} else {
return false
}
}
if that1 == nil {
return this == nil
} else if this == nil {
return false
}
if this.Operator != that1.Operator {
return false
}
if this.Match != that1.Match {
return false
}
return true
}
func (this *GetChunkRefRequest) Equal(that interface{}) bool {
if that == nil {
return this == nil
@ -3513,6 +3604,14 @@ func (this *GetChunkRefRequest) Equal(that interface{}) bool {
if this.Matchers != that1.Matchers {
return false
}
if len(this.Filters) != len(that1.Filters) {
return false
}
for i := range this.Filters {
if !this.Filters[i].Equal(that1.Filters[i]) {
return false
}
}
return true
}
func (this *GetChunkRefResponse) Equal(that interface{}) bool {
@ -4295,15 +4394,29 @@ func (this *LabelNamesForMetricNameRequest) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LineFilterExpression) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 6)
s = append(s, "&logproto.LineFilterExpression{")
s = append(s, "Operator: "+fmt.Sprintf("%#v", this.Operator)+",\n")
s = append(s, "Match: "+fmt.Sprintf("%#v", this.Match)+",\n")
s = append(s, "}")
return strings.Join(s, "")
}
func (this *GetChunkRefRequest) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 7)
s := make([]string, 0, 8)
s = append(s, "&logproto.GetChunkRefRequest{")
s = append(s, "From: "+fmt.Sprintf("%#v", this.From)+",\n")
s = append(s, "Through: "+fmt.Sprintf("%#v", this.Through)+",\n")
s = append(s, "Matchers: "+fmt.Sprintf("%#v", this.Matchers)+",\n")
if this.Filters != nil {
s = append(s, "Filters: "+fmt.Sprintf("%#v", this.Filters)+",\n")
}
s = append(s, "}")
return strings.Join(s, "")
}
@ -6284,6 +6397,41 @@ func (m *LabelNamesForMetricNameRequest) MarshalToSizedBuffer(dAtA []byte) (int,
return len(dAtA) - i, nil
}
func (m *LineFilterExpression) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *LineFilterExpression) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LineFilterExpression) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Match) > 0 {
i -= len(m.Match)
copy(dAtA[i:], m.Match)
i = encodeVarintLogproto(dAtA, i, uint64(len(m.Match)))
i--
dAtA[i] = 0x12
}
if m.Operator != 0 {
i = encodeVarintLogproto(dAtA, i, uint64(m.Operator))
i--
dAtA[i] = 0x8
}
return len(dAtA) - i, nil
}
func (m *GetChunkRefRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -6304,6 +6452,20 @@ func (m *GetChunkRefRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
if len(m.Filters) > 0 {
for iNdEx := len(m.Filters) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Filters[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintLogproto(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x22
}
}
if len(m.Matchers) > 0 {
i -= len(m.Matchers)
copy(dAtA[i:], m.Matchers)
@ -7437,6 +7599,22 @@ func (m *LabelNamesForMetricNameRequest) Size() (n int) {
return n
}
func (m *LineFilterExpression) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Operator != 0 {
n += 1 + sovLogproto(uint64(m.Operator))
}
l = len(m.Match)
if l > 0 {
n += 1 + l + sovLogproto(uint64(l))
}
return n
}
func (m *GetChunkRefRequest) Size() (n int) {
if m == nil {
return 0
@ -7453,6 +7631,12 @@ func (m *GetChunkRefRequest) Size() (n int) {
if l > 0 {
n += 1 + l + sovLogproto(uint64(l))
}
if len(m.Filters) > 0 {
for _, e := range m.Filters {
l = e.Size()
n += 1 + l + sovLogproto(uint64(l))
}
}
return n
}
@ -8093,14 +8277,31 @@ func (this *LabelNamesForMetricNameRequest) String() string {
}, "")
return s
}
func (this *LineFilterExpression) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LineFilterExpression{`,
`Operator:` + fmt.Sprintf("%v", this.Operator) + `,`,
`Match:` + fmt.Sprintf("%v", this.Match) + `,`,
`}`,
}, "")
return s
}
func (this *GetChunkRefRequest) String() string {
if this == nil {
return "nil"
}
repeatedStringForFilters := "[]*LineFilterExpression{"
for _, f := range this.Filters {
repeatedStringForFilters += strings.Replace(f.String(), "LineFilterExpression", "LineFilterExpression", 1) + ","
}
repeatedStringForFilters += "}"
s := strings.Join([]string{`&GetChunkRefRequest{`,
`From:` + fmt.Sprintf("%v", this.From) + `,`,
`Through:` + fmt.Sprintf("%v", this.Through) + `,`,
`Matchers:` + fmt.Sprintf("%v", this.Matchers) + `,`,
`Filters:` + repeatedStringForFilters + `,`,
`}`,
}, "")
return s
@ -12079,6 +12280,110 @@ func (m *LabelNamesForMetricNameRequest) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LineFilterExpression) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowLogproto
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LineFilterExpression: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LineFilterExpression: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Operator", wireType)
}
m.Operator = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowLogproto
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.Operator |= int64(b&0x7F) << shift
if b < 0x80 {
break
}
}
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Match", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowLogproto
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthLogproto
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthLogproto
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Match = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipLogproto(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthLogproto
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthLogproto
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *GetChunkRefRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@ -12178,6 +12483,40 @@ func (m *GetChunkRefRequest) Unmarshal(dAtA []byte) error {
}
m.Matchers = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 4:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Filters", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowLogproto
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthLogproto
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthLogproto
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Filters = append(m.Filters, &LineFilterExpression{})
if err := m.Filters[len(m.Filters)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipLogproto(dAtA[iNdEx:])

@ -286,6 +286,11 @@ message LabelNamesForMetricNameRequest {
];
}
message LineFilterExpression {
int64 operator = 1;
string match = 2;
}
message GetChunkRefRequest {
int64 from = 1 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
@ -296,6 +301,7 @@ message GetChunkRefRequest {
(gogoproto.nullable) = false
];
string matchers = 3;
repeated LineFilterExpression filters = 4;
}
message GetChunkRefResponse {

@ -302,6 +302,19 @@ func applyConfigToRings(r, defaults *ConfigWrapper, rc util.RingConfig, mergeWit
r.IndexGateway.Ring.ZoneAwarenessEnabled = rc.ZoneAwarenessEnabled
r.IndexGateway.Ring.KVStore = rc.KVStore
}
// BloomGateway
if mergeWithExisting || reflect.DeepEqual(r.BloomGateway.Ring, defaults.BloomGateway.Ring) {
r.BloomGateway.Ring.HeartbeatTimeout = rc.HeartbeatTimeout
r.BloomGateway.Ring.HeartbeatPeriod = rc.HeartbeatPeriod
r.BloomGateway.Ring.InstancePort = rc.InstancePort
r.BloomGateway.Ring.InstanceAddr = rc.InstanceAddr
r.BloomGateway.Ring.InstanceID = rc.InstanceID
r.BloomGateway.Ring.InstanceInterfaceNames = rc.InstanceInterfaceNames
r.BloomGateway.Ring.InstanceZone = rc.InstanceZone
r.BloomGateway.Ring.ZoneAwarenessEnabled = rc.ZoneAwarenessEnabled
r.BloomGateway.Ring.KVStore = rc.KVStore
}
}
func applyTokensFilePath(cfg *ConfigWrapper) error {
@ -332,6 +345,12 @@ func applyTokensFilePath(cfg *ConfigWrapper) error {
}
cfg.IndexGateway.Ring.TokensFilePath = f
f, err = tokensFile(cfg, "bloomgateway.tokens")
if err != nil {
return err
}
cfg.BloomGateway.Ring.TokensFilePath = f
return nil
}
@ -412,6 +431,10 @@ func appendLoopbackInterface(cfg, defaults *ConfigWrapper) {
if reflect.DeepEqual(cfg.IndexGateway.Ring.InstanceInterfaceNames, defaults.IndexGateway.Ring.InstanceInterfaceNames) {
cfg.IndexGateway.Ring.InstanceInterfaceNames = append(cfg.IndexGateway.Ring.InstanceInterfaceNames, loopbackIface)
}
if reflect.DeepEqual(cfg.BloomGateway.Ring.InstanceInterfaceNames, defaults.BloomGateway.Ring.InstanceInterfaceNames) {
cfg.BloomGateway.Ring.InstanceInterfaceNames = append(cfg.BloomGateway.Ring.InstanceInterfaceNames, loopbackIface)
}
}
// applyMemberlistConfig will change the default ingester, distributor, ruler, and query scheduler ring configurations to use memberlist.
@ -425,6 +448,7 @@ func applyMemberlistConfig(r *ConfigWrapper) {
r.QueryScheduler.SchedulerRing.KVStore.Store = memberlistStr
r.CompactorConfig.CompactorRing.KVStore.Store = memberlistStr
r.IndexGateway.Ring.KVStore.Store = memberlistStr
r.BloomGateway.Ring.KVStore.Store = memberlistStr
}
var ErrTooManyStorageConfigs = errors.New("too many storage configs provided in the common config, please only define one storage backend")

@ -30,6 +30,7 @@ import (
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/pkg/analytics"
"github.com/grafana/loki/pkg/bloomgateway"
"github.com/grafana/loki/pkg/compactor"
compactorclient "github.com/grafana/loki/pkg/compactor/client"
"github.com/grafana/loki/pkg/compactor/deletion"
@ -84,6 +85,7 @@ type Config struct {
IngesterClient ingester_client.Config `yaml:"ingester_client,omitempty"`
Ingester ingester.Config `yaml:"ingester,omitempty"`
IndexGateway indexgateway.Config `yaml:"index_gateway"`
BloomGateway bloomgateway.Config `yaml:"bloom_gateway"`
StorageConfig storage.Config `yaml:"storage_config,omitempty"`
ChunkStoreConfig config.ChunkStoreConfig `yaml:"chunk_store_config,omitempty"`
SchemaConfig config.SchemaConfig `yaml:"schema_config,omitempty"`
@ -150,6 +152,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) {
c.Ingester.RegisterFlags(f)
c.StorageConfig.RegisterFlags(f)
c.IndexGateway.RegisterFlags(f)
c.BloomGateway.RegisterFlags(f)
c.ChunkStoreConfig.RegisterFlags(f)
c.SchemaConfig.RegisterFlags(f)
c.LimitsConfig.RegisterFlags(f)
@ -310,6 +313,7 @@ type Loki struct {
querySchedulerRingManager *scheduler.RingManager
usageReport *analytics.Reporter
indexGatewayRingManager *indexgateway.RingManager
bloomGatewayRingManager *bloomgateway.RingManager
clientMetrics storage.ClientMetrics
deleteClientMetrics *deletion.DeleteRequestClientMetrics
@ -589,6 +593,8 @@ func (t *Loki) setupModuleManager() error {
mm.RegisterModule(IndexGateway, t.initIndexGateway)
mm.RegisterModule(IndexGatewayRing, t.initIndexGatewayRing, modules.UserInvisibleModule)
mm.RegisterModule(IndexGatewayInterceptors, t.initIndexGatewayInterceptors, modules.UserInvisibleModule)
mm.RegisterModule(BloomGateway, t.initBloomGateway)
mm.RegisterModule(BloomGatewayRing, t.initBloomGatewayRing, modules.UserInvisibleModule)
mm.RegisterModule(QueryScheduler, t.initQueryScheduler)
mm.RegisterModule(QuerySchedulerRing, t.initQuerySchedulerRing, modules.UserInvisibleModule)
mm.RegisterModule(Analytics, t.initAnalytics)
@ -617,15 +623,19 @@ func (t *Loki) setupModuleManager() error {
RuleEvaluator: {Ring, Server, Store, IngesterQuerier, Overrides, TenantConfigs, Analytics},
TableManager: {Server, Analytics},
Compactor: {Server, Overrides, MemberlistKV, Analytics},
IndexGateway: {Server, Store, Overrides, Analytics, MemberlistKV, IndexGatewayRing, IndexGatewayInterceptors},
IndexGateway: {Server, Store, IndexGatewayRing, IndexGatewayInterceptors, Analytics},
BloomGateway: {Server, BloomGatewayRing, Analytics},
IngesterQuerier: {Ring},
QuerySchedulerRing: {Overrides, Server, MemberlistKV},
IndexGatewayRing: {Overrides, Server, MemberlistKV},
All: {QueryScheduler, QueryFrontend, Querier, Ingester, Distributor, Ruler, Compactor},
Read: {QueryFrontend, Querier},
Write: {Ingester, Distributor},
Backend: {QueryScheduler, Ruler, Compactor, IndexGateway},
QuerySchedulerRing: {Overrides, MemberlistKV},
IndexGatewayRing: {Overrides, MemberlistKV},
BloomGatewayRing: {Overrides, MemberlistKV},
MemberlistKV: {Server},
Read: {QueryFrontend, Querier},
Write: {Ingester, Distributor},
Backend: {QueryScheduler, Ruler, Compactor, IndexGateway},
All: {QueryScheduler, QueryFrontend, Querier, Ingester, Distributor, Ruler, Compactor},
}
if t.Cfg.Querier.PerRequestLimitsEnabled {
@ -679,7 +689,7 @@ func (t *Loki) setupModuleManager() error {
}
if t.Cfg.LegacyReadTarget {
deps[Read] = append(deps[Read], QueryScheduler, Ruler, Compactor, IndexGateway)
deps[Read] = append(deps[Read], QueryScheduler, Ruler, Compactor, IndexGateway, BloomGateway)
}
if t.Cfg.InternalServer.Enable {

@ -34,6 +34,7 @@ import (
"github.com/prometheus/common/version"
"github.com/grafana/loki/pkg/analytics"
"github.com/grafana/loki/pkg/bloomgateway"
"github.com/grafana/loki/pkg/compactor"
compactorclient "github.com/grafana/loki/pkg/compactor/client"
"github.com/grafana/loki/pkg/compactor/client/grpc"
@ -102,6 +103,8 @@ const (
TableManager string = "table-manager"
MemberlistKV string = "memberlist-kv"
Compactor string = "compactor"
BloomGateway string = "bloom-gateway"
BloomGatewayRing string = "bloom-gateway-ring"
IndexGateway string = "index-gateway"
IndexGatewayRing string = "index-gateway-ring"
IndexGatewayInterceptors string = "index-gateway-interceptors"
@ -250,6 +253,7 @@ func (t *Loki) initRuntimeConfig() (services.Service, error) {
t.Cfg.CompactorConfig.CompactorRing.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.Distributor.DistributorRing.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.IndexGateway.Ring.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.BloomGateway.Ring.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.QueryScheduler.SchedulerRing.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
t.Cfg.Ruler.Ring.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig)
@ -1189,9 +1193,49 @@ func (t *Loki) addCompactorMiddleware(h http.HandlerFunc) http.Handler {
return t.HTTPAuthMiddleware.Wrap(deletion.TenantMiddleware(t.Overrides, h))
}
func (t *Loki) initIndexGateway() (services.Service, error) {
t.Cfg.IndexGateway.Ring.ListenPort = t.Cfg.Server.GRPCListenPort
func (t *Loki) initBloomGateway() (services.Service, error) {
logger := log.With(util_log.Logger, "component", "bloom-gateway")
instanceAddr := t.bloomGatewayRingManager.RingLifecycler.GetInstanceAddr()
instanceID := t.bloomGatewayRingManager.RingLifecycler.GetInstanceID()
shuffleSharding := bloomgateway.NewShuffleShardingStrategy(t.bloomGatewayRingManager.Ring, t.Overrides, instanceAddr, instanceID, logger)
gateway, err := bloomgateway.New(t.Cfg.BloomGateway, t.Cfg.SchemaConfig, t.Cfg.StorageConfig, shuffleSharding, t.clientMetrics, logger, prometheus.DefaultRegisterer)
if err != nil {
return nil, err
}
logproto.RegisterBloomGatewayServer(t.Server.GRPC, gateway)
return gateway, nil
}
func (t *Loki) initBloomGatewayRing() (services.Service, error) {
// Inherit ring listen port from gRPC config
t.Cfg.BloomGateway.Ring.ListenPort = t.Cfg.Server.GRPCListenPort
// TODO(chaudum): Do we want to integration the bloom gateway component into the backend target?
mode := bloomgateway.ClientMode
legacyReadMode := t.Cfg.LegacyReadTarget && t.isModuleActive(Read)
if t.Cfg.isModuleEnabled(BloomGateway) || t.Cfg.isModuleEnabled(Backend) || legacyReadMode {
mode = bloomgateway.ServerMode
}
manager, err := bloomgateway.NewRingManager(mode, t.Cfg.BloomGateway, util_log.Logger, prometheus.DefaultRegisterer)
if err != nil {
return nil, gerrors.Wrap(err, "error initializing bloom gateway ring manager")
}
t.bloomGatewayRingManager = manager
t.Server.HTTP.Path("/bloomgateway/ring").Methods("GET", "POST").Handler(t.bloomGatewayRingManager)
if t.Cfg.InternalServer.Enable {
t.InternalServer.HTTP.Path("/bloomgateway/ring").Methods("GET", "POST").Handler(t.bloomGatewayRingManager)
}
return t.bloomGatewayRingManager, nil
}
func (t *Loki) initIndexGateway() (services.Service, error) {
shardingStrategy := indexgateway.GetShardingStrategy(t.Cfg.IndexGateway, t.indexGatewayRingManager, t.Overrides)
var indexClients []indexgateway.IndexClientWithRange
@ -1221,7 +1265,18 @@ func (t *Loki) initIndexGateway() (services.Service, error) {
})
}
gateway, err := indexgateway.NewIndexGateway(t.Cfg.IndexGateway, util_log.Logger, prometheus.DefaultRegisterer, t.Store, indexClients)
logger := log.With(util_log.Logger, "component", "index-gateway")
var bloomQuerier indexgateway.BloomQuerier
if t.Cfg.BloomGateway.Enabled {
bloomGatewayClient, err := bloomgateway.NewGatewayClient(t.Cfg.BloomGateway.Client, t.Overrides, prometheus.DefaultRegisterer, logger)
if err != nil {
return nil, err
}
bloomQuerier = bloomgateway.NewBloomQuerier(bloomGatewayClient, logger)
}
gateway, err := indexgateway.NewIndexGateway(t.Cfg.IndexGateway, logger, prometheus.DefaultRegisterer, t.Store, indexClients, bloomQuerier)
if err != nil {
return nil, err
}
@ -1231,6 +1286,9 @@ func (t *Loki) initIndexGateway() (services.Service, error) {
}
func (t *Loki) initIndexGatewayRing() (_ services.Service, err error) {
// Inherit ring listen port from gRPC config
t.Cfg.IndexGateway.Ring.ListenPort = t.Cfg.Server.GRPCListenPort
// IndexGateway runs by default on legacy read and backend targets, and should always assume
// ring mode when run in this way.
legacyReadMode := t.Cfg.LegacyReadTarget && t.isModuleActive(Read)
@ -1244,7 +1302,6 @@ func (t *Loki) initIndexGatewayRing() (_ services.Service, err error) {
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly
t.Cfg.IndexGateway.Ring.ListenPort = t.Cfg.Server.GRPCListenPort
managerMode := indexgateway.ClientMode
if t.Cfg.isModuleEnabled(IndexGateway) || legacyReadMode || t.Cfg.isModuleEnabled(Backend) {

@ -54,7 +54,7 @@ func NewIngesterQuerier(clientCfg client.Config, ring ring.ReadRing, extraQueryD
func newIngesterQuerier(clientCfg client.Config, ring ring.ReadRing, extraQueryDelay time.Duration, clientFactory ring_client.PoolFactory) (*IngesterQuerier, error) {
iq := IngesterQuerier{
ring: ring,
pool: clientpool.NewPool(clientCfg.PoolConfig, ring, clientFactory, util_log.Logger),
pool: clientpool.NewPool("ingester", clientCfg.PoolConfig, ring, clientFactory, util_log.Logger),
extraQueryDelay: extraQueryDelay,
}

@ -32,7 +32,7 @@ import (
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig"
bloomshipperconfig "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/boltdb"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/downloads"

@ -29,10 +29,19 @@ func NewIndexGatewayClientStore(client logproto.IndexGatewayClient, logger log.L
}
func (c *IndexGatewayClientStore) GetChunkRefs(ctx context.Context, _ string, from, through model.Time, allMatchers ...*labels.Matcher) ([]logproto.ChunkRef, error) {
return c.GetChunkRefsFiltered(ctx, "", from, through, nil, allMatchers...)
}
func (c *IndexGatewayClientStore) GetChunkRefsFiltered(ctx context.Context, _ string, from, through model.Time, filters []syntax.LineFilterExpr, allMatchers ...*labels.Matcher) ([]logproto.ChunkRef, error) {
lineFilters := make([]*logproto.LineFilterExpression, 0, len(filters))
for _, filter := range filters {
lineFilters = append(lineFilters, &logproto.LineFilterExpression{Operator: int64(filter.Ty), Match: filter.Match})
}
response, err := c.client.GetChunkRef(ctx, &logproto.GetChunkRefRequest{
From: from,
Through: through,
Matchers: (&syntax.MatchersExpr{Mts: allMatchers}).String(),
Filters: lineFilters,
})
if err != nil {
return nil, err

@ -68,7 +68,7 @@ type MetaClient interface {
// Returns all metas that are within MinFingerprint-MaxFingerprint fingerprint range
// and intersect time period from StartTimestamp to EndTimestamp.
GetMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error)
PutMeta(ctx context.Context, meta Meta) (Meta, error)
PutMeta(ctx context.Context, meta Meta) error
DeleteMeta(ctx context.Context, meta Meta) error
}

@ -1,4 +1,5 @@
package bloomshipperconfig
// package bloomshipperconfig resides in its own package to prevent circular imports with storage package
package config
import (
"errors"

@ -1,6 +1,7 @@
package bloomshipper
import (
"cmp"
"context"
"fmt"
"io"
@ -10,32 +11,38 @@ import (
"strings"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"golang.org/x/exp/slices"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config"
)
type Shipper struct {
client Client
config bloomshipperconfig.Config
config config.Config
logger log.Logger
}
func NewShipper(client Client, config bloomshipperconfig.Config) (*Shipper, error) {
func NewShipper(client Client, config config.Config, logger log.Logger) (*Shipper, error) {
return &Shipper{
client: client,
config: config,
logger: log.With(logger, "component", "bloom-shipper"),
}, nil
}
func (s *Shipper) ForEachBlock(
ctx context.Context,
tenantID string,
startTimestamp, endTimestamp int64,
minFingerprint, maxFingerprint uint64,
callback func(bq *v1.BlockQuerier) error) error {
from, through time.Time,
fingerprints []uint64,
callback ForEachBlockCallback) error {
blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, startTimestamp, endTimestamp, minFingerprint, maxFingerprint)
level.Debug(s.logger).Log("msg", "ForEachBlock", "tenant", tenantID, "from", from, "through", through, "fingerprints", fingerprints)
blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from.UnixNano(), through.UnixNano(), fingerprints)
if err != nil {
return fmt.Errorf("error fetching active block references : %w", err)
}
@ -64,22 +71,37 @@ func (s *Shipper) ForEachBlock(
}
}
func (s *Shipper) Stop() {
s.client.Stop()
}
// getFromThrough returns the first and list item of a fingerprint slice
// It assumes an ascending sorted list of fingerprints.
func getFromThrough(fingerprints []uint64) (uint64, uint64) {
if len(fingerprints) == 0 {
return 0, 0
}
return fingerprints[0], fingerprints[len(fingerprints)-1]
}
func (s *Shipper) getActiveBlockRefs(
ctx context.Context,
tenantID string,
startTimestamp, endTimestamp int64,
minFingerprint, maxFingerprint uint64) ([]BlockRef, error) {
from, through int64,
fingerprints []uint64) ([]BlockRef, error) {
minFingerprint, maxFingerprint := getFromThrough(fingerprints)
metas, err := s.client.GetMetas(ctx, MetaSearchParams{
TenantID: tenantID,
MinFingerprint: minFingerprint,
MaxFingerprint: maxFingerprint,
StartTimestamp: startTimestamp,
EndTimestamp: endTimestamp,
StartTimestamp: from,
EndTimestamp: through,
})
if err != nil {
return []BlockRef{}, fmt.Errorf("error fetching meta.json files: %w", err)
}
activeBlocks := s.findBlocks(metas, minFingerprint, maxFingerprint, startTimestamp, endTimestamp)
activeBlocks := s.findBlocks(metas, from, through, fingerprints)
slices.SortStableFunc(activeBlocks, func(a, b BlockRef) int {
if a.MinFingerprint < b.MinFingerprint {
return -1
@ -93,13 +115,7 @@ func (s *Shipper) getActiveBlockRefs(
return activeBlocks, nil
}
func (s *Shipper) findBlocks(
metas []Meta,
minFingerprint uint64,
maxFingerprint uint64,
startTimestamp int64,
endTimestamp int64,
) []BlockRef {
func (s *Shipper) findBlocks(metas []Meta, startTimestamp, endTimestamp int64, fingerprints []uint64) []BlockRef {
outdatedBlocks := make(map[string]interface{})
for _, meta := range metas {
for _, tombstone := range meta.Tombstones {
@ -112,7 +128,7 @@ func (s *Shipper) findBlocks(
if _, contains := outdatedBlocks[block.BlockPath]; contains {
continue
}
if isOutsideRange(&block, minFingerprint, maxFingerprint, startTimestamp, endTimestamp) {
if isOutsideRange(&block, startTimestamp, endTimestamp, fingerprints) {
continue
}
blocksSet[block.BlockPath] = block
@ -125,15 +141,41 @@ func (s *Shipper) findBlocks(
return blockRefs
}
func isOutsideRange(
b *BlockRef,
minFingerprint uint64,
maxFingerprint uint64,
startTimestamp int64,
endTimestamp int64,
) bool {
return b.MaxFingerprint < minFingerprint || b.MinFingerprint > maxFingerprint ||
b.EndTimestamp < startTimestamp || b.StartTimestamp > endTimestamp
// getPosition returns the smallest index of element v in slice s where v > s[i]
// TODO(chaudum): Use binary search to find index instead of iteration.
func getPosition[S ~[]E, E cmp.Ordered](s S, v E) int {
for i := range s {
if v > s[i] {
continue
}
return i
}
return len(s)
}
func isOutsideRange(b *BlockRef, startTimestamp, endTimestamp int64, fingerprints []uint64) bool {
// First, check time range
if b.EndTimestamp < startTimestamp || b.StartTimestamp > endTimestamp {
return true
}
// Then, check if outside of min/max of fingerprint slice
minFp, maxFp := getFromThrough(fingerprints)
if b.MaxFingerprint < minFp || b.MinFingerprint > maxFp {
return true
}
// Check if the block range is inside a "gap" in the fingerprint slice
// e.g.
// fingerprints = [1, 2, 6, 7, 8]
// block = [3, 4, 5]
idx := getPosition[[]uint64](fingerprints, b.MinFingerprint)
// in case b.MinFingerprint is outside of the fingerprints range, return true
// this is already covered in the range check above, but I keep it as a second gate
if idx > len(fingerprints)-1 {
return true
}
return b.MaxFingerprint < fingerprints[idx]
}
// extract the files into directory and returns absolute path to this directory.

@ -2,6 +2,7 @@ package bloomshipper
import (
"bytes"
"fmt"
"io"
"math"
"os"
@ -14,7 +15,7 @@ import (
"github.com/stretchr/testify/require"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config"
)
func Test_Shipper_findBlocks(t *testing.T) {
@ -48,7 +49,7 @@ func Test_Shipper_findBlocks(t *testing.T) {
}
shipper := &Shipper{}
blocks := shipper.findBlocks(metas, 100, 200, 300, 400)
blocks := shipper.findBlocks(metas, 300, 400, []uint64{100, 200})
expectedBlockRefs := []BlockRef{
createMatchingBlockRef("block2"),
@ -102,7 +103,7 @@ func Test_Shipper_findBlocks(t *testing.T) {
t.Run(name, func(t *testing.T) {
shipper := &Shipper{}
ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp)
blocks := shipper.findBlocks([]Meta{{Blocks: []BlockRef{ref}}}, 100, 200, 300, 400)
blocks := shipper.findBlocks([]Meta{{Blocks: []BlockRef{ref}}}, 300, 400, []uint64{100, 110, 120, 130, 140, 150, 160, 170, 180, 190, 200})
if data.filtered {
require.Empty(t, blocks)
return
@ -113,16 +114,85 @@ func Test_Shipper_findBlocks(t *testing.T) {
}
}
func TestGetPosition(t *testing.T) {
for i, tc := range []struct {
s []int
v int
exp int
}{
{s: []int{}, v: 1, exp: 0},
{s: []int{1, 2, 3}, v: 0, exp: 0},
{s: []int{1, 2, 3}, v: 2, exp: 1},
{s: []int{1, 2, 3}, v: 4, exp: 3},
{s: []int{1, 2, 4, 5}, v: 3, exp: 2},
} {
tc := tc
name := fmt.Sprintf("case-%d", i)
t.Run(name, func(t *testing.T) {
got := getPosition[[]int](tc.s, tc.v)
require.Equal(t, tc.exp, got)
})
}
}
func TestIsOutsideRange(t *testing.T) {
t.Run("is outside if startTs > through", func(t *testing.T) {
b := createBlockRef("block", 0, math.MaxUint64, 100, 200)
isOutside := isOutsideRange(&b, 0, 90, []uint64{})
require.True(t, isOutside)
})
t.Run("is outside if endTs < from", func(t *testing.T) {
b := createBlockRef("block", 0, math.MaxUint64, 100, 200)
isOutside := isOutsideRange(&b, 210, 300, []uint64{})
require.True(t, isOutside)
})
t.Run("is outside if endFp < first fingerprint", func(t *testing.T) {
b := createBlockRef("block", 0, 90, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{100, 200})
require.True(t, isOutside)
})
t.Run("is outside if startFp > last fingerprint", func(t *testing.T) {
b := createBlockRef("block", 210, math.MaxUint64, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{100, 200})
require.True(t, isOutside)
})
t.Run("is outside if within gaps in fingerprints", func(t *testing.T) {
b := createBlockRef("block", 100, 200, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{0, 99, 201, 300})
require.True(t, isOutside)
})
t.Run("is not outside if within fingerprints 1", func(t *testing.T) {
b := createBlockRef("block", 100, 200, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{0, 100, 200, 300})
require.False(t, isOutside)
})
t.Run("is not outside if within fingerprints 2", func(t *testing.T) {
b := createBlockRef("block", 100, 150, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{0, 100, 200, 300})
require.False(t, isOutside)
})
t.Run("is not outside if within fingerprints 3", func(t *testing.T) {
b := createBlockRef("block", 150, 200, 100, 200)
isOutside := isOutsideRange(&b, 100, 200, []uint64{0, 100, 200, 300})
require.False(t, isOutside)
})
}
func createMatchingBlockRef(blockPath string) BlockRef {
return createBlockRef(blockPath, 0, uint64(math.MaxUint64), 0, math.MaxInt)
}
func createBlockRef(
blockPath string,
minFingerprint uint64,
maxFingerprint uint64,
startTimestamp int64,
endTimestamp int64,
minFingerprint, maxFingerprint uint64,
startTimestamp, endTimestamp int64,
) BlockRef {
return BlockRef{
Ref: Ref{
@ -172,7 +242,7 @@ func Test_Shipper_extractBlock(t *testing.T) {
require.NoError(t, err)
workingDir := t.TempDir()
shipper := Shipper{config: bloomshipperconfig.Config{WorkingDirectory: workingDir}}
shipper := Shipper{config: config.Config{WorkingDirectory: workingDir}}
ts := time.Now().UTC()
block := Block{
BlockRef: BlockRef{BlockPath: "first-period-19621/tenantA/metas/ff-fff-1695272400-1695276000-aaa"},

@ -0,0 +1,70 @@
package bloomshipper
import (
"context"
"time"
"github.com/grafana/loki/pkg/logproto"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
)
type ForEachBlockCallback func(bq *v1.BlockQuerier) error
type ReadShipper interface {
ForEachBlock(ctx context.Context, tenant string, from, through time.Time, fingerprints []uint64, callback ForEachBlockCallback) error
}
type Interface interface {
ReadShipper
Stop()
}
type Store interface {
FilterChunkRefs(ctx context.Context, tenant string, from, through time.Time, chunkRefs []*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]*logproto.ChunkRef, error)
Stop()
}
type BloomStore struct {
shipper Interface
}
func NewBloomStore(shipper Interface) (*BloomStore, error) {
return &BloomStore{
shipper: shipper,
}, nil
}
func (bs *BloomStore) Stop() {
bs.shipper.Stop()
}
func (bs *BloomStore) FilterChunkRefs(ctx context.Context, tenant string, from, through time.Time, chunkRefs []*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]*logproto.ChunkRef, error) {
fingerprints := make([]uint64, 0, len(chunkRefs))
for _, ref := range chunkRefs {
fingerprints = append(fingerprints, ref.Fingerprint)
}
blooms, err := bs.blooms(ctx, tenant, from, through, fingerprints)
if err != nil {
return nil, err
}
return blooms.FilterChunkRefs(ctx, tenant, from, through, chunkRefs, filters...)
}
func (bs *BloomStore) blooms(ctx context.Context, tenant string, from, through time.Time, fingerprints []uint64) (*bloomFilters, error) {
bf := &bloomFilters{}
err := bs.shipper.ForEachBlock(ctx, tenant, from, through, fingerprints, func(bq *v1.BlockQuerier) error {
return nil
})
return bf, err
}
type bloomFilters struct {
}
func newBloomFilters(size int) *bloomFilters {
return &bloomFilters{}
}
func (bf *bloomFilters) FilterChunkRefs(ctx context.Context, tenant string, from, through time.Time, chunkRefs []*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]*logproto.ChunkRef, error) {
return nil, nil
}

@ -0,0 +1,11 @@
package bloomshipper
import (
"testing"
)
func TestBloomShipper(t *testing.T) {
}
func TestBloomStore(t *testing.T) {
}

@ -155,7 +155,7 @@ func NewGatewayClient(cfg IndexGatewayClientConfig, r prometheus.Registerer, lim
sgClient.cfg.PoolConfig.HealthCheckIngesters = true
if sgClient.cfg.Mode == indexgateway.RingMode {
sgClient.pool = clientpool.NewPool(sgClient.cfg.PoolConfig, sgClient.ring, client.PoolAddrFunc(factory), logger)
sgClient.pool = clientpool.NewPool("index-gateway", sgClient.cfg.PoolConfig, sgClient.ring, client.PoolAddrFunc(factory), logger)
} else {
// Note we don't use clientpool.NewPool because we want to provide our own discovery function
poolCfg := client.PoolConfig{

@ -14,6 +14,7 @@ import (
"github.com/grafana/dskit/tenant"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/grafana/loki/pkg/logproto"
@ -46,11 +47,16 @@ type IndexClientWithRange struct {
TableRange config.TableRange
}
type BloomQuerier interface {
FilterChunkRefs(ctx context.Context, tenant string, from, through model.Time, chunks []*logproto.ChunkRef, filters ...*logproto.LineFilterExpression) ([]*logproto.ChunkRef, error)
}
type Gateway struct {
services.Service
indexQuerier IndexQuerier
indexClients []IndexClientWithRange
bloomQuerier BloomQuerier
cfg Config
log log.Logger
@ -60,9 +66,10 @@ type Gateway struct {
//
// In case it is configured to be in ring mode, a Basic Service wrapping the ring client is started.
// Otherwise, it starts an Idle Service that doesn't have lifecycle hooks.
func NewIndexGateway(cfg Config, log log.Logger, _ prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange) (*Gateway, error) {
func NewIndexGateway(cfg Config, log log.Logger, _ prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange, bloomQuerier BloomQuerier) (*Gateway, error) {
g := &Gateway{
indexQuerier: indexQuerier,
bloomQuerier: bloomQuerier,
cfg: cfg,
log: log,
indexClients: indexClients,
@ -199,6 +206,7 @@ func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequ
if err != nil {
return nil, err
}
result := &logproto.GetChunkRefResponse{
Refs: make([]*logproto.ChunkRef, 0, len(chunks)),
}
@ -207,6 +215,22 @@ func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequ
result.Refs = append(result.Refs, &cs[i].ChunkRef)
}
}
// Return unfiltered results if there is no bloom querier (Bloom Gateway disabled) or if there are not filters.
if g.bloomQuerier == nil || len(req.Filters) == 0 {
return result, nil
}
// TODO(chaudum): Take the chunks from the index querier's GetChunks()
// response and send them to the bloom gateway along with the filter
// expression that we got from the request object.
// The bloom gateway returns the list of matching ChunkRefs.
chunkRefs, err := g.bloomQuerier.FilterChunkRefs(ctx, instanceID, req.From, req.Through, result.Refs, req.Filters...)
if err != nil {
return nil, err
}
result.Refs = chunkRefs
return result, nil
}

@ -225,7 +225,7 @@ func TestGateway_QueryIndex_multistore(t *testing.T) {
},
},
}}
gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, nil, indexClients)
gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, nil, indexClients, nil)
require.NoError(t, err)
expectedQueries = append(expectedQueries,
@ -250,7 +250,7 @@ func TestVolume(t *testing.T) {
{Name: "bar", Volume: 38},
}}, nil)
gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, indexQuerier, nil)
gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, indexQuerier, nil, nil)
require.NoError(t, err)
ctx := user.InjectOrgID(context.Background(), "test")

@ -1,6 +1,7 @@
package limiter
import (
"github.com/grafana/loki/pkg/bloomgateway"
"github.com/grafana/loki/pkg/compactor"
"github.com/grafana/loki/pkg/distributor"
"github.com/grafana/loki/pkg/ingester"
@ -22,4 +23,5 @@ type CombinedLimits interface {
scheduler.Limits
storage.StoreLimits
indexgateway.Limits
bloomgateway.Limits
}

@ -182,6 +182,7 @@ type Limits struct {
RequiredNumberLabels int `yaml:"minimum_labels_number,omitempty" json:"minimum_labels_number,omitempty" doc:"description=Minimum number of label matchers a query should contain."`
IndexGatewayShardSize int `yaml:"index_gateway_shard_size" json:"index_gateway_shard_size"`
BloomGatewayShardSize int `yaml:"bloom_gateway_shard_size" json:"bloom_gateway_shard_size"`
AllowStructuredMetadata bool `yaml:"allow_structured_metadata,omitempty" json:"allow_structured_metadata,omitempty" doc:"description=Allow user to send structured metadata in push payload."`
MaxStructuredMetadataSize flagext.ByteSize `yaml:"max_structured_metadata_size" json:"max_structured_metadata_size" doc:"description=Maximum size accepted for structured metadata per log line."`
@ -295,6 +296,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) {
dskit_flagext.DeprecatedFlag(f, "compactor.allow-deletes", "Deprecated. Instead, see compactor.deletion-mode which is another per tenant configuration", util_log.Logger)
f.IntVar(&l.IndexGatewayShardSize, "index-gateway.shard-size", 0, "The shard size defines how many index gateways should be used by a tenant for querying. If the global shard factor is 0, the global shard factor is set to the deprecated -replication-factor for backwards compatibility reasons.")
f.IntVar(&l.BloomGatewayShardSize, "bloom-gateway.shard-size", 1, "The shard size defines how many bloom gateways should be used by a tenant for querying.")
l.ShardStreams = &shardstreams.Config{}
l.ShardStreams.RegisterFlagsWithPrefix("shard-streams", f)
@ -781,6 +783,10 @@ func (o *Overrides) IndexGatewayShardSize(userID string) int {
return o.getOverridesForUser(userID).IndexGatewayShardSize
}
func (o *Overrides) BloomGatewayShardSize(userID string) int {
return o.getOverridesForUser(userID).BloomGatewayShardSize
}
func (o *Overrides) AllowStructuredMetadata(userID string) bool {
return o.getOverridesForUser(userID).AllowStructuredMetadata
}

@ -14,6 +14,7 @@ import (
"github.com/grafana/dskit/server"
"github.com/grafana/loki/pkg/analytics"
"github.com/grafana/loki/pkg/bloomgateway"
"github.com/grafana/loki/pkg/compactor"
"github.com/grafana/loki/pkg/distributor"
"github.com/grafana/loki/pkg/ingester"
@ -97,6 +98,11 @@ var (
StructType: []reflect.Type{reflect.TypeOf(indexgateway.Config{})},
Desc: "The index_gateway block configures the Loki index gateway server, responsible for serving index queries without the need to constantly interact with the object store.",
},
{
Name: "bloom_gateway",
StructType: []reflect.Type{reflect.TypeOf(bloomgateway.Config{})},
Desc: "The bloom_gateway block configures the Loki bloom gateway server, responsible for serving queries for filtering chunks based on filter expressions.",
},
{
Name: "storage_config",
StructType: []reflect.Type{reflect.TypeOf(storage.Config{})},

Loading…
Cancel
Save