Skip to content

Commit

Permalink
ducky: introduce compaction_e2e_idempotency_test
Browse files Browse the repository at this point in the history
compaction e2e idempotency test is an iteration of compaction end
to end test which switches to more powerful verifier for idempotency
workload which also isn't affected by the ssh errors
  • Loading branch information
rystsov committed Jan 26, 2023
1 parent 84119c4 commit d75253c
Show file tree
Hide file tree
Showing 2 changed files with 144 additions and 7 deletions.
142 changes: 142 additions & 0 deletions tests/rptest/tests/compaction_e2e_idempotency_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
# Copyright 2022 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 sys
from ducktape.mark import matrix
from ducktape.utils.util import wait_until
from rptest.clients.rpk import RpkTool
from rptest.clients.types import TopicSpec

from rptest.tests.redpanda_test import RedpandaTest
from rptest.clients.default import DefaultClient
from rptest.services.cluster import cluster

from rptest.services.compacted_verifier import CompactedVerifier, Workload


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

self.segment_size = 5 * 1024 * 1024 if not self.debug_mode else 1024 * 1024
self.partition_count = 3

super(CompactionE2EIdempotencyTest,
self).__init__(test_context=test_context,
extra_rp_conf=extra_rp_conf)

def topic_segments(self):
storage = self.redpanda.node_storage(self.redpanda.nodes[0])
topic_partitions = storage.partitions("kafka", self.topic)

return [len(p.segments) for p in topic_partitions]

@cluster(num_nodes=4)
# sys.maxsize is a special high cardinality case where no keys should be
# compacted away. With transactions enabled, all the aborted batches
# should be compacted away from the log and should not show up in
# consumed list.
@matrix(initial_cleanup_policy=[
TopicSpec.CLEANUP_COMPACT, TopicSpec.CLEANUP_DELETE
])
def test_basic_compaction(self, initial_cleanup_policy):
'''
Basic end to end compaction logic test. The test verifies if last value
consumed for each key matches the last produced value for the same key.
The test waits for the compaction to merge some adjacent segments.
The test is parametrized with initial cleanup policy parameter.
This way the test validates both recovery and inflight generation
of compaction indices.
Note: This test has many parameter combinations and is prone to long
running times and/or substantial disk usage. Keep this in mind before
adding further parameters.
'''
key_set_cardinality = 10
# creating the topic manually instead of relying on topics
# because the topic depends on the test parameter
client = DefaultClient(self.redpanda)
self.topics = [
TopicSpec(partition_count=self.partition_count,
replication_factor=3,
segment_bytes=self.segment_size,
cleanup_policy=initial_cleanup_policy)
]
client.create_topic(self.topics[0])

rw_verifier = CompactedVerifier(self.test_context, self.redpanda,
Workload.IDEMPOTENCY)
rw_verifier.start()

rw_verifier.remote_start_producer(self.redpanda.brokers(), self.topic,
self.partition_count)
self.logger.info(f"Waiting for 10000 writes to ensure progress")
rw_verifier.ensure_progress(10000, 30)
self.logger.info(f"The test made progress")

rpk = RpkTool(self.redpanda)
cfgs = rpk.describe_topic_configs(self.topic)
self.logger.debug(f"Initial topic {self.topic} configuration: {cfgs}")

# @mmaslankaprv why do we need to
# make sure that segments will not be compacted when we populate
# topic partitions with data
self.logger.info(
f"setting log_compaction_interval_ms to {3600 * 1000}")
rpk.cluster_config_set("log_compaction_interval_ms", str(3600 * 1000))

def segment_number_matches(predicate):
segments_per_partition = self.topic_segments()
self.logger.debug(
f"Topic {self.topic} segments per partition: {segments_per_partition}"
)

return all([predicate(n) for n in segments_per_partition])

timeout_sec = 300
self.logger.info(
f"wait for multiple segments to appear in topic partitions")
# wait for multiple segments to appear in topic partitions
wait_until(lambda: segment_number_matches(lambda s: s >= 5),
timeout_sec=timeout_sec,
backoff_sec=2)

# enable compaction, if topic isn't compacted
if initial_cleanup_policy == TopicSpec.CLEANUP_DELETE:
rpk.alter_topic_config(self.topic,
set_key="cleanup.policy",
set_value=TopicSpec.CLEANUP_COMPACT)

self.logger.info(f"Waiting for 10000 writes to ensure progress")
rw_verifier.ensure_progress(10000, 30)
self.logger.info(f"The test made progress, stopping producer")
rw_verifier.remote_stop_producer()
rw_verifier.remote_wait_producer()
self.logger.info(f"Producer is stopped")

current_segments_per_partition = self.topic_segments()
self.logger.info(
f"Stopped producer, segments per partition: {current_segments_per_partition}"
)
# make compaction frequent
self.logger.info(f"setting log_compaction_interval_ms to {3600}")
rpk.cluster_config_set("log_compaction_interval_ms", str(3000))
self.logger.info(f"waiting for compaction to happen")

# it looks like we're guessing that the number of compacted
# segments is less than 5, a place or a potential timeout when
# we guessed wrong
wait_until(lambda: segment_number_matches(lambda s: s < 5),
timeout_sec=timeout_sec,
backoff_sec=2)

self.logger.info(f"enable consumer and validate consumed records")
rw_verifier.remote_start_consumer()
rw_verifier.remote_wait_consumer()
9 changes: 2 additions & 7 deletions tests/rptest/tests/compaction_end_to_end_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -73,11 +73,9 @@ def topic_segments(self):
initial_cleanup_policy=[
TopicSpec.CLEANUP_COMPACT, TopicSpec.CLEANUP_DELETE
],
transactions=[True, False],
tx_inject_aborts=[True, False])
def test_basic_compaction(self, key_set_cardinality,
initial_cleanup_policy, transactions,
tx_inject_aborts):
initial_cleanup_policy, tx_inject_aborts):
'''
Basic end to end compaction logic test. The test verifies if last value
consumed for each key matches the last produced value for the same key.
Expand All @@ -92,10 +90,7 @@ def test_basic_compaction(self, key_set_cardinality,
adding further parameters.
'''

if not transactions and tx_inject_aborts:
# Not a valid combination
cleanup_on_early_exit(self)
return
transactions = True

# skip debug mode tests for keys with high cardinality
if key_set_cardinality > 10 and self.debug_mode:
Expand Down

0 comments on commit d75253c

Please sign in to comment.