Skip to content

Commit

Permalink
rptest: add mixed node cluster tests to self_test_test
Browse files Browse the repository at this point in the history
  • Loading branch information
WillemKauf committed Aug 10, 2024
1 parent cefe877 commit a33abd7
Showing 1 changed file with 81 additions and 0 deletions.
81 changes: 81 additions & 0 deletions tests/rptest/tests/self_test_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,12 +9,15 @@

import re
import time
from collections import defaultdict
from rptest.services.cluster import cluster
from rptest.tests.end_to_end import EndToEndTest
from rptest.tests.redpanda_test import RedpandaTest
from rptest.clients.rpk import RpkTool
from rptest.services.admin import Admin
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST, SISettings
from rptest.services.redpanda_installer import RedpandaVersionLine
from rptest.services.redpanda_installer import InstallOptions
from ducktape.utils.util import wait_until
from ducktape.mark import matrix
from rptest.utils.functional import flat_map
Expand Down Expand Up @@ -270,3 +273,81 @@ def test_self_test_unknown_test_type(self):
else:
assert 'error' not in report
assert 'warning' not in report

@cluster(num_nodes=3)
def test_self_test_mixed_node_controller_lower_version(self):
"""Assert the self test still runs when the controller node
is of a lower version than the rest of the nodes in the cluster.
The upgraded follower nodes should be able to parse the "unknown"
checks (currently just the cloudcheck), and then run and return
their results to the controller node."""
num_nodes = 3

install_opts = InstallOptions(version=RedpandaVersionLine((24, 1)),
num_to_upgrade=2)
self.start_redpanda(
num_nodes=num_nodes,
si_settings=SISettings(test_context=self.test_context),
install_opts=install_opts)

# Attempt to run with a possibly unknown "cloud" test.
# The controller, which is of a lower version than the other nodes in the cluster,
# doesn't recognize "cloud" as a test, but the other nodes should.
request_json = {
'tests': [{
'type': 'cloud',
'backoff_ms': 100,
'timeout_ms': 5000
}]
}

redpanda_versions = {
i: self.redpanda.get_version_int_tuple(node)
for (i, node) in enumerate(self.redpanda.nodes)
}

controller_node_index = min(redpanda_versions,
key=redpanda_versions.get)
controller_node_id = controller_node_index + 1
# Make sure that the lowest version node is the controller.
self.redpanda._admin.partition_transfer_leadership(
'redpanda', 'controller', 0, controller_node_id)
wait_until(lambda: self.redpanda._admin.get_partition_leader(
namespace="redpanda", topic="controller", partition=0) ==
controller_node_id,
timeout_sec=10,
backoff_sec=1,
err_msg="Leadership did not stabilize")

# Manually invoke self test admin endpoint, using the lowest version node as the target.
self.redpanda._admin._request(
'POST',
'debug/self_test/start',
json=request_json,
node=self.redpanda.nodes[controller_node_index])

# Wait for self test completion.
node_reports = self.wait_for_self_test_completion()

unknown_checks_map = defaultdict(set)
for node, version in redpanda_versions.items():
node_id = node + 1
# Cloudcheck was introduced in 24.2.1.
# Expect that it will be unknown to nodes running
# earlier versions of redpanda.
if version < (24, 2, 1):
unknown_checks_map[node_id].add('cloud')

# Assert reports are passing, with the exception of unknown tests.
assert len(node_reports) > 0
for report in node_reports:
node = report['node_id']
results = report['results']
# Results shouldn't be empty, even for unknown checks.
assert len(results) > 0
for result in results:
if result['test_type'] in unknown_checks_map[node]:
assert 'error' in result
else:
assert 'error' not in result
assert 'warning' not in result

0 comments on commit a33abd7

Please sign in to comment.