Skip to content

Commit

Permalink
Parallelized cluster hosts discovering + async config cache update (#…
Browse files Browse the repository at this point in the history
…4077)

* fix(cluster): parallelize discovering cluster hosts and make cache update async

Fixes #4074

This PR makes the cluster hosts discovery more robust, as when the cluster.host is DOWN,
it probes all other hosts in parallel and returns response from the fastest one.
Additionally, this PR makes the call to cluster config cache async on updating cluster.

(cherry picked from commit da83642)
  • Loading branch information
karol-kokoszka authored and Michal-Leszczynski committed Dec 10, 2024
1 parent 91a79b5 commit f63ec83
Show file tree
Hide file tree
Showing 9 changed files with 494 additions and 180 deletions.
2 changes: 1 addition & 1 deletion pkg/cmd/scylla-manager/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ func (s *server) makeServices(ctx context.Context) error {
func (s *server) onClusterChange(ctx context.Context, c cluster.Change) error {
switch c.Type {
case cluster.Update:
s.configCacheSvc.ForceUpdateCluster(ctx, c.ID)
go s.configCacheSvc.ForceUpdateCluster(ctx, c.ID)
case cluster.Create:
s.configCacheSvc.ForceUpdateCluster(ctx, c.ID)
for _, t := range makeAutoHealthCheckTasks(c.ID) {
Expand Down
9 changes: 9 additions & 0 deletions pkg/scyllaclient/client_scylla.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,15 @@ func (c *Client) Datacenters(ctx context.Context) (map[string][]string, error) {
return res, errs
}

// GossiperEndpointLiveGet finds live nodes (according to gossiper).
func (c *Client) GossiperEndpointLiveGet(ctx context.Context) ([]string, error) {
live, err := c.scyllaOps.GossiperEndpointLiveGet(&operations.GossiperEndpointLiveGetParams{Context: ctx})
if err != nil {
return nil, err
}
return live.GetPayload(), nil
}

// HostDatacenter looks up the datacenter that the given host belongs to.
func (c *Client) HostDatacenter(ctx context.Context, host string) (dc string, err error) {
// Try reading from cache
Expand Down
8 changes: 4 additions & 4 deletions pkg/scyllaclient/retry_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestRetryWithTimeoutIntegration(t *testing.T) {
test := table[i]

t.Run(fmt.Sprintf("block %d nodes", test.block), func(t *testing.T) {
if err := testRetry(hosts, test.block, test.timeout); err != nil {
if err := testRetry(t, hosts, test.block, test.timeout); err != nil {
t.Fatal(err)
}
})
Expand All @@ -72,12 +72,12 @@ func allHosts() ([]string, error) {
return client.Hosts(context.Background())
}

func testRetry(hosts []string, n int, shouldTimeout bool) error {
func testRetry(t *testing.T, hosts []string, n int, shouldTimeout bool) error {
blockedHosts := make([]string, 0, len(hosts))

block := func(ctx context.Context, hosts []string) error {
for _, h := range hosts {
err := RunIptablesCommand(h, CmdBlockScyllaREST)
err := RunIptablesCommand(t, h, CmdBlockScyllaREST)
if err != nil {
return err
}
Expand All @@ -88,7 +88,7 @@ func testRetry(hosts []string, n int, shouldTimeout bool) error {

unblock := func(ctx context.Context) error {
for _, h := range blockedHosts {
err := RunIptablesCommand(h, CmdUnblockScyllaREST)
err := RunIptablesCommand(t, h, CmdUnblockScyllaREST)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/service/backup/service_backup_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -859,10 +859,10 @@ func TestBackupWithNodesDownIntegration(t *testing.T) {
WriteData(t, clusterSession, testKeyspace, 1)

Print("Given: downed node")
if err := RunIptablesCommand(IPFromTestNet("11"), CmdBlockScyllaREST); err != nil {
if err := RunIptablesCommand(t, IPFromTestNet("11"), CmdBlockScyllaREST); err != nil {
t.Fatal(err)
}
defer RunIptablesCommand(IPFromTestNet("11"), CmdUnblockScyllaREST)
defer RunIptablesCommand(t, IPFromTestNet("11"), CmdUnblockScyllaREST)

Print("When: get target")
target := backup.Target{
Expand Down
164 changes: 118 additions & 46 deletions pkg/service/cluster/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
"fmt"
"sort"
"strconv"
"sync"
"time"

"github.com/gocql/gocql"
Expand All @@ -34,7 +35,10 @@ type ProviderFunc func(ctx context.Context, id uuid.UUID) (*Cluster, error)
type ChangeType int8

// ErrNoValidKnownHost is thrown when it was not possible to connect to any of the currently known hosts of the cluster.
var ErrNoValidKnownHost = errors.New("unable to connect to any of cluster's known hosts")
var (
ErrNoValidKnownHost = errors.New("unable to connect to any of cluster's known hosts")
ErrNoLiveHostAvailable = errors.New("no single live host available")
)

// ChangeType enumeration.
const (
Expand Down Expand Up @@ -156,7 +160,7 @@ func (s *Service) clientConfig(c *Cluster) scyllaclient.Config {
}

func (s *Service) discoverAndSetClusterHosts(ctx context.Context, c *Cluster) error {
knownHosts, err := s.discoverClusterHosts(ctx, c)
knownHosts, _, err := s.discoverClusterHosts(ctx, c)
if err != nil {
if errors.Is(err, ErrNoValidKnownHost) {
s.logger.Error(ctx, "There is no single valid known host for the cluster. "+
Expand All @@ -171,55 +175,125 @@ func (s *Service) discoverAndSetClusterHosts(ctx context.Context, c *Cluster) er
return errors.Wrap(s.setKnownHosts(c, knownHosts), "update known_hosts in SM DB")
}

func (s *Service) discoverClusterHosts(ctx context.Context, c *Cluster) ([]string, error) {
var contactPoints []string
const (
discoverClusterHostsTimeout = 5 * time.Second
)

func (s *Service) discoverClusterHosts(ctx context.Context, c *Cluster) (knownHosts, liveHosts []string, err error) {
if c.Host != "" {
contactPoints = append(contactPoints, c.Host) // Go with the designated contact point first
knownHosts, liveHosts, err := s.discoverClusterHostUsingCoordinator(ctx, c, discoverClusterHostsTimeout, c.Host)
if err != nil {
s.logger.Error(ctx, "Couldn't discover hosts using stored coordinator host, proceeding with other known ones",
"coordinator-host", c.Host, "error", err)
} else {
return knownHosts, liveHosts, nil
}
} else {
s.logger.Error(ctx, "Missing --host flag. Using only previously discovered hosts instead", "cluster ID", c.ID)
}
contactPoints = append(contactPoints, c.KnownHosts...) // In case it failed, try to contact previously discovered hosts
if len(c.KnownHosts) < 1 {
return nil, nil, ErrNoValidKnownHost
}

for _, cp := range contactPoints {
if cp == "" {
s.logger.Error(ctx, "Empty contact point", "cluster ID", c.ID, "contact points", contactPoints)
continue
}
wg := sync.WaitGroup{}
type hostsTuple struct {
live, known []string
}
result := make(chan hostsTuple, len(c.KnownHosts))
discoverContext, discoverCancel := context.WithCancel(ctx)
defer discoverCancel()

config := scyllaclient.DefaultConfigWithTimeout(s.timeoutConfig)
if c.Port != 0 {
config.Port = strconv.Itoa(c.Port)
}
config.AuthToken = c.AuthToken
config.Hosts = []string{cp}
for _, cp := range c.KnownHosts {
wg.Add(1)

client, err := scyllaclient.NewClient(config, s.logger.Named("client"))
if err != nil {
s.logger.Error(ctx, "Couldn't connect to contact point", "contact point", cp, "error", err)
continue
}
go func(host string) {
defer wg.Done()

knownHosts, err := s.discoverHosts(ctx, client)
logutil.LogOnError(ctx, s.logger, client.Close, "Couldn't close scylla client")
if err != nil {
s.logger.Error(ctx, "Couldn't discover hosts", "host", cp, "error", err)
continue
}
return knownHosts, nil
knownHosts, liveHosts, err := s.discoverClusterHostUsingCoordinator(discoverContext, c, discoverClusterHostsTimeout, host)
if err != nil {
// Only log if the context hasn't been canceled
if !errors.Is(discoverContext.Err(), context.Canceled) {
s.logger.Error(ctx, "Couldn't discover hosts", "host", host, "error", err)
}
return
}
result <- hostsTuple{
live: liveHosts,
known: knownHosts,
}
}(cp)
}

go func() {
wg.Wait()
close(result)
}()

// Read results until the channel is closed
hosts, ok := <-result
if ok {
return hosts.known, hosts.live, nil
}

// If no valid results, return error
return nil, nil, ErrNoValidKnownHost
}

func (s *Service) discoverClusterHostUsingCoordinator(ctx context.Context, c *Cluster, apiCallTimeout time.Duration,
host string,
) (knownHosts, liveHosts []string, err error) {
config := scyllaclient.DefaultConfigWithTimeout(s.timeoutConfig)
if c.Port != 0 {
config.Port = strconv.Itoa(c.Port)
}
config.Timeout = apiCallTimeout
config.AuthToken = c.AuthToken
config.Hosts = []string{host}

return nil, ErrNoValidKnownHost
client, err := scyllaclient.NewClient(config, s.logger.Named("client"))
if err != nil {
return nil, nil, err
}
defer logutil.LogOnError(ctx, s.logger, client.Close, "Couldn't close scylla client")

liveHosts, err = client.GossiperEndpointLiveGet(ctx)
if err != nil {
return nil, nil, err
}
knownHosts, err = s.discoverHosts(ctx, client, liveHosts)
if err != nil {
return nil, nil, err
}
return knownHosts, liveHosts, nil
}

// discoverHosts returns a list of all hosts sorted by DC speed. This is
// an optimisation for Epsilon-Greedy host pool used internally by
// scyllaclient.Client that makes it use supposedly faster hosts first.
func (s *Service) discoverHosts(ctx context.Context, client *scyllaclient.Client) (hosts []string, err error) {
func (s *Service) discoverHosts(ctx context.Context, client *scyllaclient.Client, liveHosts []string) (hosts []string, err error) {
if len(liveHosts) == 0 {
return nil, ErrNoLiveHostAvailable
}

dcs, err := client.Datacenters(ctx)
if err != nil {
return nil, err
}
closest, err := client.ClosestDC(ctx, dcs)
// remove dead nodes from the map
liveSet := make(map[string]struct{})
for _, host := range liveHosts {
liveSet[host] = struct{}{}
}
filteredDCs := make(map[string][]string)
for dc, hosts := range dcs {
for _, host := range hosts {
if _, isLive := liveSet[host]; isLive {
filteredDCs[dc] = append(filteredDCs[dc], host)
}
}
}

closest, err := client.ClosestDC(ctx, filteredDCs)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -397,7 +471,7 @@ func (s *Service) PutCluster(ctx context.Context, c *Cluster) (err error) {
}

// Check hosts connectivity.
if err := s.validateHostsConnectivity(ctx, c); err != nil {
if err := s.ValidateHostsConnectivity(ctx, c); err != nil {
var tip string
switch scyllaclient.StatusCodeOf(err) {
case 0:
Expand Down Expand Up @@ -487,36 +561,34 @@ func (s *Service) PutCluster(ctx context.Context, c *Cluster) (err error) {
return s.notifyChangeListener(ctx, changeEvent)
}

func (s *Service) validateHostsConnectivity(ctx context.Context, c *Cluster) error {
// ValidateHostsConnectivity validates that scylla manager agent API is available and responding on all live hosts.
// Hosts are discovered using cluster.host + cluster.knownHosts saved to the manager's database.
func (s *Service) ValidateHostsConnectivity(ctx context.Context, c *Cluster) error {
if err := s.loadKnownHosts(c); err != nil && !errors.Is(err, gocql.ErrNotFound) {
return errors.Wrap(err, "load known hosts")
}

knownHosts, err := s.discoverClusterHosts(ctx, c)
knownHosts, liveHosts, err := s.discoverClusterHosts(ctx, c)
if err != nil {
return errors.Wrap(err, "discover cluster hosts")
}
c.KnownHosts = knownHosts

if len(liveHosts) == 0 {
return util.ErrValidate(errors.New("no live nodes"))
}

config := s.clientConfig(c)
config.Hosts = liveHosts
client, err := scyllaclient.NewClient(config, s.logger.Named("client"))
if err != nil {
return err
}
defer logutil.LogOnError(ctx, s.logger, client.Close, "Couldn't close scylla client")

status, err := client.Status(ctx)
if err != nil {
return errors.Wrap(err, "cluster status")
}
live := status.Live().Hosts()
if len(live) == 0 {
return util.ErrValidate(errors.New("no live nodes"))
}

var errs error
for i, err := range client.CheckHostsConnectivity(ctx, live) {
errs = multierr.Append(errs, errors.Wrap(err, live[i]))
for i, err := range client.CheckHostsConnectivity(ctx, liveHosts) {
errs = multierr.Append(errs, errors.Wrap(err, liveHosts[i]))
}
if errs != nil {
return util.ErrValidate(errors.Wrap(errs, "connectivity check"))
Expand Down
Loading

0 comments on commit f63ec83

Please sign in to comment.