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


##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.tiered.storage.TieredStorageTestBuilder;
+import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test to verify that the active segment is rolled and uploaded to remote 
storage when the segment breaches the
+ * local log retention policy.
+ */
+public class RollAndOffloadActiveSegmentTest extends TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final Integer broker0 = 0;
+        final String topicA = "topicA";
+        final Integer p0 = 0;
+        final Integer partitionCount = 1;
+        final Integer replicationFactor = 1;
+        final Integer maxBatchCountPerSegment = 1;
+        final Map<Integer, List<Integer>> replicaAssignment = null;
+        final boolean enableRemoteLogStorage = true;
+
+        // Create topicA with 1 partition, 1 RF and enabled with remote 
storage.
+        builder.createTopic(topicA, partitionCount, replicationFactor, 
maxBatchCountPerSegment, replicaAssignment,
+                        enableRemoteLogStorage)
+                // update the topic config such that it triggers the rolling 
of the active segment
+                .updateTopicConfig(topicA, configsToBeAdded(), 
Collections.emptyList())
+                // produce events to partition 0 and expect all the 4 segments 
to be offloaded
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new 
KeyValueSpec("k0", "v0"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new 
KeyValueSpec("k1", "v1"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 2, new 
KeyValueSpec("k2", "v2"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 3, new 
KeyValueSpec("k3", "v3"))
+                .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 4L)
+                .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new 
KeyValueSpec("k1", "v1"),
+                        new KeyValueSpec("k2", "v2"), new KeyValueSpec("k3", 
"v3"))
+                // consume from the beginning of the topic to read data from 
local and remote storage
+                .expectFetchFromTieredStorage(broker0, topicA, p0, 4)
+                .consume(topicA, p0, 0L, 4, 4);
+    }
+
+    private Map<String, String> configsToBeAdded() {
+        // Update localLog retentionMs to 1 ms and segment roll-time to 10 ms
+        Map<String, String> topicConfigs = new HashMap<>();
+        topicConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "1");

Review Comment:
   `localRetentionMs` is set to 1 ms, once it passed, the active segment will 
be rolled over and offloaded to the remote storage.
   
   Note that all the tests under tiered-storage integration test specifies the 
`localRetentionBytes` as 1 byte (See: 
[TieredStorageTestUtils](https://sourcegraph.com/github.com/apache/kafka/-/blob/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java?L174))
 to offload the passive segment ASAP and to keep the active segment on local 
storage to assert reading the data from both local and remote storage. We don't 
want to change the 
[deleteRetentionSizeBreachedSegments](https://sourcegraph.com/github.com/apache/kafka/-/blob/core/src/main/scala/kafka/log/UnifiedLog.scala?L1502-1513)
 behaviour.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to