Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/indexgateway/client.go

559 lines
19 KiB

package indexgateway
import (
"context"
"flag"
"fmt"
"io"
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
"math/rand"
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
"time"
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/gogo/status"
"github.com/grafana/dskit/concurrency"
"github.com/grafana/dskit/grpcclient"
"github.com/grafana/dskit/instrument"
"github.com/grafana/dskit/middleware"
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/ring/client"
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
"github.com/grafana/dskit/services"
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
"github.com/grafana/dskit/tenant"
"github.com/grpc-ecosystem/grpc-opentracing/go/otgrpc"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"github.com/grafana/loki/v3/pkg/distributor/clientpool"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/storage/stores/series/index"
"github.com/grafana/loki/v3/pkg/util/constants"
"github.com/grafana/loki/v3/pkg/util/discovery"
util_math "github.com/grafana/loki/v3/pkg/util/math"
)
const (
maxQueriesPerGrpc = 100
maxConcurrentGrpcCalls = 10
)
// ClientConfig configures the Index Gateway client used to communicate with
// the Index Gateway server.
type ClientConfig struct {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// Mode sets in which mode the client will operate. It is actually defined at the
// index_gateway YAML section and reused here.
Mode Mode `yaml:"-"`
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// PoolConfig defines the behavior of the gRPC connection pool used to communicate
// with the Index Gateway.
//
// Only relevant for the ring mode.
// It is defined at the distributors YAML section and reused here.
PoolConfig clientpool.PoolConfig `yaml:"-"`
// Ring is the Index Gateway ring used to find the appropriate Index Gateway instance
// this client should talk to.
//
// Only relevant for the ring mode.
Ring ring.ReadRing `yaml:"-"`
// GRPCClientConfig configures the gRPC connection between the Index Gateway client and the server.
//
// Used by both, ring and simple mode.
GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"`
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// Address of the Index Gateway instance responsible for retaining the index for all tenants.
//
// Only relevant for the simple mode.
Address string `yaml:"server_address,omitempty"`
// Forcefully disable the use of the index gateway client for the storage.
// This is mainly useful for the index-gateway component which should always use the storage.
Disabled bool `yaml:"-"`
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
// 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"`
GRPCUnaryClientInterceptors []grpc.UnaryClientInterceptor `yaml:"-"`
GRCPStreamClientInterceptors []grpc.StreamClientInterceptor `yaml:"-"`
}
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// RegisterFlagsWithPrefix register client-specific flags with the given prefix.
//
// Flags that are used by both, client and server, are defined in the indexgateway package.
func (i *ClientConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
i.GRPCClientConfig.RegisterFlagsWithPrefix(prefix+".grpc", f)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
f.StringVar(&i.Address, prefix+".server-address", "", "Hostname or IP of the Index Gateway gRPC server running in simple mode. Can also be prefixed with dns+, dnssrv+, or dnssrvnoa+ to resolve a DNS A record with multiple IP's, a DNS SRV record with a followup A record lookup, or a DNS SRV record without a followup A record lookup, respectively.")
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
f.BoolVar(&i.LogGatewayRequests, prefix+".log-gateway-requests", false, "Whether requests sent to the gateway should be logged or not.")
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
func (i *ClientConfig) RegisterFlags(f *flag.FlagSet) {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
i.RegisterFlagsWithPrefix("index-gateway-client", f)
}
type GatewayClient struct {
logger log.Logger
cfg ClientConfig
storeGatewayClientRequestDuration *prometheus.HistogramVec
dnsProvider *discovery.DNS
pool *client.Pool
ring ring.ReadRing
limits Limits
done chan struct{}
}
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// NewGatewayClient instantiates a new client used to communicate with an Index Gateway instance.
//
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
// If it is configured to be in ring mode, a pool of GRPC connections to all Index Gateway instances is created using a ring.
// Otherwise, it creates a GRPC connection pool to as many addresses as can be resolved from the given address.
func NewGatewayClient(cfg ClientConfig, r prometheus.Registerer, limits Limits, logger log.Logger, metricsNamespace string) (*GatewayClient, error) {
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
latency := prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: constants.Loki,
Name: "index_gateway_request_duration_seconds",
Help: "Time (in seconds) spent serving requests when using the index gateway",
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
Buckets: instrument.DefBuckets,
}, []string{"operation", "status_code"})
if r != nil {
err := r.Register(latency)
if err != nil {
alreadyErr, ok := err.(prometheus.AlreadyRegisteredError)
if !ok {
return nil, err
}
latency = alreadyErr.ExistingCollector.(*prometheus.HistogramVec)
}
}
sgClient := &GatewayClient{
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
logger: logger,
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
cfg: cfg,
storeGatewayClientRequestDuration: latency,
ring: cfg.Ring,
limits: limits,
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
done: make(chan struct{}),
}
dialOpts, err := cfg.GRPCClientConfig.DialOption(instrumentation(cfg, sgClient.storeGatewayClientRequestDuration))
if err != nil {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
return nil, errors.Wrap(err, "index gateway grpc dial option")
}
factory := func(addr string) (client.PoolClient, error) {
igPool, err := NewClientPool(addr, dialOpts)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
if err != nil {
return nil, errors.Wrap(err, "new index gateway grpc pool")
}
return igPool, nil
}
//FIXME(ewelch) we don't expose the pool configs nor set defaults, and register flags is kind of messed up with remote config being defined somewhere else
//make a separate PR to make the pool config generic so it can be used with proper names in multiple places.
sgClient.cfg.PoolConfig.RemoteTimeout = 2 * time.Second
sgClient.cfg.PoolConfig.ClientCleanupPeriod = 5 * time.Second
sgClient.cfg.PoolConfig.HealthCheckIngesters = true
if sgClient.cfg.Mode == RingMode {
sgClient.pool = clientpool.NewPool("index-gateway", sgClient.cfg.PoolConfig, sgClient.ring, client.PoolAddrFunc(factory), logger, metricsNamespace)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
} else {
// Note we don't use clientpool.NewPool because we want to provide our own discovery function
poolCfg := client.PoolConfig{
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
CheckInterval: sgClient.cfg.PoolConfig.ClientCleanupPeriod,
HealthCheckEnabled: sgClient.cfg.PoolConfig.HealthCheckIngesters,
HealthCheckTimeout: sgClient.cfg.PoolConfig.RemoteTimeout,
}
clients := prometheus.NewGauge(prometheus.GaugeOpts{
Namespace: constants.Loki,
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
Name: "index_gateway_clients",
Help: "The current number of index gateway clients.",
})
if r != nil {
err := r.Register(clients)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
if err != nil {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
alreadyErr, ok := err.(prometheus.AlreadyRegisteredError)
if !ok {
return nil, err
}
clients = alreadyErr.ExistingCollector.(prometheus.Gauge)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
}
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
//TODO(ewelch) we can't use metrics in the provider because of duplicate registration errors
dnsProvider := discovery.NewDNS(logger, sgClient.cfg.PoolConfig.ClientCleanupPeriod, sgClient.cfg.Address, nil)
sgClient.dnsProvider = dnsProvider
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
// Make an attempt to do one DNS lookup so we can start with addresses
dnsProvider.RunOnce()
discovery := func() ([]string, error) {
return dnsProvider.Addresses(), nil
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
sgClient.pool = client.NewPool("index gateway", poolCfg, discovery, client.PoolAddrFunc(factory), clients, logger)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
}
// We have to start the pool service, it will handle removing stale clients in the background
err = services.StartAndAwaitRunning(context.Background(), sgClient.pool)
if err != nil {
return nil, errors.Wrap(err, "failed to start index gateway connection pool")
}
return sgClient, nil
}
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
// Stop stops the execution of this gateway client.
func (s *GatewayClient) Stop() {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
ctx, cancel := context.WithTimeoutCause(context.Background(), 10*time.Second, errors.New("service shutdown timeout expired"))
defer cancel()
err := services.StopAndAwaitTerminated(ctx, s.pool)
if err != nil {
level.Error(s.logger).Log("msg", "failed to stop index gateway connection pool", "err", err)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
if s.cfg.Mode == SimpleMode {
s.dnsProvider.Stop()
}
}
func (s *GatewayClient) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
if len(queries) <= maxQueriesPerGrpc {
return s.doQueries(ctx, queries, callback)
}
jobsCount := len(queries) / maxQueriesPerGrpc
if len(queries)%maxQueriesPerGrpc != 0 {
jobsCount++
}
return concurrency.ForEachJob(ctx, jobsCount, maxConcurrentGrpcCalls, func(ctx context.Context, idx int) error {
return s.doQueries(ctx, queries[idx*maxQueriesPerGrpc:util_math.Min((idx+1)*maxQueriesPerGrpc, len(queries))], callback)
})
}
func (s *GatewayClient) QueryIndex(_ context.Context, _ *logproto.QueryIndexRequest, _ ...grpc.CallOption) (logproto.IndexGateway_QueryIndexClient, error) {
panic("not implemented")
}
func (s *GatewayClient) GetChunkRef(ctx context.Context, in *logproto.GetChunkRefRequest) (*logproto.GetChunkRefResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.GetChunkRefResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.GetChunkRef(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
}
func (s *GatewayClient) GetSeries(ctx context.Context, in *logproto.GetSeriesRequest) (*logproto.GetSeriesResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.GetSeriesResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.GetSeries(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
}
func (s *GatewayClient) LabelNamesForMetricName(ctx context.Context, in *logproto.LabelNamesForMetricNameRequest) (*logproto.LabelResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.LabelResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.LabelNamesForMetricName(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
}
func (s *GatewayClient) LabelValuesForMetricName(ctx context.Context, in *logproto.LabelValuesForMetricNameRequest) (*logproto.LabelResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.LabelResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.LabelValuesForMetricName(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
}
func (s *GatewayClient) GetStats(ctx context.Context, in *logproto.IndexStatsRequest) (*logproto.IndexStatsResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.IndexStatsResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.GetStats(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
}
func (s *GatewayClient) GetVolume(ctx context.Context, in *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var (
resp *logproto.VolumeResponse
err error
)
err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.GetVolume(ctx, in)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return err
})
return resp, err
}
func (s *GatewayClient) GetShards(
ctx context.Context,
in *logproto.ShardsRequest,
) (res *logproto.ShardsResponse, err error) {
// We try to get the shards from the index gateway,
// but if it's not implemented, we fall back to the stats.
// We limit the maximum number of errors to 2 to avoid
// cascading all requests to new node(s) when
// the idx-gw replicas start to update to a version
// which supports the new API.
var (
maxErrs = 2
errCt int
)
if err := s.poolDoWithStrategy(
ctx,
func(client logproto.IndexGatewayClient) error {
perReplicaResult := &logproto.ShardsResponse{}
streamer, err := client.GetShards(ctx, in)
if err != nil {
return errors.Wrap(err, "get shards")
}
// TODO(owen-d): stream currently unused (buffered) because query planning doesn't expect a streamed response,
// but can be improved easily in the future by using a stream here.
for {
resp, err := streamer.Recv()
if err == io.EOF {
break
}
if err != nil {
return errors.WithStack(err)
}
perReplicaResult.Merge(resp)
}
// Since `poolDo` retries on error, we only want to set the response if we got a successful response.
// This avoids cases where we add duplicates to the response on retries.
res = perReplicaResult
return nil
},
func(err error) bool {
errCt++
return errCt <= maxErrs
},
); err != nil {
return nil, err
}
return res, nil
}
// TODO(owen-d): this was copied from ingester_querier.go -- move it to a shared pkg
// isUnimplementedCallError tells if the GRPC error is a gRPC error with code Unimplemented.
func isUnimplementedCallError(err error) bool {
if err == nil {
return false
}
s, ok := status.FromError(err)
if !ok {
return false
}
return (s.Code() == codes.Unimplemented)
}
func (s *GatewayClient) doQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
queryKeyQueryMap := make(map[string]index.Query, len(queries))
gatewayQueries := make([]*logproto.IndexQuery, 0, len(queries))
for _, query := range queries {
queryKeyQueryMap[index.QueryKey(query)] = query
gatewayQueries = append(gatewayQueries, &logproto.IndexQuery{
TableName: query.TableName,
HashValue: query.HashValue,
RangeValuePrefix: query.RangeValuePrefix,
RangeValueStart: query.RangeValueStart,
ValueEqual: query.ValueEqual,
})
}
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
return s.poolDo(ctx, func(client logproto.IndexGatewayClient) error {
return s.clientDoQueries(ctx, gatewayQueries, queryKeyQueryMap, callback, client)
})
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
// clientDoQueries send a query request to an Index Gateway instance using the given gRPC client.
//
// It is used by both, simple and ring mode.
func (s *GatewayClient) clientDoQueries(ctx context.Context, gatewayQueries []*logproto.IndexQuery,
queryKeyQueryMap map[string]index.Query, callback index.QueryPagesCallback, client logproto.IndexGatewayClient,
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
) error {
streamer, err := client.QueryIndex(ctx, &logproto.QueryIndexRequest{Queries: gatewayQueries})
if err != nil {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
return errors.Wrap(err, "query index")
}
for {
resp, err := streamer.Recv()
if err == io.EOF {
break
}
if err != nil {
return errors.WithStack(err)
}
query, ok := queryKeyQueryMap[resp.QueryKey]
if !ok {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
level.Error(s.logger).Log("msg", fmt.Sprintf("unexpected %s QueryKey received, expected queries %s", resp.QueryKey, fmt.Sprint(queryKeyQueryMap)))
return fmt.Errorf("unexpected %s QueryKey received", resp.QueryKey)
}
if !callback(query, &readBatch{resp}) {
return nil
}
}
return nil
}
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
// poolDo executes the given function for each Index Gateway instance in the ring mapping to the correct tenant in the index.
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
// In case of callback failure, we'll try another member of the ring for that tenant ID.
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
func (s *GatewayClient) poolDo(ctx context.Context, callback func(client logproto.IndexGatewayClient) error) error {
return s.poolDoWithStrategy(ctx, callback, func(error) bool { return true })
}
func (s *GatewayClient) poolDoWithStrategy(
ctx context.Context,
callback func(client logproto.IndexGatewayClient) error,
shouldRetry func(error) bool,
) error {
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
userID, err := tenant.TenantID(ctx)
if err != nil {
return errors.Wrap(err, "index gateway client get tenant ID")
}
addrs, err := s.getServerAddresses(userID)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
if err != nil {
return err
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
if len(addrs) == 0 {
level.Error(s.logger).Log("msg", fmt.Sprintf("no index gateway instances found for tenant %s", userID))
return fmt.Errorf("no index gateway instances found for tenant %s", userID)
}
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
var lastErr error
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
for _, addr := range addrs {
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
if s.cfg.LogGatewayRequests {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
level.Debug(s.logger).Log("msg", "sending request to gateway", "gateway", addr, "tenant", userID)
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
4 years ago
}
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
genericClient, err := s.pool.GetClientFor(addr)
if err != nil {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
level.Error(s.logger).Log("msg", fmt.Sprintf("failed to get client for instance %s", addr), "err", err)
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
continue
}
client := (genericClient.(logproto.IndexGatewayClient))
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
if err := callback(client); err != nil {
lastErr = err
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
level.Error(s.logger).Log("msg", fmt.Sprintf("client do failed for instance %s", addr), "err", err)
if !shouldRetry(err) {
return err
}
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
continue
}
return nil
}
Introduce a more scalable index-gateway API. (#5892) * Introduce a more scalable index-gateway API. Because of the storage refactoring (https://github.com/grafana/loki/pull/5833) we can now introduce a better index-gateway API that fits more our index interface. Such as : ```proto rpc GetChunkRef(GetChunkRefRequest) returns (GetChunkRefResponse) {}; rpc LabelNamesForMetricName(LabelNamesForMetricNameRequest) returns (LabelResponse) {}; rpc LabelValuesForMetricName(LabelValuesForMetricNameRequest) returns (LabelResponse) {}; ``` This will avoid sending thousands of index queries to the index-gateway but instead them just a single request. The index caching, parsing and filtering is now happening all on the index-gateway side. Loki queriers will first check if the new API exists before using it, this way update can be done transparently. However the check happens only on startup, so if you want to start using the new API you need to restart queriers after fully rolling out index-gateways. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Attempt to use the new API on every call. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * Add test for the GRPC fallback. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com> * lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the parameters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedback Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <christian.simon@grafana.com>
4 years ago
return lastErr
Add a Ring to IndexGateway (#5358) * Begin to add a Ring to IndexGateway Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement missing methods for IndexGateway. - Implement missing methods for the IndexGateway to be used as a BasicLifecyclerDelegate. The methods are stored in a lifecycle file and follow the same approach of the Ruler ring - Make Gateway public and add missing parameters to the IndexGateway's initialization method * Fix failing linter Signed-off-by: Jordan Rushing <jordan.rushing@grafana.com> * Implement IndexGateway support for dynamic configs. - Add memberlist as a dependency of the IndexGateway - Add IndexGateway support for the common configuration section - Modify Loki to register IndexGateway flags. This fixes the default flag values for the IndexGateway - Make IndexGateway configurations public. Without this, we can't use them. * Implement NewBasicService for the IndexGateway. - If IndexGateway is running with a ring, it uses the IdleService. Otherwise, it uses the BasicService - Implement IndexGateway support to handle ring HTTP pages * Test IndexGateway dynamic configuration. * Implement new IndexGatewayGRPCPool entity. * Make IndexGateway attributes public. * Implement IndexGatewayRing reader. - Implement new IndexGatewayRing service - Add IndexGatewayRing as a Store dependency - Modify store to pass IndexGatewayRing as a parameter - Implement IndexGatewayClient ring mode * Implement Ring mode in the IndexGatewayClient. - Moves IndexGateway configuration to the IndexGatewayClient struct within the boltdb-shipper - Reuse the IndexGatewayClient everywhere - Implement IndexGateway gRPC pool * Add new ring index gateway parameter to new Store calls. * Use errors.Wrap instead of fmt.Errorf. * Extract tenantID from context instead of iterating on queries. * Remove indexGateway ring param. - Instead, add it to the store configuration and bubble it down to deeper modules. * Split IndexGateway server from client implementation. * Fix imports order. * Remove ring as parameter from IndexGateway-related funcs. * Fix default flag value and IndexQuerier type. * Remove additional mode field and reuse it from cfg. * Remove redundant service init. * Add sanity check for IndexGateway client constructor. * Move mode assigning to initStore method. * Reorder IndexGateway constructor. * Rewrite indexClient chunk.IndexClient as querier Index.Querier. * Fix flag registration for IndexGateway server. - In a previous PR I changed how it works and forgot to update how the flag registration occurs. * Fix flag registration for test. * Keep only one reference to indexQuerier. * Add guard-clause on IndexGatewayRing service. - Without this, we would be initializing ring-related things even if the index gateway is in simple mode. * Move IndexGatewayClientCfg to gateway_client file. * Update CHANGELOG.md for `IndexGateway` support for `RingMode` Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Update GatewayClient to use dskit tenant package Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Add listenport configuration for IndexGateway and Ring Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Make IndexGateway replication factor configurable. - Add `replication_factor` flag to the IndexGateway config struct - Modify Index Gateway ring config struct to be inline with a new struct, since it doesn't expose a replication factor config - Modify dynamic config wrapper to reuse a common replication factor on the Index Gateway ring * Randomize replication set access. - If we don't randomize access, we'll always access same Index Gateway instances in same order for the same tenant * Remove unwanted merge HEAD tags. * Move away from stores/chunk package. * Pass util_log in factory. * Change index gateway client ring to ignore replicas. - This is accomplished by using `NewIgnoreUnhealthyInstancesReplicationStrategy` - It is already done by the server * Refactor where the common replication factor is applied. * Housekeeping config_wrapper IndexGateway configs. - Remove unnecessary/wrong comments - Only set replication factor at a single place Co-authored-by: Dylan Guedes <djmgguedes@gmail.com>
4 years ago
}
func (s *GatewayClient) getServerAddresses(tenantID string) ([]string, error) {
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
var addrs []string
// The GRPC pool we use only does discovery calls when cleaning up already existing connections,
// so the list of addresses should always be provided from the external provider (ring or DNS)
// and not from the RegisteredAddresses method as this list is only populated after a call to GetClientFor
if s.cfg.Mode == RingMode {
r := GetShuffleShardingSubring(s.ring, tenantID, s.limits)
rs, err := r.GetReplicationSetForOperation(IndexesRead)
Index gateway simple mode connection pool (#10783) **What this PR does / why we need it**: The current implementation of "simple mode" (which does not use the ring for gateway discovery) only makes a single connection at startup to a single query. This doesn't lend to the best load balancing when running with many gateways. This PR introduces the same pool we use for ring mode with a DNS based discovery mechanism. Note: outside of the scope of this PR, we create a connection pool per index schema period. This isn't really a problem but I think it would be better to revisit this some day to only have a single connection pool per querier. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Signed-off-by: Edward Welch <edward.welch@grafana.com> Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Poyzan <31743851+poyzannur@users.noreply.github.com>
3 years ago
if err != nil {
return nil, errors.Wrap(err, "index gateway get ring")
}
addrs = rs.GetAddresses()
} else {
addrs = s.dnsProvider.Addresses()
}
// shuffle addresses to make sure we don't always access the same Index Gateway instances in sequence for same tenant.
rand.Shuffle(len(addrs), func(i, j int) {
addrs[i], addrs[j] = addrs[j], addrs[i]
})
return addrs, nil
}
func (s *GatewayClient) NewWriteBatch() index.WriteBatch {
panic("unsupported")
}
func (s *GatewayClient) BatchWrite(_ context.Context, _ index.WriteBatch) error {
panic("unsupported")
}
type readBatch struct {
*logproto.QueryIndexResponse
}
func (r *readBatch) Iterator() index.ReadBatchIterator {
return &grpcIter{
i: -1,
QueryIndexResponse: r.QueryIndexResponse,
}
}
type grpcIter struct {
i int
*logproto.QueryIndexResponse
}
func (b *grpcIter) Next() bool {
b.i++
return b.i < len(b.Rows)
}
func (b *grpcIter) RangeValue() []byte {
return b.Rows[b.i].RangeValue
}
func (b *grpcIter) Value() []byte {
return b.Rows[b.i].Value
}
func instrumentation(cfg ClientConfig, clientRequestDuration *prometheus.HistogramVec) ([]grpc.UnaryClientInterceptor, []grpc.StreamClientInterceptor) {
var unaryInterceptors []grpc.UnaryClientInterceptor
unaryInterceptors = append(unaryInterceptors, cfg.GRPCUnaryClientInterceptors...)
unaryInterceptors = append(unaryInterceptors, otgrpc.OpenTracingClientInterceptor(opentracing.GlobalTracer()))
unaryInterceptors = append(unaryInterceptors, middleware.ClientUserHeaderInterceptor)
unaryInterceptors = append(unaryInterceptors, middleware.UnaryClientInstrumentInterceptor(clientRequestDuration))
var streamInterceptors []grpc.StreamClientInterceptor
streamInterceptors = append(streamInterceptors, cfg.GRCPStreamClientInterceptors...)
streamInterceptors = append(streamInterceptors, otgrpc.OpenTracingStreamClientInterceptor(opentracing.GlobalTracer()))
streamInterceptors = append(streamInterceptors, middleware.StreamClientUserHeaderInterceptor)
streamInterceptors = append(streamInterceptors, middleware.StreamClientInstrumentInterceptor(clientRequestDuration))
return unaryInterceptors, streamInterceptors
}