Skip to content

Commit

Permalink
Merge pull request #22760 from vbotbuildovich/backport-pr-22747-v24.2…
Browse files Browse the repository at this point in the history
….x-220

[v24.2.x] Disable log cleanup policy for controller log
  • Loading branch information
piyushredpanda authored Aug 6, 2024
2 parents b25f128 + baef6e9 commit 4744352
Show file tree
Hide file tree
Showing 3 changed files with 73 additions and 2 deletions.
6 changes: 5 additions & 1 deletion src/v/cluster/raft0_utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,14 @@ static ss::future<consensus_ptr> create_raft0(
if (!initial_brokers.empty()) {
vlog(clusterlog.info, "Current node is a cluster founder");
}
// controller log size is maintained with controller snapshots
auto overrides = std::make_unique<storage::ntp_config::default_overrides>();
overrides->cleanup_policy_bitflags = model::cleanup_policy_bitflags::none;

return pm.local()
.manage(
storage::ntp_config(model::controller_ntp, data_directory),
storage::ntp_config(
model::controller_ntp, data_directory, std::move(overrides)),
raft::group_id(0),
std::move(initial_brokers),
raft::with_learner_recovery_throttle::no,
Expand Down
5 changes: 4 additions & 1 deletion src/v/security/audit/audit_log_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -391,7 +391,10 @@ ss::future<> audit_client::create_internal_topic() {
.value{retain_forever}},
kafka::createable_topic_config{
.name = ss::sstring(kafka::topic_property_retention_duration),
.value{seven_days}}}};
.value{seven_days}},
kafka::createable_topic_config{
.name = ss::sstring(kafka::topic_property_cleanup_policy),
.value = "delete"}}};
vlog(
adtlog.info, "Creating audit log topic with settings: {}", audit_topic);
const auto resp = co_await _client.create_topic({std::move(audit_topic)});
Expand Down
64 changes: 64 additions & 0 deletions tests/rptest/tests/controller_partition_invariants_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
# Copyright 2024 Redpanda Data, Inc.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.md
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0

import re
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.cluster import cluster
from rptest.services.admin import Admin
from rptest.clients.rpk import RpkTool

from ducktape.mark import matrix
from rptest.services.metrics_check import MetricCheck

import time


class ControllerLogInvariantsTest(RedpandaTest):
def __init__(self, *args, **kwargs):
super().__init__(
*args,
num_brokers=3,
**kwargs,
# disable controller snapshots
extra_rp_conf={'controller_snapshot_max_age_sec': 3600})

@cluster(num_nodes=3)
@matrix(cluster_cleanup_policy=['compact', 'delete'])
def test_controller_is_not_compacted_nor_deleted(self,
cluster_cleanup_policy):

metrics = [
MetricCheck(self.logger, self.redpanda, n,
re.compile("vectorized_storage_log.*"),
{'topic': 'controller'}) for n in self.redpanda.nodes
]

rpk = RpkTool(self.redpanda)
for _ in range(10):
for _ in range(20):
# create and delete the same topic
rpk.create_topic('topic-to-compact', partitions=1, replicas=3)
rpk.delete_topic('topic-to-compact')
# transfer controller leadership to trigger segment roll
Admin(self.redpanda).partition_transfer_leadership(
namespace="redpanda", topic="controller", partition=0)

self.redpanda.set_cluster_config({
"log_compaction_interval_ms":
1000,
'log_cleanup_policy':
cluster_cleanup_policy
})

time.sleep(30)
for m in metrics:
m.expect([("vectorized_storage_log_compacted_segment_total",
lambda a, b: b == a == 0),
("vectorized_storage_log_log_segments_removed_total",
lambda a, b: b == a == 0)])

0 comments on commit 4744352

Please sign in to comment.