Skip to content

Commit

Permalink
Merge pull request #5487 from andrwng/cluster_features_test
Browse files Browse the repository at this point in the history
cluster_features_test: use RedpandaInstaller
  • Loading branch information
andrwng committed Jul 19, 2022
2 parents 2a6861d + 65a6a4c commit 8ee1811
Showing 1 changed file with 113 additions and 98 deletions.
211 changes: 113 additions & 98 deletions tests/rptest/tests/cluster_features_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,18 @@
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.cluster import cluster
from ducktape.errors import TimeoutError as DucktapeTimeoutError
from rptest.services.redpanda_installer import RedpandaInstaller, wait_for_num_versions

from ducktape.errors import TimeoutError as DucktapeTimeoutError
from ducktape.utils.util import wait_until

CURRENT_LOGICAL_VERSION = 5

# The upgrade tests defined below rely on having a logical version lower than
# CURRENT_LOGICAL_VERSION. For the sake of these tests, the exact version
# shouldn't matter.
OLD_VERSION = (22, 1, 4)


class FeaturesTestBase(RedpandaTest):
"""
Expand All @@ -42,7 +50,9 @@ def _assert_default_features(self):
# This assertion will break each time we increment the value
# of `latest_version` in the redpanda source. Update it when
# that happens.
assert features_response['cluster_version'] == 5
initial_version = features_response["cluster_version"]
assert initial_version == CURRENT_LOGICAL_VERSION, \
f"Version mismatch: {initial_version} vs {CURRENT_LOGICAL_VERSION}"

assert self._get_features_map(
features_response)['central_config']['state'] == 'active'
Expand All @@ -63,77 +73,6 @@ def __init__(self, *args, **kwargs):
def test_get_features(self):
self._assert_default_features()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_upgrade(self):
"""
Verify that on updating to a new logical version, the cluster
version does not increment until all nodes are up to date.
"""
initial_version = self.admin.get_features()['cluster_version']

new_version = initial_version + 1
self.logger.info(
f"Simulating upgrade from version {initial_version} to version {new_version}"
)

# Modified environment variables apply to processes restarted from this point onwards
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f'{new_version}'})

# Restart nodes one by one. Version shouldn't increment until all three are done.
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
time.sleep(5) # Give it a chance to update
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[1]])
time.sleep(5) # Give it a chance to update
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[2]])

# Node logical versions are transmitted as part of health messages, so we may
# have to wait for the next health tick (health_monitor_tick_interval=10s) before
# the controller leader fetches health from the last restarted peer.
wait_until(lambda: new_version == self.admin.get_features()[
'cluster_version'],
timeout_sec=15,
backoff_sec=1)

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_rollback(self):
"""
Verify that on a rollback before updating all nodes, the cluster
version does not increment.
"""
initial_version = self.admin.get_features()['cluster_version']

new_version = initial_version + 1
self.logger.info(
f"Simulating upgrade from version {initial_version} to version {new_version}"
)

# Modified environment variables apply to processes restarted from this point onwards
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f'{new_version}'})

# Restart nodes one by one. Version shouldn't increment until all three are done.
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
time.sleep(5) # Give it a chance to update
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[1]])
time.sleep(5) # Give it a chance to update
assert initial_version == self.admin.get_features()['cluster_version']

self.logger.info(f"Simulating rollback to version {initial_version}")
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f'{initial_version}'})

self.redpanda.restart_nodes([self.redpanda.nodes[0]])
self.redpanda.restart_nodes([self.redpanda.nodes[1]])
time.sleep(5) # Give it a chance to update
assert initial_version == self.admin.get_features()['cluster_version']

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_explicit_activation(self):
"""
Expand Down Expand Up @@ -250,6 +189,80 @@ def is_equal_to_license_properties(license_contents,
resp['license']) is True


class FeaturesMultiNodeUpgradeTest(FeaturesTestBase):
"""
Multi-node variant of tests that exercise upgrades from older versions.
"""
def __init__(self, *args, **kwargs):
super().__init__(*args, num_brokers=3, **kwargs)
self.admin = Admin(self.redpanda)
self.installer = self.redpanda._installer

def setUp(self):
self.installer.install(self.redpanda.nodes, OLD_VERSION)
super().setUp()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_upgrade(self):
"""
Verify that on updating to a new logical version, the cluster
version does not increment until all nodes are up to date.
"""
initial_version = self.admin.get_features()['cluster_version']
assert initial_version < CURRENT_LOGICAL_VERSION, \
f"downgraded logical version {initial_version}"

self.installer.install(self.redpanda.nodes, RedpandaInstaller.HEAD)

# Restart nodes one by one. Version shouldn't increment until all three are done.
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
_ = wait_for_num_versions(self.redpanda, 2)
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[1]])
# Even after waiting a bit, the logical version shouldn't change.
time.sleep(5)
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[2]])

