You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/07/16 19:43:38 UTC

[GitHub] [kafka] jsancio commented on a change in pull request #11053: KAFKA-13015 Ducktape System Tests for Metadata Snapshots

jsancio commented on a change in pull request #11053:
URL: https://github.com/apache/kafka/pull/11053#discussion_r671454434



##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,

Review comment:
       `lambda: not self.file_exists(node, file_path)`

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,

Review comment:
       `lambda: self.file_exists(node, file_path)`

##########
File path: tests/kafkatest/services/kafka/templates/log4j.properties
##########
@@ -111,6 +111,8 @@ log4j.logger.kafka.client.ClientUtils={{ log_level|default("DEBUG") }}, kafkaInf
 log4j.logger.kafka.perf={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
 log4j.logger.kafka.perf.ProducerPerformance$ProducerThread={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
 log4j.logger.kafka={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.org.apache.kafka.raft={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.org.apache.kafka.snapshot={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender

Review comment:
       I think we can remove these lines since the root logger is already configured to this same level to stdout:
   ```
   log4j.rootLogger={{ log_level|default("DEBUG") }}, stdout
   ```

##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -154,6 +154,9 @@ class for details.
     DATA_LOG_DIR_1 = "%s-1" % (DATA_LOG_DIR_PREFIX)
     DATA_LOG_DIR_2 = "%s-2" % (DATA_LOG_DIR_PREFIX)
     CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties")
+    METADATA_LOG_DIR = os.path.join (PERSISTENT_ROOT, "kafka-metadata-logs")
+    METADATA_SNAPSHOT_SEARCH_STR = "%s/@metadata-0/*.checkpoint" % METADATA_LOG_DIR
+    METADATA_FIRST_LOG = "%s/@metadata-0/00000000000000000000.log" % METADATA_LOG_DIR

Review comment:
       Should we use `os.path.join` for these two path manipulations?

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)
+
+
+    @cluster(num_nodes=9)
+    @parametrize(metadata_quorum=quorum.remote_kraft)
+    # TODO: enable after concurrent startup has been merged
+    #@matrix(metadata_quorum=quorum.all_kraft)
+    def test_broker(self, metadata_quorum=quorum.colocated_kraft):
+        # Scenario -- stop a broker node and create metadata changes.
+        # Restart the broker and let it catch up
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 1: kill-create_topics-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.stop_node(node)
+        # Now modify the cluster to create more metadata changes
+        self.topics_created += self.create_n_topics(topic_count=10)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 2: kill-clean-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.clean_node(node)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        # and create a topic where the affected broker must be the leader
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 3: kill-clean-start-verify-produce on node %s", self.kafka.who_am_i(node))
+        self.kafka.clean_node(node)
+
+        # Create a topic where the affected broker must be the leader
+        broker_topic = "test_topic_%d" % self.topics_created
+        self.topics_created += 1
+        print("Creating topic %s" % broker_topic, flush=True)

Review comment:
       Use `self.logger`.

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)

Review comment:
       Use `self.logger`.

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]

Review comment:
       Should we move this complexity of remote vs co-located to the Kafka service?

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)
+
+
+    @cluster(num_nodes=9)
+    @parametrize(metadata_quorum=quorum.remote_kraft)
+    # TODO: enable after concurrent startup has been merged

Review comment:
       Are you planning to implement this TODO in this PR? If not, let's remove this TODO and create a Jira.
   
   Do we need to fix this issue before 3.0? In other words is this a blocker for 3.0?

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)
+
+
+    @cluster(num_nodes=9)
+    @parametrize(metadata_quorum=quorum.remote_kraft)
+    # TODO: enable after concurrent startup has been merged
+    #@matrix(metadata_quorum=quorum.all_kraft)
+    def test_broker(self, metadata_quorum=quorum.colocated_kraft):
+        # Scenario -- stop a broker node and create metadata changes.
+        # Restart the broker and let it catch up
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 1: kill-create_topics-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.stop_node(node)
+        # Now modify the cluster to create more metadata changes
+        self.topics_created += self.create_n_topics(topic_count=10)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 2: kill-clean-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.clean_node(node)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        # and create a topic where the affected broker must be the leader
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 3: kill-clean-start-verify-produce on node %s", self.kafka.who_am_i(node))
+        self.kafka.clean_node(node)
+
+        # Create a topic where the affected broker must be the leader
+        broker_topic = "test_topic_%d" % self.topics_created
+        self.topics_created += 1
+        print("Creating topic %s" % broker_topic, flush=True)
+        topic_cfg = {
+            "topic": broker_topic,
+            "replica-assignment": self.kafka.idx(node),
+            "configs": {"min.insync.replicas": 1}
+        }
+        self.kafka.create_topic(topic_cfg)
+
+        self.kafka.start_node(node)
+
+        # Produce to the newly created topic to ensure broker has caught up
+        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
+                                           broker_topic, throughput=self.producer_throughput,
+                                           message_validator=is_int)
+
+        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka,
+                                        broker_topic, consumer_timeout_ms=30000,
+                                        message_validator=is_int)
+        self.start_producer_and_consumer()
+        self.stop_producer_and_consumer()
+        self.validate()
+
+        self.validate_success()
+
+    @cluster(num_nodes=8)
+    @parametrize(metadata_quorum=quorum.remote_kraft)
+    # TODO: enable after concurrent startup has been merged
+    #@matrix(metadata_quorum=quorum.all_kraft)
+    def test_controller(self, metadata_quorum=quorum.colocated_kraft):
+        # Scenario -- stop a controller node and create metadata changes.
+        # Restart the controller and let it catch up
+        node = self.choose_controller_node()
+        self.logger.debug("Scenario 1: kill-create_topics-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.controller_quorum.stop_node(node)
+        # Now modify the cluster to create more metadata changes
+        self.topics_created += self.create_n_topics(topic_count=10)
+        self.kafka.controller_quorum.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init controller with a clean kafka dir
+        node = self.choose_controller_node()
+        self.logger.debug("Scenario 2: kill-clean-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.controller_quorum.clean_node(node)
+        self.kafka.controller_quorum.start_node(node)
+        self.validate_success()

Review comment:
       It is not clear to me that this is enough to verify that the controller loaded a snapshot and caught to the leader.

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]

Review comment:
       Should we also make this random?

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)

