Skip to content

Commit

Permalink
rptest: skip cloud storage upgrades test in Azure
Browse files Browse the repository at this point in the history
Azure CDT runs will use HNS by default. Previous versions of Redpanda
don't know how to interact with HNS containers, so these upgrade tests
fail.

I've also removed the `skip_azure_blob_storage` decorator as it's not
working as intended anymore. Haven't tracked down why that is, but an
appropriate invocation of `get_cloud_storage_type` is equivalent.

(cherry picked from commit 5232066)
  • Loading branch information
Vlad Lazar authored and vbotbuildovich committed Aug 30, 2023
1 parent c294f36 commit bb760d3
Show file tree
Hide file tree
Showing 6 changed files with 36 additions and 51 deletions.
11 changes: 8 additions & 3 deletions tests/rptest/tests/license_upgrade_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,12 @@
import time

from ducktape.utils.util import wait_until
from ducktape.mark import matrix
from rptest.utils.rpenv import sample_license
from rptest.services.admin import Admin
from ducktape.utils.util import wait_until
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda import SISettings
from rptest.services.redpanda import SISettings, CloudStorageType, get_cloud_storage_type
from rptest.services.cluster import cluster
from requests.exceptions import HTTPError
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
Expand Down Expand Up @@ -49,7 +50,9 @@ def setUp(self):
super(UpgradeToLicenseChecks, self).setUp()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_basic_upgrade(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_basic_upgrade(self, cloud_storage_type):
# Modified environment variables apply to processes restarted from this point onwards
self.redpanda.set_environment({
'__REDPANDA_LICENSE_CHECK_INTERVAL_SEC':
Expand Down Expand Up @@ -129,7 +132,9 @@ def setUp(self):
super(UpgradeMigratingLicenseVersion, self).setUp()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_license_upgrade(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_license_upgrade(self, cloud_storage_type):
license = sample_license()
if license is None:
self.logger.info(
Expand Down
4 changes: 3 additions & 1 deletion tests/rptest/tests/read_replica_e2e_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -464,7 +464,9 @@ def __init__(self, test_context: TestContext):
self.second_cluster = None

@cluster(num_nodes=8)
def test_upgrades(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_upgrades(self, cloud_storage_type):
partition_count = 1
install_opts = InstallOptions(install_previous_version=True)
self.start_redpanda(3,
Expand Down
21 changes: 13 additions & 8 deletions tests/rptest/tests/topic_creation_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,11 @@
from rptest.clients.rpk import RpkException, RpkTool
from rptest.clients.kafka_cat import KafkaCat
from rptest.services.producer_swarm import ProducerSwarm
from rptest.services.redpanda import ResourceSettings, SISettings
from rptest.services.redpanda import ResourceSettings, SISettings, CloudStorageType, get_cloud_storage_type
from rptest.services.redpanda_installer import RedpandaInstaller
from rptest.services.rpk_producer import RpkProducer
from rptest.clients.kafka_cli_tools import KafkaCliTools
from rptest.util import wait_for_local_storage_truncate, expect_exception
from rptest.utils.mode_checks import skip_azure_blob_storage
from rptest.clients.kcl import KCL

from ducktape.utils.util import wait_until
Expand Down Expand Up @@ -560,8 +559,10 @@ def _populate_tiered_storage_topic(self, topic_name, local_retention):
target_bytes=local_retention)

@cluster(num_nodes=3)
@skip_azure_blob_storage
def test_cloud_storage_sticky_enablement_v22_2_to_v22_3(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_cloud_storage_sticky_enablement_v22_2_to_v22_3(
self, cloud_storage_type):
"""
In Redpanda 22.3, the cluster defaults for cloud storage change
from being applied at runtime to being sticky at creation time,
Expand Down Expand Up @@ -628,8 +629,10 @@ def test_cloud_storage_sticky_enablement_v22_2_to_v22_3(self):
assert described['redpanda.remote.read'] == ('false', 'DEFAULT_CONFIG')

@cluster(num_nodes=3)
@skip_azure_blob_storage
def test_retention_config_on_upgrade_from_v22_2_to_v22_3(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_retention_config_on_upgrade_from_v22_2_to_v22_3(
self, cloud_storage_type):
self.install_and_start()

self.rpk.create_topic("test-topic-with-retention",
Expand Down Expand Up @@ -814,8 +817,10 @@ def is_empty():
assert len(deleted_objects) == 0

@cluster(num_nodes=3)
@skip_azure_blob_storage
def test_retention_upgrade_with_cluster_remote_write(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_retention_upgrade_with_cluster_remote_write(
self, cloud_storage_type):
"""
Validate how the cluster-wide cloud_storage_enable_remote_write
is handled on upgrades from <=22.2
Expand Down
10 changes: 5 additions & 5 deletions tests/rptest/tests/upgrade_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
from collections import defaultdict
from packaging.version import Version

from ducktape.mark import parametrize
from ducktape.mark import parametrize, matrix
from ducktape.utils.util import wait_until
from rptest.services.admin import Admin
from rptest.clients.rpk import RpkTool
Expand All @@ -28,9 +28,8 @@
wait_until_segments,
)
from rptest.utils.si_utils import BucketView
from rptest.utils.mode_checks import skip_azure_blob_storage
from rptest.services.cluster import cluster
from rptest.services.redpanda import SISettings
from rptest.services.redpanda import SISettings, CloudStorageType, get_cloud_storage_type
from rptest.services.kgo_verifier_services import (
KgoVerifierProducer,
KgoVerifierSeqConsumer,
Expand Down Expand Up @@ -380,8 +379,9 @@ def install_and_start(self):
super().setUp()

@cluster(num_nodes=4, log_allow_list=RESTART_LOG_ALLOW_LIST)
@skip_azure_blob_storage
def test_rolling_upgrade(self):
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_rolling_upgrade(self, cloud_storage_type):
"""
Verify that when tiered storage writes happen during a rolling upgrade,
we continue to write remote content that old versions can read, until
Expand Down
9 changes: 7 additions & 2 deletions tests/rptest/tests/workload_upgrade_runner_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
from rptest.clients.offline_log_viewer import OfflineLogViewer
from rptest.services.cluster import cluster
from rptest.services.admin import Admin
from rptest.services.redpanda import SISettings
from rptest.services.redpanda import SISettings, CloudStorageType, get_cloud_storage_type
from rptest.services.redpanda_installer import RedpandaInstaller, RedpandaVersion, RedpandaVersionTriple
from rptest.services.workload_protocol import PWorkload
from rptest.tests.prealloc_nodes import PreallocNodesTest
Expand All @@ -22,6 +22,7 @@
from rptest.tests.redpanda_test import RedpandaTest
from rptest.tests.workload_license import LicenseWorkload
from rptest.utils.mode_checks import skip_debug_mode
from ducktape.mark import matrix


def expand_version(
Expand Down Expand Up @@ -248,7 +249,11 @@ def cluster_version(self) -> int:

@skip_debug_mode
@cluster(num_nodes=4)
def test_workloads_through_releases(self):
# TODO(vlad): Allow this test on ABS once we have at least two versions
# of Redpanda that support Azure Hierarchical Namespaces.
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]))
def test_workloads_through_releases(self, cloud_storage_type):
# this callback will be called between each upgrade, in a mixed version state
def mid_upgrade_check(raw_versions: dict[Any, RedpandaVersion]):
rp_versions = {
Expand Down
32 changes: 0 additions & 32 deletions tests/rptest/utils/mode_checks.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,35 +63,3 @@ def f(*args, **kwargs):
return func(*args, **kwargs)

return f


def skip_azure_blob_storage(func):
"""
Decorator applied to a test class method. The property `azure_blob_storage` should be present
on the object.
If set to true, the wrapped function call is skipped, and a cleanup action
is performed instead.
If set to false, the wrapped function (usually a test case) is called.
"""
@functools.wraps(func)
def f(*args, **kwargs):
assert args, 'skip_azure_blob_storage must be placed on a test method in a class'

caller = args[0]

assert hasattr(
caller, 'azure_blob_storage'
), 'skip_azure_blob_storage called on object which does not have azure_blob_storage attribute'
assert hasattr(
caller, 'logger'
), 'skip_azure_blob_storage called on object which has no logger'
if caller.azure_blob_storage:
caller.logger.info(
"Skipping Azure Blob Storage test in (requires S3)")
cleanup_on_early_exit(caller)
return None
return func(*args, **kwargs)

return f

0 comments on commit bb760d3

Please sign in to comment.