Skip to content

Commit

Permalink
Merge pull request #15188 from andrwng/cluster-metadata-uploads-default
Browse files Browse the repository at this point in the history
config: enable cluster metadata uploads by default
  • Loading branch information
piyushredpanda authored Nov 30, 2023
2 parents a6f2825 + efad83b commit 83611a9
Show file tree
Hide file tree
Showing 13 changed files with 70 additions and 14 deletions.
2 changes: 2 additions & 0 deletions src/v/cloud_storage/tests/cloud_storage_e2e_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "cloud_storage/tests/manual_fixture.h"
#include "cloud_storage/tests/produce_utils.h"
#include "cloud_storage/tests/s3_imposter.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "cluster/health_monitor_frontend.h"
#include "config/configuration.h"
#include "kafka/server/tests/produce_consume_utils.h"
Expand All @@ -35,6 +36,7 @@ static ss::logger e2e_test_log("e2e_test");

class e2e_fixture
: public s3_imposter_fixture
, public manual_metadata_upload_mixin
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
2 changes: 2 additions & 0 deletions src/v/cloud_storage/tests/manual_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@

#include "cloud_storage/tests/produce_utils.h"
#include "cloud_storage/tests/s3_imposter.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "cluster/types.h"
#include "config/configuration.h"
#include "kafka/server/tests/produce_consume_utils.h"
Expand All @@ -20,6 +21,7 @@

class cloud_storage_manual_multinode_test_base
: public s3_imposter_fixture
, public manual_metadata_upload_mixin
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
2 changes: 2 additions & 0 deletions src/v/cloud_storage/tests/topic_recovery_service_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@

#include "cloud_storage/recovery_request.h"
#include "cloud_storage/tests/s3_imposter.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "cluster/topic_recovery_service.h"
#include "redpanda/tests/fixture.h"
#include "test_utils/fixture.h"
Expand Down Expand Up @@ -126,6 +127,7 @@ bool is_manifest_list_request(const http_test_utils::request_info& req) {

class fixture
: public s3_imposter_fixture
, public manual_metadata_upload_mixin
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#include "cloud_storage/tests/s3_imposter.h"
#include "cloud_storage/types.h"
#include "cluster/cloud_metadata/tests/cluster_metadata_utils.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "cluster/cloud_metadata/uploader.h"
#include "cluster/cluster_recovery_reconciler.h"
#include "cluster/config_frontend.h"
Expand Down Expand Up @@ -48,6 +49,7 @@ static ss::abort_source never_abort;
class ClusterRecoveryBackendTest
: public seastar_test
, public s3_imposter_fixture
, public manual_metadata_upload_mixin
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
2 changes: 2 additions & 0 deletions src/v/cluster/cloud_metadata/tests/manifest_downloads_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "cluster/cloud_metadata/cluster_manifest.h"
#include "cluster/cloud_metadata/key_utils.h"
#include "cluster/cloud_metadata/manifest_downloads.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "model/fundamental.h"
#include "redpanda/application.h"
#include "redpanda/tests/fixture.h"
Expand All @@ -29,6 +30,7 @@ using namespace cluster::cloud_metadata;

class cluster_metadata_fixture
: public s3_imposter_fixture
, public manual_metadata_upload_mixin
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
22 changes: 22 additions & 0 deletions src/v/cluster/cloud_metadata/tests/manual_mixin.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* Copyright 2023 Redpanda Data, Inc.
*
* Licensed as a Redpanda Enterprise file under the Redpanda Community
* License (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md
*/
#pragma once
#include "test_utils/scoped_config.h"

class manual_metadata_upload_mixin {
public:
manual_metadata_upload_mixin() {
test_local_cfg.get("enable_cluster_metadata_upload_loop")
.set_value(false);
}

private:
scoped_config test_local_cfg;
};
4 changes: 3 additions & 1 deletion src/v/cluster/cloud_metadata/tests/uploader_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
#include "cluster/cloud_metadata/cluster_manifest.h"
#include "cluster/cloud_metadata/key_utils.h"
#include "cluster/cloud_metadata/manifest_downloads.h"
#include "cluster/cloud_metadata/tests/manual_mixin.h"
#include "cluster/cloud_metadata/uploader.h"
#include "cluster/config_frontend.h"
#include "cluster/controller_snapshot.h"
Expand All @@ -39,7 +40,8 @@ static ss::abort_source never_abort;
} // anonymous namespace

class cluster_metadata_uploader_fixture
: public s3_imposter_fixture
: public manual_metadata_upload_mixin
, public s3_imposter_fixture
, public redpanda_thread_fixture
, public enable_cloud_storage_fixture {
public:
Expand Down
3 changes: 1 addition & 2 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1743,9 +1743,8 @@ configuration::configuration()
"enable_cluster_metadata_upload_loop",
"Enables the cluster metadata upload loop.",
{.needs_restart = needs_restart::yes, .visibility = visibility::tunable},
// TODO: set to true by default once the feature is ready.
// TODO: make this runtime configurable.
false)
true)
, cloud_storage_max_segments_pending_deletion_per_partition(
*this,
"cloud_storage_max_segments_pending_deletion_per_partition",
Expand Down
6 changes: 5 additions & 1 deletion tests/rptest/scale_tests/cloud_storage_compaction_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,7 @@ def start_workload(self):
wait_until(lambda: len(self.producer.last_acked_offsets) != 0, 30)

def _init_redpanda_read_replica(self):
extra_rp_conf = dict(enable_cluster_metadata_upload_loop=False)
self.rr_si_settings = SISettings(
self.test_context,
bypass_bucket_creation=True,
Expand All @@ -141,7 +142,10 @@ def _init_redpanda_read_replica(self):
configuration["cloud_storage_segment_max_upload_interval_sec"])
self.rr_si_settings.load_context(self.logger, self.test_context)
self.rr_cluster = make_redpanda_service(
self.test_context, num_brokers=3, si_settings=self.rr_si_settings)
self.test_context,
num_brokers=3,
si_settings=self.rr_si_settings,
extra_rp_conf=extra_rp_conf)