# Node logical versions are transmitted as part of health messages, so we may
# have to wait for the next health tick (health_monitor_tick_interval=10s) before
# the controller leader fetches health from the last restarted peer.
wait_until(lambda: CURRENT_LOGICAL_VERSION == self.admin.get_features(
)['cluster_version'],
timeout_sec=15,
backoff_sec=1)

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_rollback(self):
"""
Verify that on a rollback before updating all nodes, the cluster
version does not increment.
"""
initial_version = self.admin.get_features()['cluster_version']
assert initial_version < CURRENT_LOGICAL_VERSION, \
f"downgraded logical version {initial_version}"

self.installer.install(self.redpanda.nodes, RedpandaInstaller.HEAD)
# Restart nodes one by one. Version shouldn't increment until all three are done.
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
_ = wait_for_num_versions(self.redpanda, 2)
# Even after waiting a bit, the logical version shouldn't change.
time.sleep(5)
assert initial_version == self.admin.get_features()['cluster_version']

self.redpanda.restart_nodes([self.redpanda.nodes[1]])
time.sleep(5)
assert initial_version == self.admin.get_features()['cluster_version']

self.installer.install(self.redpanda.nodes, OLD_VERSION)
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
self.redpanda.restart_nodes([self.redpanda.nodes[1]])
_ = wait_for_num_versions(self.redpanda, 1)
assert initial_version == self.admin.get_features()['cluster_version']


class FeaturesSingleNodeTest(FeaturesTestBase):
"""
A single node variant to make sure feature_manager does its job in the absence
Expand All @@ -265,27 +278,36 @@ def __init__(self, *args, **kwargs):
def test_get_features(self):
self._assert_default_features()


class FeaturesSingleNodeUpgradeTest(FeaturesTestBase):
"""
Single-node variant of tests that exercise upgrades from older versions.
"""
def __init__(self, *args, **kwargs):
super().__init__(*args, num_brokers=1, **kwargs)
self.admin = Admin(self.redpanda)
self.installer = self.redpanda._installer

def setUp(self):
self.installer.install(self.redpanda.nodes, OLD_VERSION)
super().setUp()

@cluster(num_nodes=1, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_upgrade(self):
"""
Verify that on updating to a new logical version, the cluster
version does not increment until all nodes are up to date.
"""
initial_version = self.admin.get_features()['cluster_version']

new_version = initial_version + 1
self.logger.info(
f"Simulating upgrade from version {initial_version} to version {new_version}"
)

# Modified environment variables apply to processes restarted from this point onwards
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f'{new_version}'})
assert initial_version < CURRENT_LOGICAL_VERSION, \
f"downgraded logical version {initial_version}"

# Restart nodes one by one. Version shouldn't increment until all three are done.
self.installer.install([self.redpanda.nodes[0]],
RedpandaInstaller.HEAD)
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
wait_until(lambda: new_version == self.admin.get_features()[
'cluster_version'],
wait_until(lambda: CURRENT_LOGICAL_VERSION == self.admin.get_features(
)['cluster_version'],
timeout_sec=5,
backoff_sec=1)

Expand All @@ -295,6 +317,7 @@ def __init__(self, *args, **kwargs):
super().__init__(*args, num_brokers=4, **kwargs)

self.admin = Admin(self.redpanda)
self.installer = self.redpanda._installer

def setUp(self):
# We will start nodes by hand during test.
Expand All @@ -307,27 +330,20 @@ def test_old_node_join(self):
it is rejected.
"""

initial_version = 768

self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f"{initial_version}"})

# Pick a node to roleplay an old version of redpanda
old_node = self.redpanda.nodes[-1]
self.installer.install([old_node], OLD_VERSION)

# Start first three nodes
self.redpanda.start(self.redpanda.nodes[0:-1])

# Explicit clean because it's not included in the default
# one during start()
self.redpanda.clean_node(old_node)
self.redpanda.clean_node(old_node, preserve_current_install=True)

assert initial_version == self.admin.get_features()['cluster_version']

# Bring up the fourth node reporting an old logical version
old_version = initial_version - 1
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f"{old_version}"})
initial_version = self.admin.get_features()['cluster_version']
assert initial_version == CURRENT_LOGICAL_VERSION, \
f"Version mismatch: {initial_version} vs {CURRENT_LOGICAL_VERSION}"

try:
self.redpanda.start_node(old_node)
Expand All @@ -339,8 +355,7 @@ def test_old_node_join(self):
)

# Restart it with a sufficiently recent version and join should succeed
self.redpanda.set_environment(
{'__REDPANDA_LOGICAL_VERSION': f"{initial_version}"})
self.installer.install([old_node], RedpandaInstaller.HEAD)
self.redpanda.restart_nodes([old_node])
wait_until(lambda: self.redpanda.registered(old_node),
timeout_sec=10,
Expand Down

0 comments on commit 8ee1811

Please sign in to comment.