Skip to content

Commit

Permalink
tests: update some storage fixture tests to reset configs
Browse files Browse the repository at this point in the history
Uses the new scoped_config to reset updated configs to their
defaults at the end of each test. This ensures no side effects across
tests.
  • Loading branch information
andrwng committed Aug 31, 2023
1 parent f562831 commit d73e904
Show file tree
Hide file tree
Showing 5 changed files with 64 additions and 56 deletions.
53 changes: 27 additions & 26 deletions src/v/cloud_storage/tests/cloud_storage_e2e_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
#include "kafka/server/tests/produce_consume_utils.h"
#include "model/fundamental.h"
#include "redpanda/tests/fixture.h"
#include "test_utils/scoped_config.h"

#include <seastar/core/io_priority_class.hh>

Expand All @@ -45,11 +46,13 @@ class e2e_fixture
set_expectations_and_listen({});
wait_for_controller_leadership().get();
}

scoped_config test_local_cfg;
};

FIXTURE_TEST(test_produce_consume_from_cloud, e2e_fixture) {
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);
const model::topic topic_name("tapioca");
model::ntp ntp(model::kafka_namespace, topic_name, 0);
cluster::topic_properties props;
Expand Down Expand Up @@ -105,16 +108,15 @@ FIXTURE_TEST(test_produce_consume_from_cloud, e2e_fixture) {

FIXTURE_TEST(test_produce_consume_from_cloud_with_spillover, e2e_fixture) {
#ifndef _NDEBUG
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
config::shard_local_cfg().cloud_storage_spillover_manifest_size.set_value(
std::make_optional((size_t)0x1000));
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);
test_local_cfg.get("cloud_storage_spillover_manifest_size")
.set_value(std::make_optional((size_t)0x1000));

config::shard_local_cfg().cloud_storage_enable_segment_merging.set_value(
false);
test_local_cfg.get("cloud_storage_enable_segment_merging").set_value(false);

config::shard_local_cfg().enable_metrics_reporter.set_value(false);
config::shard_local_cfg().retention_local_strict.set_value(true);
test_local_cfg.get("enable_metrics_reporter").set_value(false);
test_local_cfg.get("retention_local_strict").set_value(true);

const model::topic topic_name("tapioca");
model::ntp ntp(model::kafka_namespace, topic_name, 0);
Expand Down Expand Up @@ -349,22 +351,20 @@ class cloud_storage_manual_e2e_test
wait_for_controller_leadership().get();

// Apply local retention frequently.
config::shard_local_cfg().log_compaction_interval_ms.set_value(
std::chrono::duration_cast<std::chrono::milliseconds>(1s));
test_local_cfg.get("log_compaction_interval_ms")
.set_value(std::chrono::duration_cast<std::chrono::milliseconds>(1s));
// We'll control uploads ourselves.
config::shard_local_cfg()
.cloud_storage_enable_segment_merging.set_value(false);
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
test_local_cfg.get("cloud_storage_enable_segment_merging")
.set_value(false);
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);
// Disable metrics to speed things up.
config::shard_local_cfg().enable_metrics_reporter.set_value(false);
test_local_cfg.get("enable_metrics_reporter").set_value(false);
// Encourage spilling over.
config::shard_local_cfg()
.cloud_storage_spillover_manifest_max_segments.set_value(
std::make_optional<size_t>(segs_per_spill));
config::shard_local_cfg()
.cloud_storage_spillover_manifest_size.set_value(
std::optional<size_t>{});
test_local_cfg.get("cloud_storage_spillover_manifest_max_segments")
.set_value(std::make_optional<size_t>(segs_per_spill));
test_local_cfg.get("cloud_storage_spillover_manifest_size")
.set_value(std::optional<size_t>{});

topic_name = model::topic("tapioca");
ntp = model::ntp(model::kafka_namespace, topic_name, 0);
Expand All @@ -382,6 +382,7 @@ class cloud_storage_manual_e2e_test
archiver = &partition->archiver()->get();
}