def _create_read_repica_topic_success(self):
try:
Expand Down
10 changes: 8 additions & 2 deletions tests/rptest/tests/cluster_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -219,6 +219,10 @@ def __init__(self, *args, **kwargs):
# config version that would disrupt our tests.
rp_conf['cluster_id'] = "placeholder"

# Explicitly disable metadata uploads, since some tests may mess around
# with cloud configs and prevent uploads from succeeding.
rp_conf['enable_cluster_metadata_upload_loop'] = False

super(ClusterConfigTest, self).__init__(*args,
extra_rp_conf=rp_conf,
**kwargs)
Expand Down Expand Up @@ -1431,8 +1435,10 @@ def test_aliasing(self, prop_set: PropertyAliasData):
"""
# Aliases should work when used in bootstrap
# NOTE due to https://github.com/redpanda-data/redpanda/issues/13362 this is incomplete (see commented line)
self.redpanda.set_extra_rp_conf(
{prop_set.aliased_name: prop_set.test_values[0]})
self.redpanda.set_extra_rp_conf({
prop_set.aliased_name:
prop_set.test_values[0],
})
self.redpanda.start()
# self._check_value_everywhere(prop_set.primary_name, prop_set.values[0])

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,13 @@ def __init__(self, test_context):
self.epoch_offsets = {}

def start_second_cluster(self) -> None:
# NOTE: the RRR cluster won't have a bucket, so don't upload.
extra_rp_conf = dict(enable_cluster_metadata_upload_loop=False)
self.second_cluster = make_redpanda_service(
self.test_context, num_brokers=3, si_settings=self.rr_settings)
self.test_context,
num_brokers=3,
si_settings=self.rr_settings,
extra_rp_conf=extra_rp_conf)
self.second_cluster.start(start_si=False)

def create_source_topic(self):
Expand Down
17 changes: 11 additions & 6 deletions tests/rptest/tests/read_replica_e2e_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -157,8 +157,13 @@ def __init__(self, test_context: TestContext):
self.second_cluster = None

def start_second_cluster(self) -> None:
# NOTE: the RRR cluster won't have a bucket, so don't upload.
extra_rp_conf = dict(enable_cluster_metadata_upload_loop=False)
self.second_cluster = make_redpanda_service(
self.test_context, num_brokers=3, si_settings=self.rr_settings)
self.test_context,
num_brokers=3,
si_settings=self.rr_settings,
extra_rp_conf=extra_rp_conf)
self.second_cluster.start(start_si=False)

def create_read_replica_topic(self) -> None:
Expand Down Expand Up @@ -450,12 +455,12 @@ def __init__(self, test_context: TestContext):
cloud_storage_segment_max_upload_interval_sec=3,
fast_uploads=True))

# Read replica shouldn't have it's own bucket.
# We're adding 'none' as a bucket name without creating
# an actual bucket with such name.
# Unlike other read replica tests, create a bucket so we can upgrade to
# a version that uses cluster metadata uploads by default (that expects
# a bucket).
self.rr_settings = SISettings(
test_context,
bypass_bucket_creation=True,
bypass_bucket_creation=False,
cloud_storage_max_connections=5,
log_segment_size=self.log_segment_size,
cloud_storage_readreplica_manifest_sync_timeout_ms=500,
Expand Down Expand Up @@ -496,7 +501,7 @@ def test_upgrades(self, cloud_storage_type):
RedpandaInstaller.HEAD)
self.second_cluster._installer.install(self.second_cluster.nodes,
previous_version)
self.second_cluster.start(start_si=False)
self.second_cluster.start(start_si=True)
create_read_replica_topic(self.second_cluster, self.topic_name,
self.si_settings.cloud_storage_bucket)

Expand Down
5 changes: 4 additions & 1 deletion tests/rptest/tests/timequery_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -531,9 +531,12 @@ def __init__(self, test_context):
self.rr_cluster = None

def start_read_replica_cluster(self, num_brokers) -> None:
# NOTE: the RRR cluster won't have a bucket, so don't upload.
extra_rp_conf = dict(enable_cluster_metadata_upload_loop=False)
self.rr_cluster = make_redpanda_service(self.test_context,
num_brokers=num_brokers,
si_settings=self.rr_settings)
si_settings=self.rr_settings,
extra_rp_conf=extra_rp_conf)
self.rr_cluster.start(start_si=False)

def create_read_replica_topic(self) -> None:
Expand Down

0 comments on commit 83611a9

Please sign in to comment.