From 065335febfdf84959a66542cb9efcc73a821f832 Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 22 Oct 2024 15:38:36 +0200 Subject: [PATCH 1/8] 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. --- pkg/cmd/scylla-manager/server.go | 2 +- pkg/scyllaclient/client_scylla.go | 9 + pkg/service/cluster/service.go | 163 +++++++++++++----- .../cluster/service_integration_test.go | 115 ++++++++++++ .../healthcheck/service_integration_test.go | 32 ++++ pkg/testutils/exec.go | 39 +++++ 6 files changed, 313 insertions(+), 47 deletions(-) diff --git a/pkg/cmd/scylla-manager/server.go b/pkg/cmd/scylla-manager/server.go index 8c9a40c22..580683132 100644 --- a/pkg/cmd/scylla-manager/server.go +++ b/pkg/cmd/scylla-manager/server.go @@ -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) { diff --git a/pkg/scyllaclient/client_scylla.go b/pkg/scyllaclient/client_scylla.go index 193fb9e15..03210b83f 100644 --- a/pkg/scyllaclient/client_scylla.go +++ b/pkg/scyllaclient/client_scylla.go @@ -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 diff --git a/pkg/service/cluster/service.go b/pkg/service/cluster/service.go index 353d67fea..160393009 100644 --- a/pkg/service/cluster/service.go +++ b/pkg/service/cluster/service.go @@ -9,6 +9,7 @@ import ( "fmt" "sort" "strconv" + "sync" "time" "github.com/gocql/gocql" @@ -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 ( @@ -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. "+ @@ -171,55 +175,124 @@ 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 + for hosts := range result { + 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 } @@ -397,7 +470,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: @@ -487,36 +560,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")) diff --git a/pkg/service/cluster/service_integration_test.go b/pkg/service/cluster/service_integration_test.go index e70c6f18e..551ef6965 100644 --- a/pkg/service/cluster/service_integration_test.go +++ b/pkg/service/cluster/service_integration_test.go @@ -33,6 +33,121 @@ import ( "github.com/scylladb/scylla-manager/v3/pkg/util/uuid" ) +func TestValidateHostConnectivityIntegration(t *testing.T) { + Print("given: the fresh cluster") + var ( + ctx = context.Background() + session = CreateScyllaManagerDBSession(t) + secretsStore = store.NewTableStore(session, table.Secrets) + c = &cluster.Cluster{ + AuthToken: "token", + Host: ManagedClusterHost(), + } + ) + s, err := cluster.NewService(session, metrics.NewClusterMetrics(), secretsStore, scyllaclient.DefaultTimeoutConfig(), + server.DefaultConfig().ClientCacheTimeout, log.NewDevelopment()) + if err != nil { + t.Fatal(err) + } + + err = s.PutCluster(context.Background(), c) + if err != nil { + t.Fatal(err) + } + + allHosts := ManagedClusterHosts() + for _, tc := range []struct { + name string + hostsDown []string + result error + timeout time.Duration + }{ + { + name: "coordinator host is DOWN", + hostsDown: []string{ManagedClusterHost()}, + result: nil, + timeout: 6 * time.Second, + }, + { + name: "only one is UP", + hostsDown: allHosts[:len(allHosts)-1], + result: nil, + timeout: 6 * time.Second, + }, + { + name: "all hosts are DOWN", + hostsDown: allHosts, + result: cluster.ErrNoValidKnownHost, + timeout: 11 * time.Second, // the 5 seconds calls will timeout twice + }, + { + name: "all hosts are UP", + hostsDown: nil, + result: nil, + timeout: 6 * time.Second, + }, + } { + t.Run(tc.name, func(t *testing.T) { + defer func() { + for _, host := range tc.hostsDown { + if err := StartService(host, "scylla"); err != nil { + t.Logf("error on starting stopped scylla service on host={%s}, err={%s}", host, err) + } + if err := RunIptablesCommand(host, CmdUnblockScyllaREST); err != nil { + t.Logf("error trying to unblock REST API on host = {%s}, err={%s}", host, err) + } + } + }() + + Printf("then: validate that call to validate host connectivity takes less than %v seconds", tc.timeout.Seconds()) + testCluster, err := s.GetClusterByID(context.Background(), c.ID) + if err != nil { + t.Fatal(err) + } + if err := callValidateHostConnectivityWithTimeout(ctx, s, tc.timeout, testCluster); err != nil { + t.Fatal(err) + } + Printf("when: the scylla service is stopped and the scylla API is timing out on some hosts") + // It's needed to block Scylla REST API, so that the clients are just hanging when they call the API. + // Scylla service must be stopped to make the node to report DOWN status. Blocking REST API is not + // enough. + for _, host := range tc.hostsDown { + if err := StopService(host, "scylla"); err != nil { + t.Fatal(err) + } + if err := RunIptablesCommand(host, CmdBlockScyllaREST); err != nil { + t.Error(err) + } + } + + Printf("then: validate that call still takes less than %v seconds", tc.timeout.Seconds()) + if err := callValidateHostConnectivityWithTimeout(ctx, s, tc.timeout, testCluster); !errors.Is(err, tc.result) { + t.Fatal(err) + } + }) + } +} + +func callValidateHostConnectivityWithTimeout(ctx context.Context, s *cluster.Service, timeout time.Duration, + c *cluster.Cluster) error { + + callCtx, cancel := context.WithCancel(ctx) + defer cancel() + + done := make(chan error) + go func() { + done <- s.ValidateHostsConnectivity(callCtx, c) + }() + + select { + case <-time.After(timeout): + cancel() + return fmt.Errorf("expected s.ValidateHostsConnectivity to complete in less than %v seconds, time exceeded", timeout.Seconds()) + case err := <-done: + return err + } +} + func TestClientIntegration(t *testing.T) { expectedHosts := ManagedClusterHosts() diff --git a/pkg/service/healthcheck/service_integration_test.go b/pkg/service/healthcheck/service_integration_test.go index 4744e2414..2cd1b9e32 100644 --- a/pkg/service/healthcheck/service_integration_test.go +++ b/pkg/service/healthcheck/service_integration_test.go @@ -13,6 +13,7 @@ import ( "net/http" "os" "strings" + "sync" "testing" "time" @@ -22,6 +23,7 @@ import ( "github.com/scylladb/scylla-manager/v3/pkg/metrics" "github.com/scylladb/scylla-manager/v3/pkg/service/cluster" "github.com/scylladb/scylla-manager/v3/pkg/service/configcache" + "go.uber.org/multierr" "go.uber.org/zap/zapcore" "github.com/scylladb/scylla-manager/v3/pkg/schema/table" @@ -53,6 +55,9 @@ func TestStatus_Ping_Independent_From_REST_Integration(t *testing.T) { tryUnblockREST(t, ManagedClusterHosts()) tryUnblockAlternator(t, ManagedClusterHosts()) tryStartAgent(t, ManagedClusterHosts()) + if err := ensureNodesAreUP(t, ManagedClusterHosts(), time.Minute); err != nil { + t.Fatalf("not all nodes are UP, err = {%v}", err) + } logger := log.NewDevelopmentWithLevel(zapcore.InfoLevel).Named("healthcheck") @@ -562,6 +567,33 @@ func tryStartAgent(t *testing.T, hosts []string) { } } +func ensureNodesAreUP(t *testing.T, hosts []string, timeout time.Duration) error { + t.Helper() + + var ( + allErrors error + mu sync.Mutex + ) + + wg := sync.WaitGroup{} + for _, host := range hosts { + wg.Add(1) + + go func(h string) { + defer wg.Done() + + if err := WaitForNodeUPOrTimeout(h, timeout); err != nil { + mu.Lock() + allErrors = multierr.Combine(allErrors, err) + mu.Unlock() + } + }(host) + } + wg.Wait() + + return allErrors +} + const pingPath = "/storage_service/scylla_release_version" func fakeHealthCheckStatus(host string, code int) http.RoundTripper { diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index 624dea384..745516106 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -4,8 +4,10 @@ package testutils import ( "bytes" + "fmt" "net" "strings" + "time" "github.com/pkg/errors" "golang.org/x/crypto/ssh" @@ -107,3 +109,40 @@ func StartService(h, service string) error { } return nil } + +// WaitForNodeUPOrTimeout waits until nodetool status report UN status for the given node. +// The nodetool status CLI is executed on the same node. +func WaitForNodeUPOrTimeout(h string, timeout time.Duration) error { + nodeIsReady := make(chan struct{}) + done := make(chan struct{}) + go func() { + defer close(nodeIsReady) + for { + select { + case <-done: + return + default: + stdout, _, err := ExecOnHost(h, "nodetool status | grep "+h) + if err != nil { + continue + } + if strings.HasPrefix(stdout, "UN") { + return + } + select { + case <-done: + return + case <-time.After(time.Second): + } + } + } + }() + + select { + case <-nodeIsReady: + return nil + case <-time.After(timeout): + close(done) + return fmt.Errorf("node %s haven't reach UP status", h) + } +} From 602806980968cd6850065cf54b85423347ecc57e Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 26 Nov 2024 17:57:19 +0100 Subject: [PATCH 2/8] review fix --- .../cluster/service_integration_test.go | 7 + .../healthcheck/service_integration_test.go | 173 +++--------------- pkg/testutils/exec.go | 151 +++++++++++++++ 3 files changed, 183 insertions(+), 148 deletions(-) diff --git a/pkg/service/cluster/service_integration_test.go b/pkg/service/cluster/service_integration_test.go index 551ef6965..5c0572a54 100644 --- a/pkg/service/cluster/service_integration_test.go +++ b/pkg/service/cluster/service_integration_test.go @@ -98,6 +98,13 @@ func TestValidateHostConnectivityIntegration(t *testing.T) { } } }() + TryUnblockCQL(t, ManagedClusterHosts()) + TryUnblockREST(t, ManagedClusterHosts()) + TryUnblockAlternator(t, ManagedClusterHosts()) + TryStartAgent(t, ManagedClusterHosts()) + if err := EnsureNodesAreUP(t, ManagedClusterHosts(), time.Minute); err != nil { + t.Fatalf("not all nodes are UP, err = {%v}", err) + } Printf("then: validate that call to validate host connectivity takes less than %v seconds", tc.timeout.Seconds()) testCluster, err := s.GetClusterByID(context.Background(), c.ID) diff --git a/pkg/service/healthcheck/service_integration_test.go b/pkg/service/healthcheck/service_integration_test.go index 2cd1b9e32..894807bbc 100644 --- a/pkg/service/healthcheck/service_integration_test.go +++ b/pkg/service/healthcheck/service_integration_test.go @@ -13,7 +13,6 @@ import ( "net/http" "os" "strings" - "sync" "testing" "time" @@ -23,7 +22,6 @@ import ( "github.com/scylladb/scylla-manager/v3/pkg/metrics" "github.com/scylladb/scylla-manager/v3/pkg/service/cluster" "github.com/scylladb/scylla-manager/v3/pkg/service/configcache" - "go.uber.org/multierr" "go.uber.org/zap/zapcore" "github.com/scylladb/scylla-manager/v3/pkg/schema/table" @@ -51,11 +49,11 @@ func TestStatus_Ping_Independent_From_REST_Integration(t *testing.T) { } // Given - tryUnblockCQL(t, ManagedClusterHosts()) - tryUnblockREST(t, ManagedClusterHosts()) - tryUnblockAlternator(t, ManagedClusterHosts()) - tryStartAgent(t, ManagedClusterHosts()) - if err := ensureNodesAreUP(t, ManagedClusterHosts(), time.Minute); err != nil { + TryUnblockCQL(t, ManagedClusterHosts()) + TryUnblockREST(t, ManagedClusterHosts()) + TryUnblockAlternator(t, ManagedClusterHosts()) + TryStartAgent(t, ManagedClusterHosts()) + if err := EnsureNodesAreUP(t, ManagedClusterHosts(), time.Minute); err != nil { t.Fatalf("not all nodes are UP, err = {%v}", err) } @@ -122,8 +120,8 @@ func TestStatus_Ping_Independent_From_REST_Integration(t *testing.T) { } // When #2 -> one of the hosts has unresponsive REST API - defer unblockREST(t, hostWithUnresponsiveREST) - blockREST(t, hostWithUnresponsiveREST) + defer UnblockREST(t, hostWithUnresponsiveREST) + BlockREST(t, hostWithUnresponsiveREST) // Then #2 -> only REST ping fails, CQL and Alternator are fine status, err = healthSvc.Status(context.Background(), testCluster.ID) @@ -216,16 +214,16 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster // Tests here do not test the dynamic t/o functionality c := DefaultConfig() - tryUnblockCQL(t, ManagedClusterHosts()) - tryUnblockREST(t, ManagedClusterHosts()) - tryUnblockAlternator(t, ManagedClusterHosts()) - tryStartAgent(t, ManagedClusterHosts()) + TryUnblockCQL(t, ManagedClusterHosts()) + TryUnblockREST(t, ManagedClusterHosts()) + TryUnblockAlternator(t, ManagedClusterHosts()) + TryStartAgent(t, ManagedClusterHosts()) defer func() { - tryUnblockCQL(t, ManagedClusterHosts()) - tryUnblockREST(t, ManagedClusterHosts()) - tryUnblockAlternator(t, ManagedClusterHosts()) - tryStartAgent(t, ManagedClusterHosts()) + TryUnblockCQL(t, ManagedClusterHosts()) + TryUnblockREST(t, ManagedClusterHosts()) + TryUnblockAlternator(t, ManagedClusterHosts()) + TryStartAgent(t, ManagedClusterHosts()) }() hrt := NewHackableRoundTripper(scyllaclient.DefaultTransport()) @@ -288,8 +286,8 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster t.Run("node REST TIMEOUT", func(t *testing.T) { host := IPFromTestNet("12") - blockREST(t, host) - defer unblockREST(t, host) + BlockREST(t, host) + defer UnblockREST(t, host) status, err := s.Status(context.Background(), clusterID) if err != nil { @@ -314,8 +312,8 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster t.Run("node CQL TIMEOUT", func(t *testing.T) { host := IPFromTestNet("12") - blockCQL(t, host, sslEnabled) - defer unblockCQL(t, host, sslEnabled) + BlockCQL(t, host, sslEnabled) + defer UnblockCQL(t, host, sslEnabled) status, err := s.Status(context.Background(), clusterID) if err != nil { @@ -340,8 +338,8 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster t.Run("node Alternator TIMEOUT", func(t *testing.T) { host := IPFromTestNet("12") - blockAlternator(t, host) - defer unblockAlternator(t, host) + BlockAlternator(t, host) + defer UnblockAlternator(t, host) status, err := s.Status(context.Background(), clusterID) if err != nil { @@ -366,8 +364,8 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster t.Run("node REST DOWN", func(t *testing.T) { host := IPFromTestNet("12") - stopAgent(t, host) - defer startAgent(t, host) + StopAgent(t, host) + defer StartAgent(t, host) status, err := s.Status(context.Background(), clusterID) if err != nil { @@ -445,11 +443,11 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster defer cancel() for _, h := range ManagedClusterHosts() { - blockREST(t, h) + BlockREST(t, h) } defer func() { for _, h := range ManagedClusterHosts() { - unblockREST(t, h) + UnblockREST(t, h) } }() @@ -473,127 +471,6 @@ func testStatusIntegration(t *testing.T, clusterID uuid.UUID, clusterSvc cluster }) } -func blockREST(t *testing.T, h string) { - t.Helper() - if err := RunIptablesCommand(h, CmdBlockScyllaREST); err != nil { - t.Error(err) - } -} - -func unblockREST(t *testing.T, h string) { - t.Helper() - if err := RunIptablesCommand(h, CmdUnblockScyllaREST); err != nil { - t.Error(err) - } -} - -func tryUnblockREST(t *testing.T, hosts []string) { - t.Helper() - for _, host := range hosts { - _ = RunIptablesCommand(host, CmdUnblockScyllaREST) - } -} - -func blockCQL(t *testing.T, h string, sslEnabled bool) { - t.Helper() - cmd := CmdBlockScyllaCQL - if sslEnabled { - cmd = CmdBlockScyllaCQLSSL - } - if err := RunIptablesCommand(h, cmd); err != nil { - t.Error(err) - } -} - -func unblockCQL(t *testing.T, h string, sslEnabled bool) { - t.Helper() - cmd := CmdUnblockScyllaCQL - if sslEnabled { - cmd = CmdUnblockScyllaCQLSSL - } - if err := RunIptablesCommand(h, cmd); err != nil { - t.Error(err) - } -} - -func tryUnblockCQL(t *testing.T, hosts []string) { - t.Helper() - for _, host := range hosts { - _ = RunIptablesCommand(host, CmdUnblockScyllaCQL) - } -} - -func blockAlternator(t *testing.T, h string) { - t.Helper() - if err := RunIptablesCommand(h, CmdBlockScyllaAlternator); err != nil { - t.Error(err) - } -} - -func unblockAlternator(t *testing.T, h string) { - t.Helper() - if err := RunIptablesCommand(h, CmdUnblockScyllaAlternator); err != nil { - t.Error(err) - } -} - -func tryUnblockAlternator(t *testing.T, hosts []string) { - t.Helper() - for _, host := range hosts { - _ = RunIptablesCommand(host, CmdUnblockScyllaAlternator) - } -} - -const agentService = "scylla-manager-agent" - -func stopAgent(t *testing.T, h string) { - t.Helper() - if err := StopService(h, agentService); err != nil { - t.Error(err) - } -} - -func startAgent(t *testing.T, h string) { - t.Helper() - if err := StartService(h, agentService); err != nil { - t.Error(err) - } -} - -func tryStartAgent(t *testing.T, hosts []string) { - t.Helper() - for _, host := range hosts { - _ = StartService(host, agentService) - } -} - -func ensureNodesAreUP(t *testing.T, hosts []string, timeout time.Duration) error { - t.Helper() - - var ( - allErrors error - mu sync.Mutex - ) - - wg := sync.WaitGroup{} - for _, host := range hosts { - wg.Add(1) - - go func(h string) { - defer wg.Done() - - if err := WaitForNodeUPOrTimeout(h, timeout); err != nil { - mu.Lock() - allErrors = multierr.Combine(allErrors, err) - mu.Unlock() - } - }(host) - } - wg.Wait() - - return allErrors -} - const pingPath = "/storage_service/scylla_release_version" func fakeHealthCheckStatus(host string, code int) http.RoundTripper { diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index 745516106..c5b880588 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -7,9 +7,12 @@ import ( "fmt" "net" "strings" + "sync" + "testing" "time" "github.com/pkg/errors" + "go.uber.org/multierr" "golang.org/x/crypto/ssh" ) @@ -146,3 +149,151 @@ func WaitForNodeUPOrTimeout(h string, timeout time.Duration) error { return fmt.Errorf("node %s haven't reach UP status", h) } } + +// BlockREST blocks the Scylla API ports on h machine by dropping TCP packets. +func BlockREST(t *testing.T, h string) { + t.Helper() + if err := RunIptablesCommand(h, CmdBlockScyllaREST); err != nil { + t.Error(err) + } +} + +// UnblockREST unblocks the Scylla API ports on []hosts machines. +func UnblockREST(t *testing.T, h string) { + t.Helper() + if err := RunIptablesCommand(h, CmdUnblockScyllaREST); err != nil { + t.Error(err) + } +} + +// TryUnblockREST tries to unblock the Scylla API ports on []hosts machines. +// Logs an error if the execution failed, but doesn't return it. +func TryUnblockREST(t *testing.T, hosts []string) { + t.Helper() + for _, host := range hosts { + if err := RunIptablesCommand(host, CmdUnblockScyllaREST); err != nil { + t.Log(err) + } + } +} + +// BlockCQL blocks the CQL ports on h machine by dropping TCP packets. +func BlockCQL(t *testing.T, h string, sslEnabled bool) { + t.Helper() + cmd := CmdBlockScyllaCQL + if sslEnabled { + cmd = CmdBlockScyllaCQLSSL + } + if err := RunIptablesCommand(h, cmd); err != nil { + t.Error(err) + } +} + +// UnblockCQL unblocks the CQL ports on []hosts machines. +func UnblockCQL(t *testing.T, h string, sslEnabled bool) { + t.Helper() + cmd := CmdUnblockScyllaCQL + if sslEnabled { + cmd = CmdUnblockScyllaCQLSSL + } + if err := RunIptablesCommand(h, cmd); err != nil { + t.Error(err) + } +} + +// TryUnblockCQL tries to unblock the CQL ports on []hosts machines. +// Logs an error if the execution failed, but doesn't return it. +func TryUnblockCQL(t *testing.T, hosts []string) { + t.Helper() + for _, host := range hosts { + if err := RunIptablesCommand(host, CmdUnblockScyllaCQL); err != nil { + t.Log(err) + } + } +} + +// BlockAlternator blocks the Scylla Alternator ports on h machine by dropping TCP packets. +func BlockAlternator(t *testing.T, h string) { + t.Helper() + if err := RunIptablesCommand(h, CmdBlockScyllaAlternator); err != nil { + t.Error(err) + } +} + +// UnblockAlternator unblocks the Alternator ports on []hosts machines. +func UnblockAlternator(t *testing.T, h string) { + t.Helper() + if err := RunIptablesCommand(h, CmdUnblockScyllaAlternator); err != nil { + t.Error(err) + } +} + +// TryUnblockAlternator tries to unblock the Alternator API ports on []hosts machines. +// Logs an error if the execution failed, but doesn't return it. +func TryUnblockAlternator(t *testing.T, hosts []string) { + t.Helper() + for _, host := range hosts { + if err := RunIptablesCommand(host, CmdUnblockScyllaAlternator); err != nil { + t.Log(err) + } + } +} + +const agentService = "scylla-manager-agent" + +// StopAgent stops scylla-manager-agent service on the h machine. +func StopAgent(t *testing.T, h string) { + t.Helper() + if err := StopService(h, agentService); err != nil { + t.Error(err) + } +} + +// StartAgent starts scylla-manager-agent service on the h machine. +func StartAgent(t *testing.T, h string) { + t.Helper() + if err := StartService(h, agentService); err != nil { + t.Error(err) + } +} + +// TryStartAgent tries to start scylla-manager-agent service on the []hosts machines. +// It logs an error on failures, but doesn't return it. +func TryStartAgent(t *testing.T, hosts []string) { + t.Helper() + for _, host := range hosts { + if err := StartService(host, agentService); err != nil { + t.Log(err) + } + } +} + +// EnsureNodesAreUP validates if scylla-service is up and running on every []hosts and nodes are reporting their status +// correctly via `nodetool status` command. +// It waits for each node to report UN status for the duration specified in timeout parameter. +func EnsureNodesAreUP(t *testing.T, hosts []string, timeout time.Duration) error { + t.Helper() + + var ( + allErrors error + mu sync.Mutex + ) + + wg := sync.WaitGroup{} + for _, host := range hosts { + wg.Add(1) + + go func(h string) { + defer wg.Done() + + if err := WaitForNodeUPOrTimeout(h, timeout); err != nil { + mu.Lock() + allErrors = multierr.Combine(allErrors, err) + mu.Unlock() + } + }(host) + } + wg.Wait() + + return allErrors +} From f679981d2b87adc7d6196eec0a1a387ae2be85e9 Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 26 Nov 2024 18:15:03 +0100 Subject: [PATCH 3/8] review fix --- pkg/service/cluster/service.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/service/cluster/service.go b/pkg/service/cluster/service.go index 160393009..e06225c84 100644 --- a/pkg/service/cluster/service.go +++ b/pkg/service/cluster/service.go @@ -230,11 +230,12 @@ func (s *Service) discoverClusterHosts(ctx context.Context, c *Cluster) (knownHo }() // Read results until the channel is closed - for hosts := range result { + hosts, ok := <-result + if ok { return hosts.known, hosts.live, nil } - // If no valid results, return error< + // If no valid results, return error return nil, nil, ErrNoValidKnownHost } From 62525b1d7a5c12a69854c74a8961c84557cca8a4 Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 26 Nov 2024 18:45:28 +0100 Subject: [PATCH 4/8] review fix --- pkg/scyllaclient/retry_integration_test.go | 8 ++--- .../backup/service_backup_integration_test.go | 4 +-- .../cluster/service_integration_test.go | 8 ++--- pkg/testutils/exec.go | 32 +++++++++++-------- pkg/testutils/netwait_integration_test.go | 4 +-- 5 files changed, 30 insertions(+), 26 deletions(-) diff --git a/pkg/scyllaclient/retry_integration_test.go b/pkg/scyllaclient/retry_integration_test.go index 5e37b9996..db81d62c5 100644 --- a/pkg/scyllaclient/retry_integration_test.go +++ b/pkg/scyllaclient/retry_integration_test.go @@ -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) } }) @@ -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 } @@ -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 } diff --git a/pkg/service/backup/service_backup_integration_test.go b/pkg/service/backup/service_backup_integration_test.go index 610ab0f63..0b25c39aa 100644 --- a/pkg/service/backup/service_backup_integration_test.go +++ b/pkg/service/backup/service_backup_integration_test.go @@ -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{ diff --git a/pkg/service/cluster/service_integration_test.go b/pkg/service/cluster/service_integration_test.go index 5c0572a54..8d478c485 100644 --- a/pkg/service/cluster/service_integration_test.go +++ b/pkg/service/cluster/service_integration_test.go @@ -93,7 +93,7 @@ func TestValidateHostConnectivityIntegration(t *testing.T) { if err := StartService(host, "scylla"); err != nil { t.Logf("error on starting stopped scylla service on host={%s}, err={%s}", host, err) } - if err := RunIptablesCommand(host, CmdUnblockScyllaREST); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaREST); err != nil { t.Logf("error trying to unblock REST API on host = {%s}, err={%s}", host, err) } } @@ -122,7 +122,7 @@ func TestValidateHostConnectivityIntegration(t *testing.T) { if err := StopService(host, "scylla"); err != nil { t.Fatal(err) } - if err := RunIptablesCommand(host, CmdBlockScyllaREST); err != nil { + if err := RunIptablesCommand(t, host, CmdBlockScyllaREST); err != nil { t.Error(err) } } @@ -675,10 +675,10 @@ func TestServiceStorageIntegration(t *testing.T) { c := validCluster() c.Host = h1 - if err := RunIptablesCommand(h2, CmdBlockScyllaREST); err != nil { + if err := RunIptablesCommand(t, h2, CmdBlockScyllaREST); err != nil { t.Fatal(err) } - defer RunIptablesCommand(h2, CmdUnblockScyllaREST) + defer RunIptablesCommand(t, h2, CmdUnblockScyllaREST) if err := s.PutCluster(ctx, c); err == nil { t.Fatal("expected put cluster to fail because of connectivity issues") diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index c5b880588..2a76d1131 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -47,16 +47,20 @@ func makeIPV6Rule(rule string) string { } // RunIptablesCommand executes iptables command, repeats same command for IPV6 iptables rule. -func RunIptablesCommand(host, cmd string) error { +func RunIptablesCommand(t *testing.T, host, cmd string) error { + t.Helper() if IsIPV6Network() { - return ExecOnHostStatus(host, makeIPV6Rule(cmd)) + return ExecOnHostStatus(t, host, makeIPV6Rule(cmd)) } - return ExecOnHostStatus(host, cmd) + return ExecOnHostStatus(t, host, cmd) } // ExecOnHostStatus executes the given command on the given host and returns on error. -func ExecOnHostStatus(host, cmd string) error { - _, _, err := ExecOnHost(host, cmd) +func ExecOnHostStatus(t *testing.T, host, cmd string) error { + stdOut, stdErr, err := ExecOnHost(host, cmd) + if err != nil { + t.Logf("cnd: {%s}, stdout: {%s}, stderr: {%s}", cmd, stdOut, stdErr) + } return errors.Wrapf(err, "run command %s", cmd) } @@ -153,7 +157,7 @@ func WaitForNodeUPOrTimeout(h string, timeout time.Duration) error { // BlockREST blocks the Scylla API ports on h machine by dropping TCP packets. func BlockREST(t *testing.T, h string) { t.Helper() - if err := RunIptablesCommand(h, CmdBlockScyllaREST); err != nil { + if err := RunIptablesCommand(t, h, CmdBlockScyllaREST); err != nil { t.Error(err) } } @@ -161,7 +165,7 @@ func BlockREST(t *testing.T, h string) { // UnblockREST unblocks the Scylla API ports on []hosts machines. func UnblockREST(t *testing.T, h string) { t.Helper() - if err := RunIptablesCommand(h, CmdUnblockScyllaREST); err != nil { + if err := RunIptablesCommand(t, h, CmdUnblockScyllaREST); err != nil { t.Error(err) } } @@ -171,7 +175,7 @@ func UnblockREST(t *testing.T, h string) { func TryUnblockREST(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(host, CmdUnblockScyllaREST); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaREST); err != nil { t.Log(err) } } @@ -184,7 +188,7 @@ func BlockCQL(t *testing.T, h string, sslEnabled bool) { if sslEnabled { cmd = CmdBlockScyllaCQLSSL } - if err := RunIptablesCommand(h, cmd); err != nil { + if err := RunIptablesCommand(t, h, cmd); err != nil { t.Error(err) } } @@ -196,7 +200,7 @@ func UnblockCQL(t *testing.T, h string, sslEnabled bool) { if sslEnabled { cmd = CmdUnblockScyllaCQLSSL } - if err := RunIptablesCommand(h, cmd); err != nil { + if err := RunIptablesCommand(t, h, cmd); err != nil { t.Error(err) } } @@ -206,7 +210,7 @@ func UnblockCQL(t *testing.T, h string, sslEnabled bool) { func TryUnblockCQL(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(host, CmdUnblockScyllaCQL); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaCQL); err != nil { t.Log(err) } } @@ -215,7 +219,7 @@ func TryUnblockCQL(t *testing.T, hosts []string) { // BlockAlternator blocks the Scylla Alternator ports on h machine by dropping TCP packets. func BlockAlternator(t *testing.T, h string) { t.Helper() - if err := RunIptablesCommand(h, CmdBlockScyllaAlternator); err != nil { + if err := RunIptablesCommand(t, h, CmdBlockScyllaAlternator); err != nil { t.Error(err) } } @@ -223,7 +227,7 @@ func BlockAlternator(t *testing.T, h string) { // UnblockAlternator unblocks the Alternator ports on []hosts machines. func UnblockAlternator(t *testing.T, h string) { t.Helper() - if err := RunIptablesCommand(h, CmdUnblockScyllaAlternator); err != nil { + if err := RunIptablesCommand(t, h, CmdUnblockScyllaAlternator); err != nil { t.Error(err) } } @@ -233,7 +237,7 @@ func UnblockAlternator(t *testing.T, h string) { func TryUnblockAlternator(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(host, CmdUnblockScyllaAlternator); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaAlternator); err != nil { t.Log(err) } } diff --git a/pkg/testutils/netwait_integration_test.go b/pkg/testutils/netwait_integration_test.go index 0ce873d46..5f95d6818 100644 --- a/pkg/testutils/netwait_integration_test.go +++ b/pkg/testutils/netwait_integration_test.go @@ -22,11 +22,11 @@ func TestWaiterTimeoutIntegration(t *testing.T) { } host := ManagedClusterHost() - err := RunIptablesCommand(host, CmdBlockScyllaCQL) + err := RunIptablesCommand(t, host, CmdBlockScyllaCQL) if err != nil { t.Fatal(err) } - defer RunIptablesCommand(host, CmdUnblockScyllaCQL) + defer RunIptablesCommand(t, host, CmdUnblockScyllaCQL) w := &netwait.Waiter{ DialTimeout: 5 * time.Millisecond, From 21f96ee90f3f01f324bf8f84717d1c491a6fe17d Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 26 Nov 2024 18:55:47 +0100 Subject: [PATCH 5/8] review fix --- pkg/testutils/exec.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index 2a76d1131..ec26d0be3 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -57,6 +57,7 @@ func RunIptablesCommand(t *testing.T, host, cmd string) error { // ExecOnHostStatus executes the given command on the given host and returns on error. func ExecOnHostStatus(t *testing.T, host, cmd string) error { + t.Helper() stdOut, stdErr, err := ExecOnHost(host, cmd) if err != nil { t.Logf("cnd: {%s}, stdout: {%s}, stderr: {%s}", cmd, stdOut, stdErr) From 54c5fd91ee687bc62dab3bd03378c569125b8953 Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Tue, 26 Nov 2024 19:29:16 +0100 Subject: [PATCH 6/8] review fix --- pkg/testutils/exec.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index ec26d0be3..909fbf9dc 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -40,6 +40,9 @@ const ( // CmdUnblockScyllaAlternator defines the command used for unblocking the Scylla Alternator access. CmdUnblockScyllaAlternator = "iptables -D INPUT -p tcp --destination-port 8000 -j DROP" + + // CmdOrTrueAppend let to accept shell command failure and proceed + CmdOrTrueAppend = " || true" ) func makeIPV6Rule(rule string) string { @@ -176,7 +179,7 @@ func UnblockREST(t *testing.T, h string) { func TryUnblockREST(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(t, host, CmdUnblockScyllaREST); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaREST+CmdOrTrueAppend); err != nil { t.Log(err) } } @@ -211,7 +214,7 @@ func UnblockCQL(t *testing.T, h string, sslEnabled bool) { func TryUnblockCQL(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(t, host, CmdUnblockScyllaCQL); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaCQL+CmdOrTrueAppend); err != nil { t.Log(err) } } @@ -238,7 +241,7 @@ func UnblockAlternator(t *testing.T, h string) { func TryUnblockAlternator(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := RunIptablesCommand(t, host, CmdUnblockScyllaAlternator); err != nil { + if err := RunIptablesCommand(t, host, CmdUnblockScyllaAlternator+CmdOrTrueAppend); err != nil { t.Log(err) } } @@ -267,7 +270,7 @@ func StartAgent(t *testing.T, h string) { func TryStartAgent(t *testing.T, hosts []string) { t.Helper() for _, host := range hosts { - if err := StartService(host, agentService); err != nil { + if err := StartService(host, agentService+CmdOrTrueAppend); err != nil { t.Log(err) } } From f39b7fdc66326ee8706b227929f4b77d6861dd52 Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Wed, 27 Nov 2024 10:42:32 +0100 Subject: [PATCH 7/8] review fix --- pkg/service/cluster/service_integration_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/service/cluster/service_integration_test.go b/pkg/service/cluster/service_integration_test.go index 8d478c485..71a93deef 100644 --- a/pkg/service/cluster/service_integration_test.go +++ b/pkg/service/cluster/service_integration_test.go @@ -34,6 +34,10 @@ import ( ) func TestValidateHostConnectivityIntegration(t *testing.T) { + if IsIPV6Network() { + t.Skip("DB node do not have ip6tables and related modules to make it work properly") + } + Print("given: the fresh cluster") var ( ctx = context.Background() From 964f87077de54f617f0582bc7df097fddf9a31df Mon Sep 17 00:00:00 2001 From: Karol Kokoszka Date: Wed, 27 Nov 2024 10:50:47 +0100 Subject: [PATCH 8/8] review fix --- pkg/testutils/exec.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/testutils/exec.go b/pkg/testutils/exec.go index 909fbf9dc..556ca8b1c 100644 --- a/pkg/testutils/exec.go +++ b/pkg/testutils/exec.go @@ -41,7 +41,7 @@ const ( // CmdUnblockScyllaAlternator defines the command used for unblocking the Scylla Alternator access. CmdUnblockScyllaAlternator = "iptables -D INPUT -p tcp --destination-port 8000 -j DROP" - // CmdOrTrueAppend let to accept shell command failure and proceed + // CmdOrTrueAppend let to accept shell command failure and proceed. CmdOrTrueAppend = " || true" )