scoped_config test_local_cfg;
model::topic topic_name;
model::ntp ntp;
cluster::partition* partition;
Expand Down Expand Up @@ -422,7 +423,7 @@ ss::future<bool> check_consume_from_beginning(
} // namespace

FIXTURE_TEST(test_consume_during_spillover, cloud_storage_manual_e2e_test) {
config::shard_local_cfg().fetch_max_bytes.set_value(size_t{10});
test_local_cfg.get("fetch_max_bytes").set_value(size_t{10});
const auto records_per_seg = 5;
const auto num_segs = 40;
tests::remote_segment_generator gen(make_kafka_client().get(), *partition);
Expand Down Expand Up @@ -470,8 +471,8 @@ FIXTURE_TEST(test_consume_during_spillover, cloud_storage_manual_e2e_test) {
FIXTURE_TEST(
reclaimable_reported_in_health_report,
cloud_storage_manual_multinode_test_base) {
config::shard_local_cfg().retention_local_trim_interval.set_value(
std::chrono::milliseconds(2000));
test_local_cfg.get("retention_local_trim_interval")
.set_value(std::chrono::milliseconds(2000));

// start a second fixutre and wait for stable setup
auto fx2 = start_second_fixture();
Expand Down
28 changes: 14 additions & 14 deletions src/v/cloud_storage/tests/delete_records_e2e_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include "redpanda/tests/fixture.h"
#include "storage/disk_log_impl.h"
#include "test_utils/async.h"
#include "test_utils/scoped_config.h"

#include <seastar/core/io_priority_class.hh>

Expand Down Expand Up @@ -84,23 +85,21 @@ class delete_records_e2e_fixture
wait_for_controller_leadership().get();

// Apply local retention frequently.
config::shard_local_cfg().log_compaction_interval_ms.set_value(
std::chrono::duration_cast<std::chrono::milliseconds>(1s));
test_local_cfg.get("log_compaction_interval_ms")
.set_value(std::chrono::duration_cast<std::chrono::milliseconds>(1s));
// We'll control uploads ourselves.
config::shard_local_cfg()
.cloud_storage_enable_segment_merging.set_value(false);
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
test_local_cfg.get("cloud_storage_enable_segment_merging")
.set_value(false);
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);
// Disable metrics to speed things up.
config::shard_local_cfg().enable_metrics_reporter.set_value(false);
test_local_cfg.get("enable_metrics_reporter").set_value(false);
// Encourage spilling over.
config::shard_local_cfg()
.cloud_storage_spillover_manifest_max_segments.set_value(
std::make_optional<size_t>(segs_per_spill));
config::shard_local_cfg()
.cloud_storage_spillover_manifest_size.set_value(
std::optional<size_t>{});
config::shard_local_cfg().retention_local_strict.set_value(true);
test_local_cfg.get("cloud_storage_spillover_manifest_max_segments")
.set_value(std::make_optional<size_t>(segs_per_spill));
test_local_cfg.get("cloud_storage_spillover_manifest_size")
.set_value(std::optional<size_t>{});
test_local_cfg.get("retention_local_strict").set_value(true);

topic_name = model::topic("tapioca");
ntp = model::ntp(model::kafka_namespace, topic_name, 0);
Expand Down Expand Up @@ -131,6 +130,7 @@ class delete_records_e2e_fixture
new_archiver.manifest().get_start_kafka_offset_override(),
model::offset{});
}
scoped_config test_local_cfg;

model::topic topic_name;
model::ntp ntp;
Expand Down
10 changes: 6 additions & 4 deletions src/v/cloud_storage/tests/manual_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
#include "model/fundamental.h"
#include "redpanda/tests/fixture.h"
#include "storage/disk_log_impl.h"
#include "test_utils/scoped_config.h"