Review comment:
       I took a look at the implementations for `self.kafka.all_nodes_support_topic_ids` and `self.kafka.check_protocol_errors` and it doesn't look like these checks are enough.
   
   Also, this call `self.kafka.check_protocol_errors(self)` seems strange. It looks like the implementation of that method never uses that parameter.

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)

Review comment:
       Missing spaces before and after the `%` string formatter. This applies to a few places I this file.

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)

Review comment:
       Is this enough to verify success?

##########
File path: tests/kafkatest/tests/core/snapshot_test.py
##########
@@ -0,0 +1,227 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize, matrix
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.kafka import quorum
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import config_property
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+import random
+
+class TestSnapshots(ProduceConsumeValidateTest):
+
+    METADATA_LOG_RETENTION_BYTES = "4096"
+    METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS = "4096"
+    METADATA_LOG_SEGMENT_BYTES = "9000000"
+    METADATA_LOG_SEGMENT_MS = "10000"
+
+    def __init__(self, test_context):
+        super(TestSnapshots, self).__init__(test_context=test_context)
+        self.topics_created = 0
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+        self.num_nodes = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+        security_protocol = 'PLAINTEXT'
+        # Setup Custom Config to ensure snapshot will be generated deterministically
+        self.kafka = KafkaService(self.test_context, self.num_nodes, zk=None,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}},
+                                  server_prop_overrides=[
+                                      [config_property.METADATA_LOG_DIR, KafkaService.METADATA_LOG_DIR],
+                                      [config_property.METADATA_LOG_SEGMENT_MS, TestSnapshots.METADATA_LOG_SEGMENT_MS],
+                                      [config_property.METADATA_LOG_RETENTION_BYTES, TestSnapshots.METADATA_LOG_RETENTION_BYTES],
+                                      [config_property.METADATA_LOG_SEGMENT_BYTES, TestSnapshots.METADATA_LOG_SEGMENT_BYTES],
+                                      [config_property.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS, TestSnapshots.METADATA_LOG_BYTES_BETWEEN_SNAPSHOTS]
+                                  ])
+
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.security_protocol = security_protocol
+
+    def setUp(self):
+        # Start the cluster and ensure that a snapshot is generated
+        self.logger.info("Starting the cluster and running until snapshot creation")
+
+        assert quorum.for_test(self.test_context) in quorum.all_kraft, \
+                "Snapshot test should be run Kraft Modes only"
+
+        self.kafka.start()
+
+        topic_count = 10
+        self.topics_created += self.create_n_topics(topic_count)
+
+        if self.kafka.remote_controller_quorum:
+            controller_nodes = self.kafka.remote_controller_quorum.nodes
+        else:
+            controller_nodes = self.kafka.nodes[:self.kafka.num_nodes_controller_role]
+
+        # Waiting for snapshot creation and first log segment
+        # cleanup on all controller nodes
+        for node in controller_nodes:
+            self.logger.debug("Waiting for snapshot on: %s" % self.kafka.who_am_i(node))
+            self.wait_for_log_segment_delete(node)
+            self.wait_for_snapshot(node)
+        self.logger.debug("Verified Snapshots exist on controller nodes")
+
+    def create_n_topics(self, topic_count):
+        for i in range(self.topics_created, topic_count):
+            topic = "test_topic_%d" % i
+            print("Creating topic %s" % topic, flush=True)
+            topic_cfg = {
+                "topic": topic,
+                "partitions": self.partitions,
+                "replication-factor": self.replication_factor,
+                "configs": {"min.insync.replicas": 2}
+            }
+            self.kafka.create_topic(topic_cfg)
+        self.logger.debug("Created %d more topics" % topic_count)
+        return topic_count
+
+    def wait_for_log_segment_delete(self, node):
+        file_path = self.kafka.METADATA_FIRST_LOG
+        # Wait until the first log segment in metadata log is marked for deletion
+        wait_until(lambda: self.file_exists(node, file_path) == False,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify cleanup of log file %s in a reasonable amount of time" % file_path)
+
+    def wait_for_snapshot(self, node):
+        # Wait for a snapshot file to show up
+        file_path = self.kafka.METADATA_SNAPSHOT_SEARCH_STR
+        wait_until(lambda: self.file_exists(node, file_path) == True,
+                   timeout_sec=100,
+                   backoff_sec=1, err_msg="Not able to verify snapshot existence in a reasonable amount of time")
+
+    def choose_controller_node(self):
+        if self.kafka.remote_controller_quorum:
+            # We are running in remote kraft
+            controller_node = random.choice(self.kafka.remote_controller_quorum.nodes)
+        else:
+            # This is colocated kraft
+            # Picking the first node as that is guaranteed to be a controller
+            controller_node = self.kafka.nodes[0]
+
+        self.logger.debug("Picking the controller node %s" % self.kafka.who_am_i(controller_node))
+        return controller_node
+
+    def file_exists(self, node, file_path):
+        # Check if the first log segment is cleaned up
+        self.logger.debug("Checking if file %s exists"%file_path)
+        cmd = "ls %s" % file_path
+        files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
+
+        if len(files) is 0:
+            self.logger.debug("File %s does not exist"%file_path)
+            return False
+        else:
+            self.logger.debug("File %s was found"%file_path)
+            return True
+
+    def validate_success(self):
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.all_nodes_support_topic_ids()
+        assert self.kafka.check_protocol_errors(self)
+
+
+    @cluster(num_nodes=9)
+    @parametrize(metadata_quorum=quorum.remote_kraft)
+    # TODO: enable after concurrent startup has been merged
+    #@matrix(metadata_quorum=quorum.all_kraft)
+    def test_broker(self, metadata_quorum=quorum.colocated_kraft):
+        # Scenario -- stop a broker node and create metadata changes.
+        # Restart the broker and let it catch up
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 1: kill-create_topics-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.stop_node(node)
+        # Now modify the cluster to create more metadata changes
+        self.topics_created += self.create_n_topics(topic_count=10)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 2: kill-clean-start on node %s", self.kafka.who_am_i(node))
+        self.kafka.clean_node(node)
+        self.kafka.start_node(node)
+        self.validate_success()
+
+        # Scenario -- Re-init broker after cleaning up all persistent state
+        # and create a topic where the affected broker must be the leader
+        node = random.choice(self.kafka.nodes)
+        self.logger.debug("Scenario 3: kill-clean-start-verify-produce on node %s", self.kafka.who_am_i(node))

Review comment:
       Do we think that we need these 3 scenarios in this test? Do you think that we only need this scenario to verify that snapshot are getting generated and loaded by the brokers?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org