kamalcph commented on code in PR #14116:
URL: https://github.com/apache/kafka/pull/14116#discussion_r1286072822


##########
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java:
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.
+ */
+package org.apache.kafka.tiered.storage;
+
+import org.apache.kafka.tiered.storage.actions.BounceBrokerAction;
+import org.apache.kafka.tiered.storage.actions.ConsumeAction;
+import org.apache.kafka.tiered.storage.actions.CreatePartitionsAction;
+import org.apache.kafka.tiered.storage.actions.CreateTopicAction;
+import org.apache.kafka.tiered.storage.actions.DeleteRecordsAction;
+import org.apache.kafka.tiered.storage.actions.DeleteTopicAction;
+import org.apache.kafka.tiered.storage.actions.EraseBrokerStorageAction;
+import org.apache.kafka.tiered.storage.actions.ExpectBrokerInISRAction;
+import org.apache.kafka.tiered.storage.actions.ExpectEmptyRemoteStorageAction;
+import org.apache.kafka.tiered.storage.actions.ExpectLeaderAction;
+import 
org.apache.kafka.tiered.storage.actions.ExpectLeaderEpochCheckpointAction;
+import org.apache.kafka.tiered.storage.actions.ExpectListOffsetsAction;
+import 
org.apache.kafka.tiered.storage.actions.ExpectTopicIdToMatchInRemoteStorageAction;
+import 
org.apache.kafka.tiered.storage.actions.ExpectUserTopicMappedToMetadataPartitionsAction;
+import org.apache.kafka.tiered.storage.actions.ProduceAction;
+import org.apache.kafka.tiered.storage.actions.ReassignReplicaAction;
+import org.apache.kafka.tiered.storage.actions.ShrinkReplicaAction;
+import org.apache.kafka.tiered.storage.actions.StartBrokerAction;
+import org.apache.kafka.tiered.storage.actions.StopBrokerAction;
+import org.apache.kafka.tiered.storage.actions.UpdateBrokerConfigAction;
+import org.apache.kafka.tiered.storage.actions.UpdateTopicConfigAction;
+import org.apache.kafka.tiered.storage.specs.ConsumableSpec;
+import org.apache.kafka.tiered.storage.specs.DeletableSpec;
+import org.apache.kafka.tiered.storage.specs.ExpandPartitionCountSpec;
+import org.apache.kafka.tiered.storage.specs.FetchableSpec;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+import org.apache.kafka.tiered.storage.specs.OffloadableSpec;
+import org.apache.kafka.tiered.storage.specs.OffloadedSegmentSpec;
+import org.apache.kafka.tiered.storage.specs.ProducableSpec;
+import org.apache.kafka.tiered.storage.specs.RemoteDeleteSegmentSpec;
+import org.apache.kafka.tiered.storage.specs.RemoteFetchSpec;
+import org.apache.kafka.tiered.storage.specs.TopicSpec;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public final class TieredStorageTestBuilder {
+
+    private final int defaultProducedBatchSize = 1;
+    private final long defaultEarliestLocalOffsetExpectedInLogDirectory = 0;
+
+    private Map<TopicPartition, ProducableSpec> producables = new HashMap<>();
+    private Map<TopicPartition, List<OffloadableSpec>> offloadables = new 
HashMap<>();
+    private Map<TopicPartition, ConsumableSpec> consumables = new HashMap<>();
+    private Map<TopicPartition, FetchableSpec> fetchables = new HashMap<>();
+    private Map<TopicPartition, List<DeletableSpec>> deletables = new 
HashMap<>();
+    private List<TieredStorageTestAction> actions = new ArrayList<>();
+
+    public TieredStorageTestBuilder() {
+    }
+
+    public TieredStorageTestBuilder createTopic(String topic,
+                                                Integer partitionCount,
+                                                Integer replicationFactor,
+                                                Integer 
maxBatchCountPerSegment,
+                                                Map<Integer, List<Integer>> 
replicaAssignment,
+                                                Boolean 
enableRemoteLogStorage) {
+        assert maxBatchCountPerSegment >= 1 : "Segments size for topic " + 
topic + " needs to be >= 1";
+        assert partitionCount >= 1 : "Partition count for topic " + topic + " 
needs to be >= 1";
+        assert replicationFactor >= 1 : "Replication factor for topic " + 
topic + " needs to be >= 1";
+        maybeCreateProduceAction();
+        maybeCreateConsumeActions();
+        Map<String, String> properties = new HashMap<>();
+        properties.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, 
enableRemoteLogStorage.toString());
+        TopicSpec topicSpec = new TopicSpec(topic, partitionCount, 
replicationFactor, maxBatchCountPerSegment,
+                replicaAssignment, properties);
+        actions.add(new CreateTopicAction(topicSpec));
+        return this;
+    }
+
+    public TieredStorageTestBuilder createPartitions(String topic,
+                                                     Integer partitionCount,
+                                                     Map<Integer, 
List<Integer>> replicaAssignment) {
+        assert partitionCount >= 1 : "Partition count for topic " + topic + " 
needs to be >= 1";
+        maybeCreateProduceAction();
+        maybeCreateConsumeActions();
+        ExpandPartitionCountSpec spec = new ExpandPartitionCountSpec(topic, 
partitionCount, replicaAssignment);
+        actions.add(new CreatePartitionsAction(spec));
+        return this;
+    }
+
+    public TieredStorageTestBuilder updateTopicConfig(String topic,
+                                                      Map<String, String> 
configsToBeAdded,
+                                                      List<String> 
configsToBeDeleted) {
+        assert !configsToBeAdded.isEmpty() || !configsToBeDeleted.isEmpty()
+                : "Topic " + topic + " configs shouldn't be empty";
+        maybeCreateProduceAction();
+        maybeCreateConsumeActions();
+        actions.add(new UpdateTopicConfigAction(topic, configsToBeAdded, 
configsToBeDeleted));
+        return this;
+    }
+
+    public TieredStorageTestBuilder updateBrokerConfig(Integer brokerId,
+                                                       Map<String, String> 
configsToBeAdded,
+                                                       List<String> 
configsToBeDeleted) {
+        assert !configsToBeAdded.isEmpty() || !configsToBeDeleted.isEmpty()
+                : "Broker " + brokerId + " configs shouldn't be empty";
+        maybeCreateProduceAction();
+        maybeCreateConsumeActions();
+        actions.add(new UpdateBrokerConfigAction(brokerId, configsToBeAdded, 
configsToBeDeleted));
+        return this;
+    }
+
+    public TieredStorageTestBuilder deleteTopic(List<String> topics) {
+        maybeCreateProduceAction();
+        maybeCreateConsumeActions();
+        topics.forEach(topic -> actions.add(buildDeleteTopicAction(topic, 
true)));
+        return this;
+    }
+
+    public TieredStorageTestBuilder produce(String topic,

Review Comment:
   Updated the `produce` and `consume` methods to be deterministic. 
   
   Previously, the DSL can be written as:
   ```
   .produce(topicA, p0, new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", 
"v2"))
   .withBatchSize(topicA, p0, batchSize)
   .expectSegmentToBeOffloaded(broker, topicA, p0, 0, new KeyValueSpec("k1", 
"v1"))
   .expectEarliestLocalOffsetInLogDirectory(topicBA, p0, 1L)
   ```
   
   Now, with this change, the expectation needs to be called before the 
`produce` action:
   ```
   .withBatchSize(topicA, p0, batchSize)
   .expectSegmentToBeOffloaded(broker, topicA, p0, 0, new KeyValueSpec("k1", 
"v1"))
   .expectEarliestLocalOffsetInLogDirectory(topicBA, p0, 1L)
   .produce(topicA, p0, new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", 
"v2"))
   ```
   
   Updated the code as the changes are intuitive to the user.
   



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to