class cloud_storage_manual_multinode_test_base
: public s3_imposter_fixture
Expand All @@ -29,10 +30,10 @@ class cloud_storage_manual_multinode_test_base
// No expectations: tests will PUT and GET organically.
set_expectations_and_listen({});

config::shard_local_cfg()
.cloud_storage_enable_segment_merging.set_value(false);
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
test_local_cfg.get("cloud_storage_enable_segment_merging")
.set_value(false);
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);

wait_for_controller_leadership().get();
}
Expand All @@ -53,4 +54,5 @@ class cloud_storage_manual_multinode_test_base
get_archival_config(),
get_cloud_config(httpd_port_number()));
}
scoped_config test_local_cfg;
};
20 changes: 11 additions & 9 deletions src/v/cloud_storage/tests/read_replica_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#include "model/fundamental.h"
#include "redpanda/tests/fixture.h"
#include "storage/disk_log_impl.h"
#include "test_utils/scoped_config.h"

using tests::kafka_consume_transport;

Expand All @@ -38,17 +39,17 @@ class read_replica_e2e_fixture
wait_for_controller_leadership().get();

// Disable metrics to speed things up.
config::shard_local_cfg().enable_metrics_reporter.set_value(false);
config::shard_local_cfg().disable_metrics.set_value(true);
config::shard_local_cfg().disable_public_metrics.set_value(true);
test_local_cfg.get("enable_metrics_reporter").set_value(false);
test_local_cfg.get("disable_metrics").set_value(true);
test_local_cfg.get("disable_public_metrics").set_value(true);

// Avoid background work since we'll control uploads ourselves.
config::shard_local_cfg()
.cloud_storage_enable_segment_merging.set_value(false);
config::shard_local_cfg()
.cloud_storage_disable_upload_loop_for_tests.set_value(true);
config::shard_local_cfg()
.cloud_storage_disable_read_replica_loop_for_tests.set_value(true);
test_local_cfg.get("cloud_storage_enable_segment_merging")
.set_value(false);
test_local_cfg.get("cloud_storage_disable_upload_loop_for_tests")
.set_value(true);
test_local_cfg.get("cloud_storage_disable_read_replica_loop_for_tests")
.set_value(true);
}

std::unique_ptr<redpanda_thread_fixture> start_read_replica_fixture() {
Expand All @@ -66,6 +67,7 @@ class read_replica_e2e_fixture
get_archival_config(),
get_cloud_config(httpd_port_number()));
}
scoped_config test_local_cfg;
};

FIXTURE_TEST(test_read_replica_basic_sync, read_replica_e2e_fixture) {
Expand Down
9 changes: 6 additions & 3 deletions src/v/storage/tests/storage_e2e_fixture_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
#include "kafka/server/tests/produce_consume_utils.h"
#include "redpanda/tests/fixture.h"
#include "test_utils/fixture.h"
#include "test_utils/scoped_config.h"

#include <seastar/core/lowres_clock.hh>

Expand All @@ -20,7 +21,9 @@
using namespace std::chrono_literals;
using std::vector;

struct storage_e2e_fixture : public redpanda_thread_fixture {};
struct storage_e2e_fixture : public redpanda_thread_fixture {
scoped_config test_local_cfg;
};

namespace {

Expand All @@ -42,8 +45,8 @@ ss::future<> produce_to_fixture(
} // namespace

FIXTURE_TEST(test_compaction_segment_ms, storage_e2e_fixture) {
config::shard_local_cfg().log_segment_ms_min.set_value(
std::chrono::duration_cast<std::chrono::milliseconds>(1ms));
test_local_cfg.get("log_segment_ms_min")
.set_value(std::chrono::duration_cast<std::chrono::milliseconds>(1ms));
const auto topic_name = model::topic("tapioca");
const auto ntp = model::ntp(model::kafka_namespace, topic_name, 0);

Expand Down

0 comments on commit d73e904

Please sign in to comment.