Skip to content

Commit

Permalink
tests: use RedpandaInstaller in upgrade test
Browse files Browse the repository at this point in the history
We previously relied on logical versions to partially simulate different
Redpanda versions. This commit uses the newly added RedpandaInstaller to
allow performing a more organic upgrade.

This also makes the test to wait for a bit in between restarting to
ensure we actually run through ops during the upgrade.
  • Loading branch information
andrwng committed Jul 25, 2022
1 parent c37bb1a commit 18b610c
Showing 1 changed file with 34 additions and 21 deletions.
55 changes: 34 additions & 21 deletions tests/rptest/tests/controller_upgrade_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,35 +7,37 @@
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0

import random
import threading
import time
import re

from rptest.services.admin import Admin
from rptest.services.cluster import cluster
from ducktape.utils.util import wait_until
from rptest.clients.types import TopicSpec
from rptest.tests.end_to_end import EndToEndTest
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST, RedpandaService
from rptest.clients.default import DefaultClient
from ducktape.utils.util import wait_until
from rptest.services.admin import Admin
from rptest.services.admin_ops_fuzzer import AdminOperationsFuzzer
from rptest.services.cluster import cluster
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST, RedpandaService
from rptest.services.redpanda_installer import RedpandaInstaller
from rptest.tests.end_to_end import EndToEndTest

from ducktape.mark import matrix

# TODO: fix https://github.com/redpanda-data/redpanda/issues/5629
ALLOWED_LOGS = [
# e.g. cluster - controller_backend.cc:466 - exception while executing partition operation: {type: update_finished, ntp: {kafka/test-topic-1944-1639161306808363/1}, offset: 413, new_assignment: { id: 1, group_id: 65, replicas: {{node_id: 3, shard: 2}, {node_id: 4, shard: 2}, {node_id: 1, shard: 0}} }, previous_assignment: {nullopt}} - std::__1::__fs::filesystem::filesystem_error (error system:39, filesystem error: remove failed: Directory not empty [/var/lib/redpanda/data/kafka/test-topic-1944-1639161306808363])
re.compile("cluster - .*Directory not empty"),
]

class ControllerUpgradeTest(EndToEndTest):
@cluster(num_nodes=7, log_allow_list=RESTART_LOG_ALLOW_LIST)
@cluster(num_nodes=7, log_allow_list=RESTART_LOG_ALLOW_LIST + ALLOWED_LOGS)
def test_updating_cluster_when_executing_operations(self):
'''
Validates that cluster is operational when upgrading controller log
'''

# set redpanda logical version to value without __consumer_offsets support
self.redpanda = RedpandaService(
self.test_context,
5,
# set redpanda logical version to v22.1 - last version without serde encoding
environment={"__REDPANDA_LOGICAL_VERSION": 3})
# set redpanda version to v22.1 - last version without serde encoding
self.redpanda = RedpandaService(self.test_context, 5)

installer = self.redpanda._installer
installer.install(self.redpanda.nodes, (22, 1, 4))

self.redpanda.start()
admin_fuzz = AdminOperationsFuzzer(self.redpanda)
Expand Down Expand Up @@ -63,17 +65,28 @@ def cluster_is_stable():

return True

# reset environment to use latest redpanda logical version
self.redpanda.set_environment({})
for n in self.redpanda.nodes:
# Get to a stable starting point before asserting anything about our
# workload.
wait_until(cluster_is_stable, 90, backoff_sec=2)

self.redpanda.restart_nodes(n, stop_timeout=60)
admin_fuzz.wait(5, 240)

# reset to use latest Redpanda version
installer.install(self.redpanda.nodes, RedpandaInstaller.HEAD)

for n in self.redpanda.nodes:
# NOTE: we're not doing a rolling restart to avoid interfering with
# the admin operations.
self.redpanda.restart_nodes([n], stop_timeout=60, start_timeout=90)
num_executed_before_restart = admin_fuzz.executed

# wait for leader balancer to start evening out leadership
wait_until(cluster_is_stable, 90, backoff_sec=2)
admin_fuzz.wait(num_executed_before_restart + 2, 240)

self.run_validation(min_records=100000,
producer_timeout_sec=300,
consumer_timeout_sec=180)
admin_fuzz.wait(20, 240)
num_executed_after_upgrade = admin_fuzz.executed
admin_fuzz.wait(num_executed_after_upgrade + 5, 240)
admin_fuzz.stop()

0 comments on commit 18b610c

Please sign in to comment.