mirror of https://github.com/grafana/loki
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>pull/6207/head
parent
b90c460769
commit
440fb7d74c
@ -0,0 +1,255 @@ |
||||
package indexgateway |
||||
|
||||
import ( |
||||
"context" |
||||
"fmt" |
||||
"net/http" |
||||
"time" |
||||
|
||||
"github.com/go-kit/log" |
||||
"github.com/go-kit/log/level" |
||||
"github.com/grafana/dskit/kv" |
||||
"github.com/grafana/dskit/ring" |
||||
"github.com/grafana/dskit/services" |
||||
"github.com/pkg/errors" |
||||
"github.com/prometheus/client_golang/prometheus" |
||||
|
||||
loki_util "github.com/grafana/loki/pkg/util" |
||||
) |
||||
|
||||
const ( |
||||
ringAutoForgetUnhealthyPeriods = 10 |
||||
ringNameForServer = "index-gateway" |
||||
ringNumTokens = 128 |
||||
ringCheckPeriod = 3 * time.Second |
||||
|
||||
// RingIdentifier is used as a unique name to register the Index Gateway ring.
|
||||
RingIdentifier = "index-gateway" |
||||
|
||||
// RingKey is the name of the key used to register the different Index Gateway instances in the key-value store.
|
||||
RingKey = "index-gateway" |
||||
) |
||||
|
||||
// ManagerMode defines the different modes for the RingManager to execute.
|
||||
//
|
||||
// The RingManager and its modes are only relevant if the IndexGateway is running in ring mode.
|
||||
type ManagerMode int |
||||
|
||||
const ( |
||||
// ClientMode is the RingManager mode executed by Loki components that are clients of the IndexGateway.
|
||||
// The RingManager in client will have its own ring key-value store but it won't try to register itself in the ring.
|
||||
ClientMode ManagerMode = iota |
||||
|
||||
// ServerMode is the RingManager mode execute by the IndexGateway.
|
||||
// The RingManager in server mode will register itself in the ring.
|
||||
ServerMode |
||||
) |
||||
|
||||
// RingManager is a component instantiated before all the others and is responsible for the ring setup.
|
||||
//
|
||||
// All Loki components that are involved with the IndexGateway (including the IndexGateway itself) will
|
||||
// require a RingManager. However, the components that are clients of the IndexGateway will ran it in client
|
||||
// mode while the IndexGateway itself will ran the manager in server mode.
|
||||
type RingManager struct { |
||||
services.Service |
||||
|
||||
subservices *services.Manager |
||||
subservicesWatcher *services.FailureWatcher |
||||
|
||||
RingLifecycler *ring.BasicLifecycler |
||||
Ring *ring.Ring |
||||
managerMode ManagerMode |
||||
|
||||
cfg Config |
||||
|
||||
log log.Logger |
||||
} |
||||
|
||||
// NewRingManager is the recommended way of instantiating a RingManager.
|
||||
//
|
||||
// The other functions will assume the RingManager was instantiated through this function.
|
||||
func NewRingManager(managerMode ManagerMode, cfg Config, log log.Logger, registerer prometheus.Registerer) (*RingManager, error) { |
||||
rm := &RingManager{ |
||||
cfg: cfg, log: log, managerMode: managerMode, |
||||
} |
||||
|
||||
if cfg.Mode != RingMode { |
||||
return nil, fmt.Errorf("ring manager shouldn't be invoked when index gateway not in ring mode") |
||||
} |
||||
|
||||
// instantiate kv store for both modes.
|
||||
ringStore, err := kv.NewClient( |
||||
rm.cfg.Ring.KVStore, |
||||
ring.GetCodec(), |
||||
kv.RegistererWithKVName(prometheus.WrapRegistererWithPrefix("loki_", registerer), "index-gateway-ring-manager"), |
||||
rm.log, |
||||
) |
||||
if err != nil { |
||||
return nil, errors.Wrap(err, "index gateway ring manager create KV store client") |
||||
} |
||||
|
||||
// instantiate ring for both mode modes.
|
||||
ringCfg := rm.cfg.Ring.ToRingConfig(rm.cfg.Ring.ReplicationFactor) |
||||
rm.Ring, err = ring.NewWithStoreClientAndStrategy(ringCfg, ringNameForServer, RingKey, ringStore, ring.NewIgnoreUnhealthyInstancesReplicationStrategy(), prometheus.WrapRegistererWithPrefix("loki_", registerer), rm.log) |
||||
if err != nil { |
||||
return nil, errors.Wrap(err, "index gateway ring manager create ring client") |
||||
} |
||||
|
||||
if managerMode == ServerMode { |
||||
if err := rm.startServerMode(ringStore, registerer); err != nil { |
||||
return nil, err |
||||
} |
||||
return rm, nil |
||||
} |
||||
|
||||
if err := rm.startClientMode(); err != nil { |
||||
return nil, err |
||||
} |
||||
return rm, nil |
||||
} |
||||
|
||||
func (rm *RingManager) startServerMode(ringStore kv.Client, registerer prometheus.Registerer) error { |
||||
lifecyclerCfg, err := rm.cfg.Ring.ToLifecyclerConfig(ringNumTokens, rm.log) |
||||
if err != nil { |
||||
return errors.Wrap(err, "invalid ring lifecycler config") |
||||
} |
||||
|
||||
delegate := ring.BasicLifecyclerDelegate(rm) |
||||
delegate = ring.NewLeaveOnStoppingDelegate(delegate, rm.log) |
||||
delegate = ring.NewTokensPersistencyDelegate(rm.cfg.Ring.TokensFilePath, ring.JOINING, delegate, rm.log) |
||||
delegate = ring.NewAutoForgetDelegate(ringAutoForgetUnhealthyPeriods*rm.cfg.Ring.HeartbeatTimeout, delegate, rm.log) |
||||
|
||||
rm.RingLifecycler, err = ring.NewBasicLifecycler(lifecyclerCfg, ringNameForServer, RingKey, ringStore, delegate, rm.log, registerer) |
||||
if err != nil { |
||||
return errors.Wrap(err, "index gateway ring manager create ring lifecycler") |
||||
} |
||||
|
||||
svcs := []services.Service{rm.RingLifecycler, rm.Ring} |
||||
rm.subservices, err = services.NewManager(svcs...) |
||||
if err != nil { |
||||
return errors.Wrap(err, "new index gateway services manager in server mode") |
||||
} |
||||
|
||||
rm.subservicesWatcher = services.NewFailureWatcher() |
||||
rm.subservicesWatcher.WatchManager(rm.subservices) |
||||
rm.Service = services.NewBasicService(rm.starting, rm.running, rm.stopping) |
||||
|
||||
return nil |
||||
} |
||||
|
||||
func (rm *RingManager) startClientMode() error { |
||||
var err error |
||||
|
||||
svcs := []services.Service{rm.Ring} |
||||
rm.subservices, err = services.NewManager(svcs...) |
||||
if err != nil { |
||||
return errors.Wrap(err, "new index gateway services manager in client mode") |
||||
} |
||||
|
||||
rm.subservicesWatcher = services.NewFailureWatcher() |
||||
rm.subservicesWatcher.WatchManager(rm.subservices) |
||||
|
||||
rm.Service = services.NewIdleService(func(ctx context.Context) error { |
||||
return services.StartManagerAndAwaitHealthy(ctx, rm.subservices) |
||||
}, func(failureCase error) error { |
||||
return services.StopManagerAndAwaitStopped(context.Background(), rm.subservices) |
||||
}) |
||||
|
||||
return nil |
||||
} |
||||
|
||||
// starting implements the Lifecycler interface and is one of the lifecycle hooks.
|
||||
func (rm *RingManager) starting(ctx context.Context) (err error) { |
||||
// In case this function will return error we want to unregister the instance
|
||||
// from the ring. We do it ensuring dependencies are gracefully stopped if they
|
||||
// were already started.
|
||||
defer func() { |
||||
if err == nil || rm.subservices == nil { |
||||
return |
||||
} |
||||
|
||||
if stopErr := services.StopManagerAndAwaitStopped(context.Background(), rm.subservices); stopErr != nil { |
||||
level.Error(rm.log).Log("msg", "failed to gracefully stop index gateway ring manager dependencies", "err", stopErr) |
||||
} |
||||
}() |
||||
|
||||
if err := services.StartManagerAndAwaitHealthy(ctx, rm.subservices); err != nil { |
||||
return errors.Wrap(err, "unable to start index gateway ring manager subservices") |
||||
} |
||||
|
||||
// The BasicLifecycler does not automatically move state to ACTIVE such that any additional work that
|
||||
// someone wants to do can be done before becoming ACTIVE. For the index gateway we don't currently
|
||||
// have any additional work so we can become ACTIVE right away.
|
||||
// Wait until the ring client detected this instance in the JOINING
|
||||
// state to make sure that when we'll run the initial sync we already
|
||||
// know the tokens assigned to this instance.
|
||||
level.Info(rm.log).Log("msg", "waiting until index gateway is JOINING in the ring") |
||||
if err := ring.WaitInstanceState(ctx, rm.Ring, rm.RingLifecycler.GetInstanceID(), ring.JOINING); err != nil { |
||||
return err |
||||
} |
||||
level.Info(rm.log).Log("msg", "index gateway is JOINING in the ring") |
||||
|
||||
if err = rm.RingLifecycler.ChangeState(ctx, ring.ACTIVE); err != nil { |
||||
return errors.Wrapf(err, "switch instance to %s in the ring", ring.ACTIVE) |
||||
} |
||||
|
||||
// Wait until the ring client detected this instance in the ACTIVE state to
|
||||
// make sure that when we'll run the loop it won't be detected as a ring
|
||||
// topology change.
|
||||
level.Info(rm.log).Log("msg", "waiting until index gateway is ACTIVE in the ring") |
||||
if err := ring.WaitInstanceState(ctx, rm.Ring, rm.RingLifecycler.GetInstanceID(), ring.ACTIVE); err != nil { |
||||
return err |
||||
} |
||||
level.Info(rm.log).Log("msg", "index gateway is ACTIVE in the ring") |
||||
|
||||
return nil |
||||
} |
||||
|
||||
// running implements the Lifecycler interface and is one of the lifecycle hooks.
|
||||
func (rm *RingManager) running(ctx context.Context) error { |
||||
t := time.NewTicker(ringCheckPeriod) |
||||
defer t.Stop() |
||||
for { |
||||
select { |
||||
case <-ctx.Done(): |
||||
return nil |
||||
case err := <-rm.subservicesWatcher.Chan(): |
||||
return errors.Wrap(err, "running index gateway ring manager subservice failed") |
||||
case <-t.C: |
||||
continue |
||||
} |
||||
} |
||||
} |
||||
|
||||
// stopping implements the Lifecycler interface and is one of the lifecycle hooks.
|
||||
func (rm *RingManager) stopping(_ error) error { |
||||
level.Debug(rm.log).Log("msg", "stopping index gateway ring manager") |
||||
return services.StopManagerAndAwaitStopped(context.Background(), rm.subservices) |
||||
} |
||||
|
||||
// IndexGatewayOwnsTenant dictates if a given tenant should be ignored by an IndexGateway or not.
|
||||
//
|
||||
// It fallbacks to true so that the IndexGateway will only skip tenants if it is certain of that.
|
||||
// This implementation relies on the tokens assigned to an IndexGateway instance to define if a tenant
|
||||
// is assigned or not.
|
||||
func (rm *RingManager) IndexGatewayOwnsTenant(tenant string) bool { |
||||
if rm.cfg.Mode != RingMode { |
||||
return true |
||||
} |
||||
|
||||
if rm.managerMode == ClientMode { |
||||
level.Error(rm.log).Log("msg", "ring manager in client mode doesn't support tenant in boundaries interface") |
||||
return true |
||||
} |
||||
|
||||
return loki_util.IsAssignedKey(rm.Ring, rm.RingLifecycler.GetInstanceAddr(), tenant) |
||||
} |
||||
|
||||
// ServeHTTP serves the HTTP route /indexgateway/ring.
|
||||
func (rm *RingManager) ServeHTTP(w http.ResponseWriter, req *http.Request) { |
||||
if rm.cfg.Mode == RingMode { |
||||
rm.Ring.ServeHTTP(w, req) |
||||
} else { |
||||
_, _ = w.Write([]byte("IndexGateway running with 'useIndexGatewayRing' disabled.")) |
||||
} |
||||
} |
@ -0,0 +1,141 @@ |
||||
package util |
||||
|
||||
import ( |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/grafana/dskit/ring" |
||||
"github.com/prometheus/client_golang/prometheus" |
||||
"github.com/stretchr/testify/mock" |
||||
) |
||||
|
||||
func TestTokenFor(t *testing.T) { |
||||
if TokenFor("userID", "labels") != 2908432762 { |
||||
t.Errorf("TokenFor(userID, labels) = %v, want 2908432762", TokenFor("userID", "labels")) |
||||
} |
||||
} |
||||
|
||||
func TestIsAssignedKey(t *testing.T) { |
||||
for _, tc := range []struct { |
||||
desc string |
||||
ring ring.ReadRing |
||||
userID string |
||||
exp bool |
||||
addr string |
||||
}{ |
||||
{ |
||||
desc: "basic ring and tenant are assigned key", |
||||
ring: newReadRingMock([]ring.InstanceDesc{{Addr: "127.0.0.1", Timestamp: time.Now().UnixNano(), State: ring.ACTIVE, Tokens: []uint32{1, 2, 3}}}), |
||||
userID: "1", |
||||
exp: true, |
||||
addr: "127.0.0.1", |
||||
}, |
||||
{ |
||||
desc: "basic ring and tenant are not assigned key", |
||||
ring: newReadRingMock([]ring.InstanceDesc{{Addr: "127.0.0.2", Timestamp: time.Now().UnixNano(), State: ring.ACTIVE, Tokens: []uint32{1, 2, 3}}}), |
||||
userID: "1", |
||||
exp: false, |
||||
addr: "127.0.0.1", |
||||
}, |
||||
} { |
||||
t.Run(tc.desc, func(t *testing.T) { |
||||
if res := IsAssignedKey(tc.ring, newReadLifecyclerMock(tc.addr).addr, tc.userID); res != tc.exp { |
||||
t.Errorf("IsAssignedKey(%v, %v) = %v, want %v", tc.ring, tc.userID, res, tc.exp) |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
type readRingMock struct { |
||||
replicationSet ring.ReplicationSet |
||||
} |
||||
|
||||
func newReadRingMock(ingesters []ring.InstanceDesc) *readRingMock { |
||||
return &readRingMock{ |
||||
replicationSet: ring.ReplicationSet{ |
||||
Instances: ingesters, |
||||
MaxErrors: 0, |
||||
}, |
||||
} |
||||
} |
||||
|
||||
func (r *readRingMock) Describe(ch chan<- *prometheus.Desc) { |
||||
} |
||||
|
||||
func (r *readRingMock) Collect(ch chan<- prometheus.Metric) { |
||||
} |
||||
|
||||
func (r *readRingMock) Get(key uint32, op ring.Operation, buf []ring.InstanceDesc, _ []string, _ []string) (ring.ReplicationSet, error) { |
||||
return r.replicationSet, nil |
||||
} |
||||
|
||||
func (r *readRingMock) ShuffleShard(identifier string, size int) ring.ReadRing { |
||||
// pass by value to copy
|
||||
return func(r readRingMock) *readRingMock { |
||||
r.replicationSet.Instances = r.replicationSet.Instances[:size] |
||||
return &r |
||||
}(*r) |
||||
} |
||||
|
||||
func (r *readRingMock) BatchGet(keys []uint32, op ring.Operation) ([]ring.ReplicationSet, error) { |
||||
return []ring.ReplicationSet{r.replicationSet}, nil |
||||
} |
||||
|
||||
func (r *readRingMock) GetAllHealthy(op ring.Operation) (ring.ReplicationSet, error) { |
||||
return r.replicationSet, nil |
||||
} |
||||
|
||||
func (r *readRingMock) GetReplicationSetForOperation(op ring.Operation) (ring.ReplicationSet, error) { |
||||
return r.replicationSet, nil |
||||
} |
||||
|
||||
func (r *readRingMock) ReplicationFactor() int { |
||||
return 1 |
||||
} |
||||
|
||||
func (r *readRingMock) InstancesCount() int { |
||||
return len(r.replicationSet.Instances) |
||||
} |
||||
|
||||
func (r *readRingMock) Subring(key uint32, n int) ring.ReadRing { |
||||
return r |
||||
} |
||||
|
||||
func (r *readRingMock) HasInstance(instanceID string) bool { |
||||
for _, ing := range r.replicationSet.Instances { |
||||
if ing.Addr != instanceID { |
||||
return true |
||||
} |
||||
} |
||||
return false |
||||
} |
||||
|
||||
func (r *readRingMock) ShuffleShardWithLookback(identifier string, size int, lookbackPeriod time.Duration, now time.Time) ring.ReadRing { |
||||
return r |
||||
} |
||||
|
||||
func (r *readRingMock) CleanupShuffleShardCache(identifier string) {} |
||||
|
||||
func (r *readRingMock) GetInstanceState(instanceID string) (ring.InstanceState, error) { |
||||
return 0, nil |
||||
} |
||||
|
||||
type readLifecyclerMock struct { |
||||
mock.Mock |
||||
addr string |
||||
} |
||||
|
||||
func newReadLifecyclerMock(addr string) *readLifecyclerMock { |
||||
return &readLifecyclerMock{ |
||||
addr: addr, |
||||
} |
||||
} |
||||
|
||||
func (m *readLifecyclerMock) HealthyInstancesCount() int { |
||||
args := m.Called() |
||||
return args.Int(0) |
||||
} |
||||
|
||||
func (m *readLifecyclerMock) GetInstanceAddr() string { |
||||
return m.addr |
||||
} |
Loading…
Reference in new issue