Skip to content

Commit

Permalink
rptest: set min_cleanable_dirty_ratio for ducktape tests
Browse files Browse the repository at this point in the history
Most of these tests expect/need unconditional compacting of the log.
Set the `min_cleanable_dirty_ratio` at the topic/cluster level in these
tests to mimic the old compaction scheduling behavior before this
configuration was added.
  • Loading branch information
WillemKauf committed Feb 27, 2025
1 parent 6697885 commit cba07b8
Show file tree
Hide file tree
Showing 5 changed files with 15 additions and 12 deletions.
3 changes: 2 additions & 1 deletion tests/rptest/tests/datalake/compaction_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,8 @@ def __init__(self, test_ctx, *args, **kwargs):
"iceberg_enabled": "true",
"iceberg_catalog_commit_interval_ms": 5000,
"datalake_coordinator_snapshot_max_delay_secs": 10,
"log_compaction_interval_ms": 5000
"log_compaction_interval_ms": 5000,
"min_cleanable_dirty_ratio": 0.0
},
*args,
**kwargs)
Expand Down
3 changes: 2 additions & 1 deletion tests/rptest/tests/e2e_shadow_indexing_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,8 @@ def __init__(self, test_context, extra_rp_conf=None, environment=None):
cloud_storage_cluster_metadata_upload_interval_ms=1000,
# Tests may configure spillover manually.
cloud_storage_spillover_manifest_size=None,
controller_snapshot_max_age_sec=1)
controller_snapshot_max_age_sec=1,
min_cleanable_dirty_ratio=0.0)
if extra_rp_conf:
for k, v in conf.items():
extra_rp_conf[k] = v
Expand Down
3 changes: 2 additions & 1 deletion tests/rptest/tests/log_compaction_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ def __init__(self, test_context):
'log_segment_size': 2 * 1024**2, # 2 MiB
'retention_bytes': 25 * 1024**2, # 25 MiB
'compacted_log_segment_size': 1024**2, # 1 MiB
'storage_compaction_key_map_memory': key_map_memory_kb * 1024
'storage_compaction_key_map_memory': key_map_memory_kb * 1024,
'min_cleanable_dirty_ratio': 0.0
}

# This environment variable is required to get around the map memory bounds
Expand Down
11 changes: 5 additions & 6 deletions tests/rptest/tests/shadow_indexing_compacted_topic_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,11 @@ def __init__(self, test_context):
self.si_settings = SISettings(test_context,
cloud_storage_max_connections=5,
fast_uploads=True)
extra_rp_conf = dict(
enable_leader_balancer=False,
partition_autobalancing_mode="off",
group_initial_rebalance_delay=300,
compacted_log_segment_size=self.segment_size,
)
extra_rp_conf = dict(enable_leader_balancer=False,
partition_autobalancing_mode="off",
group_initial_rebalance_delay=300,
compacted_log_segment_size=self.segment_size,
min_cleanable_dirty_ratio=0.0)
self.redpanda = make_redpanda_service(context=self.test_context,
num_brokers=self.num_brokers,
si_settings=self.si_settings,
Expand Down
7 changes: 4 additions & 3 deletions tests/rptest/transactions/compaction_e2e_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@

class CompactionE2EIdempotencyTest(RedpandaTest):
def __init__(self, test_context):
extra_rp_conf = {}
extra_rp_conf = {"min_cleanable_dirty_ratio": 0.0}

self.segment_size = 5 * 1024 * 1024
self.partition_count = 3
Expand Down Expand Up @@ -186,7 +186,8 @@ def __init__(self, test_context):
# keep read size low to ensure reads fall within a transaction
extra_rp_conf = {
"raft_recovery_default_read_size": 1024,
"log_compaction_interval_ms": 2000
"log_compaction_interval_ms": 2000,
"min_cleanable_dirty_ratio": 0.0
}
super(CompactionWithRecoveryTest,
self).__init__(test_context=test_context,
Expand Down Expand Up @@ -254,7 +255,7 @@ def __init__(self, test_context):
self.segment_size = 5 * 1024 * 1024
super(CompactionE2ERebootTest,
self).__init__(test_context=test_context,
extra_rp_conf={},
extra_rp_conf={"min_cleanable_dirty_ratio": 0.0},
log_level="trace")

@skip_debug_mode
Expand Down

0 comments on commit cba07b8

Please sign in to comment.