From b49b3ce6e132801f2dcac8243cd55b47f6255d16 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Mon, 16 Oct 2023 16:00:00 +0200 Subject: [PATCH] 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 --- docs/sources/configure/_index.md | 131 ++ pkg/bloomgateway/bloomgateway.go | 162 ++ pkg/bloomgateway/bloomgateway_test.go | 193 +++ pkg/bloomgateway/client.go | 310 ++++ pkg/bloomgateway/client_test.go | 121 ++ pkg/bloomgateway/config.go | 47 + pkg/bloomgateway/querier.go | 64 + pkg/bloomgateway/querier_test.go | 86 + pkg/bloomgateway/ringmanager.go | 264 +++ pkg/bloomgateway/sharding.go | 162 ++ .../clientpool/ingester_client_pool.go | 12 +- pkg/distributor/distributor.go | 3 +- pkg/ingester/client/client.go | 2 +- pkg/logproto/bloomgateway.pb.go | 1519 +++++++++++++++++ pkg/logproto/bloomgateway.proto | 47 + pkg/logproto/logproto.pb.go | 639 +++++-- pkg/logproto/logproto.proto | 6 + pkg/loki/config_wrapper.go | 24 + pkg/loki/loki.go | 26 +- pkg/loki/modules.go | 65 +- pkg/querier/ingester_querier.go | 2 +- pkg/storage/factory.go | 2 +- .../series/series_index_gateway_store.go | 9 + .../stores/shipper/bloomshipper/client.go | 2 +- .../{bloomshipperconfig => config}/config.go | 3 +- .../stores/shipper/bloomshipper/shipper.go | 100 +- .../shipper/bloomshipper/shipper_test.go | 86 +- .../stores/shipper/bloomshipper/store.go | 70 + .../stores/shipper/bloomshipper/store_test.go | 11 + .../gatewayclient/gateway_client.go | 2 +- .../indexshipper/indexgateway/gateway.go | 26 +- .../indexshipper/indexgateway/gateway_test.go | 4 +- pkg/util/limiter/combined_limits.go | 2 + pkg/validation/limits.go | 6 + tools/doc-generator/parse/root_blocks.go | 6 + 35 files changed, 4000 insertions(+), 214 deletions(-) create mode 100644 pkg/bloomgateway/bloomgateway.go create mode 100644 pkg/bloomgateway/bloomgateway_test.go create mode 100644 pkg/bloomgateway/client.go create mode 100644 pkg/bloomgateway/client_test.go create mode 100644 pkg/bloomgateway/config.go create mode 100644 pkg/bloomgateway/querier.go create mode 100644 pkg/bloomgateway/querier_test.go create mode 100644 pkg/bloomgateway/ringmanager.go create mode 100644 pkg/bloomgateway/sharding.go create mode 100644 pkg/logproto/bloomgateway.pb.go create mode 100644 pkg/logproto/bloomgateway.proto rename pkg/storage/stores/shipper/bloomshipper/{bloomshipperconfig => config}/config.go (80%) create mode 100644 pkg/storage/stores/shipper/bloomshipper/store.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/store_test.go diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 58fc91867c3a3..9350d5adba659 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -161,6 +161,10 @@ Pass the `-config.expand-env` flag at the command line to enable this way of set # object store. [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: ] + # 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: | 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: | default = "consul"] + + # The prefix for the keys in the store. Should end with a /. + # CLI flag: -bloom-gateway.ring.prefix + [prefix: | 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: ] + + # 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: ] + + multi: + # Primary backend storage used by multi-client. + # CLI flag: -bloom-gateway.ring.multi.primary + [primary: | default = ""] + + # Secondary backend storage used by multi-client. + # CLI flag: -bloom-gateway.ring.multi.secondary + [secondary: | default = ""] + + # Mirror writes to secondary store. + # CLI flag: -bloom-gateway.ring.multi.mirror-enabled + [mirror_enabled: | default = false] + + # Timeout for storing value to secondary store. + # CLI flag: -bloom-gateway.ring.multi.mirror-timeout + [mirror_timeout: | default = 2s] + + # Period at which to heartbeat to the ring. 0 = disabled. + # CLI flag: -bloom-gateway.ring.heartbeat-period + [heartbeat_period: | 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: | 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: | default = ""] + + # True to enable zone-awareness and replicate blocks across different + # availability zones. + # CLI flag: -bloom-gateway.ring.zone-awareness-enabled + [zone_awareness_enabled: | default = false] + + # Instance ID to register in the ring. + # CLI flag: -bloom-gateway.ring.instance-id + [instance_id: | default = ""] + + # Name of network interface to read address from. + # CLI flag: -bloom-gateway.ring.instance-interface-names + [instance_interface_names: | default = []] + + # Port to advertise in the ring (defaults to server.grpc-listen-port). + # CLI flag: -bloom-gateway.ring.instance-port + [instance_port: | default = 0] + + # IP address to advertise in the ring. + # CLI flag: -bloom-gateway.ring.instance-addr + [instance_addr: | 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: | default = ""] + + # Enable using a IPv6 instance address. + # CLI flag: -bloom-gateway.ring.instance-enable-ipv6 + [instance_enable_ipv6: | default = false] + + # Factor for data replication on the bloom gateways. + # CLI flag: -bloom-gateway.replication-factor + [replication_factor: | default = 3] + +# Flag to enable or disable the usage of the bloom gatway component. +# CLI flag: -bloom-gateway.enabled +[enabled: | default = false] + +client: + # Configures the behavior of the connection pool. + pool_config: + [client_cleanup_period: ] + + [health_check_ingesters: ] + + [remote_timeout: ] + + # 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: ] + + # 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: | 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: | 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: | default = 1] + # Allow user to send structured metadata in push payload. # CLI flag: -validation.allow-structured-metadata [allow_structured_metadata: | default = false] @@ -3362,6 +3490,7 @@ ring: Configuration for a Consul client. Only applies if the selected kvstore is `consul`. The supported CLI flags `` 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 `` 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 `` used to reference this configuration block are: - `bigtable` +- `bloom-gateway-client.grpc` - `boltdb.shipper.index-gateway-client.grpc` - `frontend.grpc-client-config` - `ingester.client` diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go new file mode 100644 index 0000000000000..39937fda67db5 --- /dev/null +++ b/pkg/bloomgateway/bloomgateway.go @@ -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 +} diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go new file mode 100644 index 0000000000000..58d178b21c0c0 --- /dev/null +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -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()) + }) + +} diff --git a/pkg/bloomgateway/client.go b/pkg/bloomgateway/client.go new file mode 100644 index 0000000000000..5ee8c910e7da4 --- /dev/null +++ b/pkg/bloomgateway/client.go @@ -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 +} diff --git a/pkg/bloomgateway/client_test.go b/pkg/bloomgateway/client_test.go new file mode 100644 index 0000000000000..3d56eb367f6bf --- /dev/null +++ b/pkg/bloomgateway/client_test.go @@ -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) + }) + } +} diff --git a/pkg/bloomgateway/config.go b/pkg/bloomgateway/config.go new file mode 100644 index 0000000000000..226ca8c7cefa3 --- /dev/null +++ b/pkg/bloomgateway/config.go @@ -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) +} diff --git a/pkg/bloomgateway/querier.go b/pkg/bloomgateway/querier.go new file mode 100644 index 0000000000000..6ae19aeeb72d9 --- /dev/null +++ b/pkg/bloomgateway/querier.go @@ -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 +} diff --git a/pkg/bloomgateway/querier_test.go b/pkg/bloomgateway/querier_test.go new file mode 100644 index 0000000000000..e1aaeb449ca71 --- /dev/null +++ b/pkg/bloomgateway/querier_test.go @@ -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) + }) +} diff --git a/pkg/bloomgateway/ringmanager.go b/pkg/bloomgateway/ringmanager.go new file mode 100644 index 0000000000000..415d89f292488 --- /dev/null +++ b/pkg/bloomgateway/ringmanager.go @@ -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) { +} diff --git a/pkg/bloomgateway/sharding.go b/pkg/bloomgateway/sharding.go new file mode 100644 index 0000000000000..c7bfdc7af92c4 --- /dev/null +++ b/pkg/bloomgateway/sharding.go @@ -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 +} diff --git a/pkg/distributor/clientpool/ingester_client_pool.go b/pkg/distributor/clientpool/ingester_client_pool.go index 9058b1b4c5ccf..3c4fa75401c95 100644 --- a/pkg/distributor/clientpool/ingester_client_pool.go +++ b/pkg/distributor/clientpool/ingester_client_pool.go @@ -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) } diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 969c35a8d55ed..bd79bde7e4607 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -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), diff --git a/pkg/ingester/client/client.go b/pkg/ingester/client/client.go index 51086ef8de391..fa14294911ba4 100644 --- a/pkg/ingester/client/client.go +++ b/pkg/ingester/client/client.go @@ -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.") diff --git a/pkg/logproto/bloomgateway.pb.go b/pkg/logproto/bloomgateway.pb.go new file mode 100644 index 0000000000000..0739d1e556000 --- /dev/null +++ b/pkg/logproto/bloomgateway.pb.go @@ -0,0 +1,1519 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/logproto/bloomgateway.proto + +package logproto + +import ( + context "context" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + github_com_prometheus_common_model "github.com/prometheus/common/model" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type FilterChunkRefRequest 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"` + Refs []*ChunkRef `protobuf:"bytes,3,rep,name=refs,proto3" json:"refs,omitempty"` + Filters []*LineFilterExpression `protobuf:"bytes,4,rep,name=filters,proto3" json:"filters,omitempty"` +} + +func (m *FilterChunkRefRequest) Reset() { *m = FilterChunkRefRequest{} } +func (*FilterChunkRefRequest) ProtoMessage() {} +func (*FilterChunkRefRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_a50b5dd1dbcd1415, []int{0} +} +func (m *FilterChunkRefRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FilterChunkRefRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_FilterChunkRefRequest.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 *FilterChunkRefRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_FilterChunkRefRequest.Merge(m, src) +} +func (m *FilterChunkRefRequest) XXX_Size() int { + return m.Size() +} +func (m *FilterChunkRefRequest) XXX_DiscardUnknown() { + xxx_messageInfo_FilterChunkRefRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_FilterChunkRefRequest proto.InternalMessageInfo + +func (m *FilterChunkRefRequest) GetRefs() []*ChunkRef { + if m != nil { + return m.Refs + } + return nil +} + +func (m *FilterChunkRefRequest) GetFilters() []*LineFilterExpression { + if m != nil { + return m.Filters + } + return nil +} + +type FilterChunkRefResponse struct { + ChunkRefs []*GroupedChunkRefs `protobuf:"bytes,1,rep,name=chunkRefs,proto3" json:"chunkRefs,omitempty"` +} + +func (m *FilterChunkRefResponse) Reset() { *m = FilterChunkRefResponse{} } +func (*FilterChunkRefResponse) ProtoMessage() {} +func (*FilterChunkRefResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_a50b5dd1dbcd1415, []int{1} +} +func (m *FilterChunkRefResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FilterChunkRefResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_FilterChunkRefResponse.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 *FilterChunkRefResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_FilterChunkRefResponse.Merge(m, src) +} +func (m *FilterChunkRefResponse) XXX_Size() int { + return m.Size() +} +func (m *FilterChunkRefResponse) XXX_DiscardUnknown() { + xxx_messageInfo_FilterChunkRefResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_FilterChunkRefResponse proto.InternalMessageInfo + +func (m *FilterChunkRefResponse) GetChunkRefs() []*GroupedChunkRefs { + if m != nil { + return m.ChunkRefs + } + return nil +} + +type ShortRef 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"` + Checksum uint32 `protobuf:"varint,3,opt,name=checksum,proto3" json:"checksum,omitempty"` +} + +func (m *ShortRef) Reset() { *m = ShortRef{} } +func (*ShortRef) ProtoMessage() {} +func (*ShortRef) Descriptor() ([]byte, []int) { + return fileDescriptor_a50b5dd1dbcd1415, []int{2} +} +func (m *ShortRef) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ShortRef) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ShortRef.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 *ShortRef) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShortRef.Merge(m, src) +} +func (m *ShortRef) XXX_Size() int { + return m.Size() +} +func (m *ShortRef) XXX_DiscardUnknown() { + xxx_messageInfo_ShortRef.DiscardUnknown(m) +} + +var xxx_messageInfo_ShortRef proto.InternalMessageInfo + +func (m *ShortRef) GetChecksum() uint32 { + if m != nil { + return m.Checksum + } + return 0 +} + +type GroupedChunkRefs struct { + Fingerprint uint64 `protobuf:"varint,1,opt,name=fingerprint,proto3" json:"fingerprint,omitempty"` + Tenant string `protobuf:"bytes,2,opt,name=tenant,proto3" json:"tenant,omitempty"` + Refs []*ShortRef `protobuf:"bytes,3,rep,name=refs,proto3" json:"refs,omitempty"` +} + +func (m *GroupedChunkRefs) Reset() { *m = GroupedChunkRefs{} } +func (*GroupedChunkRefs) ProtoMessage() {} +func (*GroupedChunkRefs) Descriptor() ([]byte, []int) { + return fileDescriptor_a50b5dd1dbcd1415, []int{3} +} +func (m *GroupedChunkRefs) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GroupedChunkRefs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_GroupedChunkRefs.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 *GroupedChunkRefs) XXX_Merge(src proto.Message) { + xxx_messageInfo_GroupedChunkRefs.Merge(m, src) +} +func (m *GroupedChunkRefs) XXX_Size() int { + return m.Size() +} +func (m *GroupedChunkRefs) XXX_DiscardUnknown() { + xxx_messageInfo_GroupedChunkRefs.DiscardUnknown(m) +} + +var xxx_messageInfo_GroupedChunkRefs proto.InternalMessageInfo + +func (m *GroupedChunkRefs) GetFingerprint() uint64 { + if m != nil { + return m.Fingerprint + } + return 0 +} + +func (m *GroupedChunkRefs) GetTenant() string { + if m != nil { + return m.Tenant + } + return "" +} + +func (m *GroupedChunkRefs) GetRefs() []*ShortRef { + if m != nil { + return m.Refs + } + return nil +} + +func init() { + proto.RegisterType((*FilterChunkRefRequest)(nil), "logproto.FilterChunkRefRequest") + proto.RegisterType((*FilterChunkRefResponse)(nil), "logproto.FilterChunkRefResponse") + proto.RegisterType((*ShortRef)(nil), "logproto.ShortRef") + proto.RegisterType((*GroupedChunkRefs)(nil), "logproto.GroupedChunkRefs") +} + +func init() { proto.RegisterFile("pkg/logproto/bloomgateway.proto", fileDescriptor_a50b5dd1dbcd1415) } + +var fileDescriptor_a50b5dd1dbcd1415 = []byte{ + // 467 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x53, 0x3d, 0x6f, 0xd4, 0x40, + 0x10, 0xf5, 0xe6, 0x4e, 0xc9, 0x65, 0x03, 0x02, 0xad, 0x20, 0xb2, 0x8c, 0xb4, 0x67, 0x59, 0x08, + 0xae, 0x3a, 0x4b, 0xa1, 0x49, 0x7d, 0x11, 0x44, 0x48, 0x54, 0x0b, 0xa2, 0x48, 0xe7, 0x73, 0xc6, + 0x1f, 0x3a, 0x7b, 0xc7, 0xec, 0xae, 0x05, 0x74, 0xb4, 0x74, 0xfc, 0x0c, 0x7e, 0x01, 0xbf, 0x21, + 0xe5, 0x95, 0x11, 0x45, 0xc4, 0xf9, 0x1a, 0xca, 0xfc, 0x04, 0x94, 0x75, 0x7c, 0x77, 0x89, 0x48, + 0x43, 0x45, 0xe5, 0xdd, 0x99, 0x37, 0x4f, 0xfb, 0xde, 0x1b, 0xd3, 0x61, 0x35, 0x4b, 0xc3, 0x02, + 0xd3, 0x4a, 0xa1, 0xc1, 0x70, 0x5a, 0x20, 0x96, 0x69, 0x64, 0xe0, 0x63, 0xf4, 0x79, 0x6c, 0x4b, + 0x6c, 0xd0, 0x35, 0xbd, 0x47, 0x29, 0xa6, 0xd8, 0xe2, 0xae, 0x4e, 0x6d, 0xdf, 0x7b, 0x72, 0x83, + 0xa0, 0x3b, 0xb4, 0xcd, 0xe0, 0xeb, 0x16, 0x7d, 0xfc, 0x2a, 0x2f, 0x0c, 0xa8, 0xa3, 0xac, 0x96, + 0x33, 0x01, 0x89, 0x80, 0x0f, 0x35, 0x68, 0xc3, 0x8e, 0x68, 0x3f, 0x51, 0x58, 0xba, 0xc4, 0x27, + 0xa3, 0xde, 0x24, 0x3c, 0xbb, 0x18, 0x3a, 0x3f, 0x2f, 0x86, 0xcf, 0xd3, 0xdc, 0x64, 0xf5, 0x74, + 0x1c, 0x63, 0x19, 0x56, 0x0a, 0x4b, 0x30, 0x19, 0xd4, 0x3a, 0x8c, 0xb1, 0x2c, 0x51, 0x86, 0x25, + 0x9e, 0x42, 0x31, 0x7e, 0x97, 0x97, 0x20, 0xec, 0x30, 0x7b, 0x4d, 0x77, 0x4c, 0xa6, 0xb0, 0x4e, + 0x33, 0x77, 0xeb, 0xdf, 0x78, 0xba, 0x79, 0xf6, 0x8c, 0xf6, 0x15, 0x24, 0xda, 0xed, 0xf9, 0xbd, + 0xd1, 0xde, 0x01, 0x1b, 0xaf, 0x84, 0xac, 0x1e, 0x6e, 0xfb, 0xec, 0x90, 0xee, 0x24, 0x56, 0x90, + 0x76, 0xfb, 0x16, 0xca, 0xd7, 0xd0, 0x37, 0xb9, 0x84, 0x56, 0xed, 0xcb, 0x4f, 0x95, 0x02, 0xad, + 0x73, 0x94, 0xa2, 0x83, 0x07, 0x82, 0xee, 0xdf, 0xb6, 0x42, 0x57, 0x28, 0x35, 0xb0, 0x43, 0xba, + 0x1b, 0x5f, 0xd7, 0xb4, 0x4b, 0x2c, 0xab, 0xb7, 0x66, 0x3d, 0x56, 0x58, 0x57, 0x70, 0xda, 0x4d, + 0x69, 0xb1, 0x06, 0x07, 0x3f, 0x08, 0x1d, 0xbc, 0xcd, 0x50, 0x19, 0x01, 0xc9, 0x7f, 0x67, 0xa9, + 0x47, 0x07, 0x71, 0x06, 0xf1, 0x4c, 0xd7, 0xa5, 0xdb, 0xf3, 0xc9, 0xe8, 0xbe, 0x58, 0xdd, 0x03, + 0x43, 0x1f, 0xde, 0xd6, 0xc5, 0x7c, 0xba, 0x97, 0xe4, 0x32, 0x05, 0x55, 0xa9, 0x5c, 0x1a, 0x2b, + 0xa3, 0x2f, 0x36, 0x4b, 0x6c, 0x9f, 0x6e, 0x1b, 0x90, 0x91, 0x34, 0xf6, 0x6d, 0xbb, 0xe2, 0xfa, + 0x76, 0x77, 0x78, 0x9d, 0x37, 0x6d, 0x78, 0x07, 0x09, 0xbd, 0x37, 0xb9, 0xda, 0xf0, 0xe3, 0x76, + 0xc3, 0xd9, 0x7b, 0xfa, 0xe0, 0x66, 0x24, 0x9a, 0x0d, 0xd7, 0xc3, 0x7f, 0x5d, 0x5c, 0xcf, 0xbf, + 0x1b, 0xd0, 0xc6, 0x19, 0x38, 0x93, 0x93, 0xf9, 0x82, 0x3b, 0xe7, 0x0b, 0xee, 0x5c, 0x2e, 0x38, + 0xf9, 0xd2, 0x70, 0xf2, 0xbd, 0xe1, 0xe4, 0xac, 0xe1, 0x64, 0xde, 0x70, 0xf2, 0xab, 0xe1, 0xe4, + 0x77, 0xc3, 0x9d, 0xcb, 0x86, 0x93, 0x6f, 0x4b, 0xee, 0xcc, 0x97, 0xdc, 0x39, 0x5f, 0x72, 0xe7, + 0xe4, 0xe9, 0x86, 0xc3, 0xa9, 0x8a, 0x92, 0x48, 0x46, 0x61, 0x81, 0xb3, 0x3c, 0xdc, 0xfc, 0xc3, + 0xa6, 0xdb, 0xf6, 0xf3, 0xe2, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe6, 0xf6, 0xab, 0x52, 0xb9, + 0x03, 0x00, 0x00, +} + +func (this *FilterChunkRefRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*FilterChunkRefRequest) + if !ok { + that2, ok := that.(FilterChunkRefRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.From.Equal(that1.From) { + return false + } + if !this.Through.Equal(that1.Through) { + return false + } + if len(this.Refs) != len(that1.Refs) { + return false + } + for i := range this.Refs { + if !this.Refs[i].Equal(that1.Refs[i]) { + 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 *FilterChunkRefResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*FilterChunkRefResponse) + if !ok { + that2, ok := that.(FilterChunkRefResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.ChunkRefs) != len(that1.ChunkRefs) { + return false + } + for i := range this.ChunkRefs { + if !this.ChunkRefs[i].Equal(that1.ChunkRefs[i]) { + return false + } + } + return true +} +func (this *ShortRef) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ShortRef) + if !ok { + that2, ok := that.(ShortRef) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.From.Equal(that1.From) { + return false + } + if !this.Through.Equal(that1.Through) { + return false + } + if this.Checksum != that1.Checksum { + return false + } + return true +} +func (this *GroupedChunkRefs) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*GroupedChunkRefs) + if !ok { + that2, ok := that.(GroupedChunkRefs) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Fingerprint != that1.Fingerprint { + return false + } + if this.Tenant != that1.Tenant { + return false + } + if len(this.Refs) != len(that1.Refs) { + return false + } + for i := range this.Refs { + if !this.Refs[i].Equal(that1.Refs[i]) { + return false + } + } + return true +} +func (this *FilterChunkRefRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&logproto.FilterChunkRefRequest{") + s = append(s, "From: "+fmt.Sprintf("%#v", this.From)+",\n") + s = append(s, "Through: "+fmt.Sprintf("%#v", this.Through)+",\n") + if this.Refs != nil { + s = append(s, "Refs: "+fmt.Sprintf("%#v", this.Refs)+",\n") + } + if this.Filters != nil { + s = append(s, "Filters: "+fmt.Sprintf("%#v", this.Filters)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *FilterChunkRefResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&logproto.FilterChunkRefResponse{") + if this.ChunkRefs != nil { + s = append(s, "ChunkRefs: "+fmt.Sprintf("%#v", this.ChunkRefs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ShortRef) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&logproto.ShortRef{") + s = append(s, "From: "+fmt.Sprintf("%#v", this.From)+",\n") + s = append(s, "Through: "+fmt.Sprintf("%#v", this.Through)+",\n") + s = append(s, "Checksum: "+fmt.Sprintf("%#v", this.Checksum)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *GroupedChunkRefs) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&logproto.GroupedChunkRefs{") + s = append(s, "Fingerprint: "+fmt.Sprintf("%#v", this.Fingerprint)+",\n") + s = append(s, "Tenant: "+fmt.Sprintf("%#v", this.Tenant)+",\n") + if this.Refs != nil { + s = append(s, "Refs: "+fmt.Sprintf("%#v", this.Refs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringBloomgateway(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// BloomGatewayClient is the client API for BloomGateway service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type BloomGatewayClient interface { + FilterChunkRefs(ctx context.Context, in *FilterChunkRefRequest, opts ...grpc.CallOption) (*FilterChunkRefResponse, error) +} + +type bloomGatewayClient struct { + cc *grpc.ClientConn +} + +func NewBloomGatewayClient(cc *grpc.ClientConn) BloomGatewayClient { + return &bloomGatewayClient{cc} +} + +func (c *bloomGatewayClient) FilterChunkRefs(ctx context.Context, in *FilterChunkRefRequest, opts ...grpc.CallOption) (*FilterChunkRefResponse, error) { + out := new(FilterChunkRefResponse) + err := c.cc.Invoke(ctx, "/logproto.BloomGateway/FilterChunkRefs", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// BloomGatewayServer is the server API for BloomGateway service. +type BloomGatewayServer interface { + FilterChunkRefs(context.Context, *FilterChunkRefRequest) (*FilterChunkRefResponse, error) +} + +// UnimplementedBloomGatewayServer can be embedded to have forward compatible implementations. +type UnimplementedBloomGatewayServer struct { +} + +func (*UnimplementedBloomGatewayServer) FilterChunkRefs(ctx context.Context, req *FilterChunkRefRequest) (*FilterChunkRefResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method FilterChunkRefs not implemented") +} + +func RegisterBloomGatewayServer(s *grpc.Server, srv BloomGatewayServer) { + s.RegisterService(&_BloomGateway_serviceDesc, srv) +} + +func _BloomGateway_FilterChunkRefs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FilterChunkRefRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(BloomGatewayServer).FilterChunkRefs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/logproto.BloomGateway/FilterChunkRefs", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(BloomGatewayServer).FilterChunkRefs(ctx, req.(*FilterChunkRefRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _BloomGateway_serviceDesc = grpc.ServiceDesc{ + ServiceName: "logproto.BloomGateway", + HandlerType: (*BloomGatewayServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "FilterChunkRefs", + Handler: _BloomGateway_FilterChunkRefs_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/logproto/bloomgateway.proto", +} + +func (m *FilterChunkRefRequest) 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 *FilterChunkRefRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *FilterChunkRefRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = 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 = encodeVarintBloomgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if len(m.Refs) > 0 { + for iNdEx := len(m.Refs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Refs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBloomgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.Through != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.Through)) + i-- + dAtA[i] = 0x10 + } + if m.From != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.From)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *FilterChunkRefResponse) 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 *FilterChunkRefResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *FilterChunkRefResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ChunkRefs) > 0 { + for iNdEx := len(m.ChunkRefs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ChunkRefs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBloomgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ShortRef) 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 *ShortRef) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShortRef) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Checksum != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.Checksum)) + i-- + dAtA[i] = 0x18 + } + if m.Through != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.Through)) + i-- + dAtA[i] = 0x10 + } + if m.From != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.From)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *GroupedChunkRefs) 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 *GroupedChunkRefs) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GroupedChunkRefs) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Refs) > 0 { + for iNdEx := len(m.Refs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Refs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBloomgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Tenant) > 0 { + i -= len(m.Tenant) + copy(dAtA[i:], m.Tenant) + i = encodeVarintBloomgateway(dAtA, i, uint64(len(m.Tenant))) + i-- + dAtA[i] = 0x12 + } + if m.Fingerprint != 0 { + i = encodeVarintBloomgateway(dAtA, i, uint64(m.Fingerprint)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintBloomgateway(dAtA []byte, offset int, v uint64) int { + offset -= sovBloomgateway(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *FilterChunkRefRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.From != 0 { + n += 1 + sovBloomgateway(uint64(m.From)) + } + if m.Through != 0 { + n += 1 + sovBloomgateway(uint64(m.Through)) + } + if len(m.Refs) > 0 { + for _, e := range m.Refs { + l = e.Size() + n += 1 + l + sovBloomgateway(uint64(l)) + } + } + if len(m.Filters) > 0 { + for _, e := range m.Filters { + l = e.Size() + n += 1 + l + sovBloomgateway(uint64(l)) + } + } + return n +} + +func (m *FilterChunkRefResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.ChunkRefs) > 0 { + for _, e := range m.ChunkRefs { + l = e.Size() + n += 1 + l + sovBloomgateway(uint64(l)) + } + } + return n +} + +func (m *ShortRef) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.From != 0 { + n += 1 + sovBloomgateway(uint64(m.From)) + } + if m.Through != 0 { + n += 1 + sovBloomgateway(uint64(m.Through)) + } + if m.Checksum != 0 { + n += 1 + sovBloomgateway(uint64(m.Checksum)) + } + return n +} + +func (m *GroupedChunkRefs) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Fingerprint != 0 { + n += 1 + sovBloomgateway(uint64(m.Fingerprint)) + } + l = len(m.Tenant) + if l > 0 { + n += 1 + l + sovBloomgateway(uint64(l)) + } + if len(m.Refs) > 0 { + for _, e := range m.Refs { + l = e.Size() + n += 1 + l + sovBloomgateway(uint64(l)) + } + } + return n +} + +func sovBloomgateway(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozBloomgateway(x uint64) (n int) { + return sovBloomgateway(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *FilterChunkRefRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForRefs := "[]*ChunkRef{" + for _, f := range this.Refs { + repeatedStringForRefs += strings.Replace(fmt.Sprintf("%v", f), "ChunkRef", "ChunkRef", 1) + "," + } + repeatedStringForRefs += "}" + repeatedStringForFilters := "[]*LineFilterExpression{" + for _, f := range this.Filters { + repeatedStringForFilters += strings.Replace(fmt.Sprintf("%v", f), "LineFilterExpression", "LineFilterExpression", 1) + "," + } + repeatedStringForFilters += "}" + s := strings.Join([]string{`&FilterChunkRefRequest{`, + `From:` + fmt.Sprintf("%v", this.From) + `,`, + `Through:` + fmt.Sprintf("%v", this.Through) + `,`, + `Refs:` + repeatedStringForRefs + `,`, + `Filters:` + repeatedStringForFilters + `,`, + `}`, + }, "") + return s +} +func (this *FilterChunkRefResponse) String() string { + if this == nil { + return "nil" + } + repeatedStringForChunkRefs := "[]*GroupedChunkRefs{" + for _, f := range this.ChunkRefs { + repeatedStringForChunkRefs += strings.Replace(f.String(), "GroupedChunkRefs", "GroupedChunkRefs", 1) + "," + } + repeatedStringForChunkRefs += "}" + s := strings.Join([]string{`&FilterChunkRefResponse{`, + `ChunkRefs:` + repeatedStringForChunkRefs + `,`, + `}`, + }, "") + return s +} +func (this *ShortRef) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ShortRef{`, + `From:` + fmt.Sprintf("%v", this.From) + `,`, + `Through:` + fmt.Sprintf("%v", this.Through) + `,`, + `Checksum:` + fmt.Sprintf("%v", this.Checksum) + `,`, + `}`, + }, "") + return s +} +func (this *GroupedChunkRefs) String() string { + if this == nil { + return "nil" + } + repeatedStringForRefs := "[]*ShortRef{" + for _, f := range this.Refs { + repeatedStringForRefs += strings.Replace(f.String(), "ShortRef", "ShortRef", 1) + "," + } + repeatedStringForRefs += "}" + s := strings.Join([]string{`&GroupedChunkRefs{`, + `Fingerprint:` + fmt.Sprintf("%v", this.Fingerprint) + `,`, + `Tenant:` + fmt.Sprintf("%v", this.Tenant) + `,`, + `Refs:` + repeatedStringForRefs + `,`, + `}`, + }, "") + return s +} +func valueToStringBloomgateway(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *FilterChunkRefRequest) 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 ErrIntOverflowBloomgateway + } + 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: FilterChunkRefRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FilterChunkRefRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + m.From = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.From |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Through", wireType) + } + m.Through = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Through |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Refs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBloomgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBloomgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Refs = append(m.Refs, &ChunkRef{}) + if err := m.Refs[len(m.Refs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + 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 ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBloomgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBloomgateway + } + 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 := skipBloomgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FilterChunkRefResponse) 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 ErrIntOverflowBloomgateway + } + 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: FilterChunkRefResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FilterChunkRefResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkRefs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBloomgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBloomgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ChunkRefs = append(m.ChunkRefs, &GroupedChunkRefs{}) + if err := m.ChunkRefs[len(m.ChunkRefs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBloomgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ShortRef) 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 ErrIntOverflowBloomgateway + } + 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: ShortRef: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShortRef: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + m.From = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.From |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Through", wireType) + } + m.Through = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Through |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Checksum", wireType) + } + m.Checksum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Checksum |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipBloomgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GroupedChunkRefs) 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 ErrIntOverflowBloomgateway + } + 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: GroupedChunkRefs: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GroupedChunkRefs: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Fingerprint", wireType) + } + m.Fingerprint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Fingerprint |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tenant", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + 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 ErrInvalidLengthBloomgateway + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthBloomgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tenant = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Refs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBloomgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBloomgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Refs = append(m.Refs, &ShortRef{}) + if err := m.Refs[len(m.Refs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBloomgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBloomgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipBloomgateway(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthBloomgateway + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthBloomgateway + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBloomgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipBloomgateway(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthBloomgateway + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthBloomgateway = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowBloomgateway = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/logproto/bloomgateway.proto b/pkg/logproto/bloomgateway.proto new file mode 100644 index 0000000000000..8dd79e15a46c9 --- /dev/null +++ b/pkg/logproto/bloomgateway.proto @@ -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) {} +} diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index 433503a6f3a8f..6aa905ab98a82 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -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:]) diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 543010a73521a..35824a78e202e 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -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 { diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 95c084ab517c7..ed32e1678066b 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -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") diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 22f4c04407e00..796c56ca49235 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -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 { diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 9efef05a78fc0..c206ac9338b49 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -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) { diff --git a/pkg/querier/ingester_querier.go b/pkg/querier/ingester_querier.go index a1234504889b4..1312cf7168ea6 100644 --- a/pkg/querier/ingester_querier.go +++ b/pkg/querier/ingester_querier.go @@ -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, } diff --git a/pkg/storage/factory.go b/pkg/storage/factory.go index 5df4d7632b047..0005560e18015 100644 --- a/pkg/storage/factory.go +++ b/pkg/storage/factory.go @@ -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" diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index 55639cdf67e39..55a36d133a438 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -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 diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index a1ba78b724d19..cb4f91c85c505 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -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 } diff --git a/pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig/config.go b/pkg/storage/stores/shipper/bloomshipper/config/config.go similarity index 80% rename from pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig/config.go rename to pkg/storage/stores/shipper/bloomshipper/config/config.go index a7ae1c4db752c..7e9ab787ff3ab 100644 --- a/pkg/storage/stores/shipper/bloomshipper/bloomshipperconfig/config.go +++ b/pkg/storage/stores/shipper/bloomshipper/config/config.go @@ -1,4 +1,5 @@ -package bloomshipperconfig +// package bloomshipperconfig resides in its own package to prevent circular imports with storage package +package config import ( "errors" diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index c8703fc435db2..9d1679d6ca3b6 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -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. diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index f837aca9070d8..45450c0e3838b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -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"}, diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go new file mode 100644 index 0000000000000..e174fbd1696de --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -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 +} diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go new file mode 100644 index 0000000000000..8048102c156ba --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -0,0 +1,11 @@ +package bloomshipper + +import ( + "testing" +) + +func TestBloomShipper(t *testing.T) { +} + +func TestBloomStore(t *testing.T) { +} diff --git a/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go b/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go index d41b51caeb696..dab3458b77757 100644 --- a/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go +++ b/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go @@ -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{ diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go index 1120e94c6eabc..8b7613ffe048f 100644 --- a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go @@ -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 } diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go index 192061038a33e..171672cf272e9 100644 --- a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go @@ -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") diff --git a/pkg/util/limiter/combined_limits.go b/pkg/util/limiter/combined_limits.go index 2c2440b0f9f8d..59f0b6dec3a49 100644 --- a/pkg/util/limiter/combined_limits.go +++ b/pkg/util/limiter/combined_limits.go @@ -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 } diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index edec616249538..0600b9ebb5ffc 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -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 } diff --git a/tools/doc-generator/parse/root_blocks.go b/tools/doc-generator/parse/root_blocks.go index 74fe3e2fabace..f83f731be9071 100644 --- a/tools/doc-generator/parse/root_blocks.go +++ b/tools/doc-generator/parse/root_blocks.go @@ -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{})},