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


##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 kafka.api.TransactionsTest;
+import kafka.server.HostedPartition;
+import kafka.server.KafkaBroker;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+import static 
org.apache.kafka.tiered.storage.utils.TieredStorageTestUtils.createPropsForRemoteStorage;
+import static 
org.apache.kafka.tiered.storage.utils.TieredStorageTestUtils.createTopicConfigForRemoteStorage;
+import static 
org.apache.kafka.tiered.storage.utils.TieredStorageTestUtils.STORAGE_WAIT_TIMEOUT_SEC;
+
+public class TransactionsWithTieredStoreTest extends TransactionsTest {
+
+    private String testClassName;
+    private String storageDirPath;
+
+    @BeforeEach
+    @Override
+    public void setUp(TestInfo testInfo) {
+        testClassName = 
testInfo.getTestClass().get().getSimpleName().toLowerCase(Locale.getDefault());
+        storageDirPath = TestUtils.tempDirectory("kafka-remote-tier-" + 
testClassName).getAbsolutePath();
+        super.setUp(testInfo);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public void modifyConfigs(Seq<Properties> props) {
+        for (Properties p : JavaConverters.seqAsJavaList(props)) {
+            p.putAll(overridingProps());
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public Seq<Properties> kraftControllerConfigs() {
+        return 
JavaConverters.asScalaBuffer(Collections.singletonList(overridingProps())).toSeq();
+    }
+
+    private Properties overridingProps() {
+        int numRemoteLogMetadataPartitions = 3;
+        boolean deleteOnClose = false;
+        return createPropsForRemoteStorage(testClassName, storageDirPath, 
brokerCount(),
+                numRemoteLogMetadataPartitions, deleteOnClose);
+    }
+
+    @Override
+    public Properties topicConfig() {
+        boolean enableRemoteStorage = true;
+        long localLogRetentionBytes = 1;
+        int maxBatchCountPerSegment = 1;
+        Properties overridingTopicProps = super.topicConfig();
+        overridingTopicProps.putAll(createTopicConfigForRemoteStorage(
+                enableRemoteStorage, localLogRetentionBytes, 
maxBatchCountPerSegment));
+        return overridingTopicProps;
+    }
+
+    // NOTE: Not able to refer TestInfoUtils#TestWithParameterizedQuorumName() 
in the ParameterizedTest name.
+    @ParameterizedTest(name = "{displayName}.quorum={0}")
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testFencingOnTransactionExpiration(String quorum) {
+        // FIXME: Disabled the test with tiered storage since it's failing and 
needs to be fixed.
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public void 
maybeWaitForAtLeastOneSegmentUpload(scala.collection.immutable.Seq<TopicPartition>
 topicPartitions) {
+        JavaConverters.seqAsJavaList(topicPartitions).forEach(topicPartition 
-> {
+            List<BrokerLocalStorage> localStorages = 
JavaConverters.bufferAsJavaList(brokers()).stream()
+                    .map(b -> new BrokerLocalStorage(b.config().brokerId(), 
b.config().logDirs().head(), STORAGE_WAIT_TIMEOUT_SEC))
+                    .collect(Collectors.toList());
+            Optional<BrokerLocalStorage> brokerLocalStorageOpt = localStorages
+                    .stream()
+                    // Select brokers which are assigned a replica of the 
topic-partition
+                    .filter(s -> isAssignedReplica(topicPartition, 
s.getBrokerId()))
+                    // Filter out inactive brokers, which may still contain 
log segments we would expect
+                    // to be deleted based on the retention configuration.
+                    .filter(s -> isAlive(s.getBrokerId()))
+                    .findFirst();
+            brokerLocalStorageOpt.ifPresent(localStorage ->
+                    // Wait until the brokers local storage have been cleared 
from the inactive log segments.
+                    
localStorage.waitForAtLeastEarliestLocalOffset(topicPartition, 1L));
+        });
+    }

Review Comment:
   The reason for using the prefix `maybe` is that in `TransactionsTest` 
without tiered storage, we won't be uploading any segments.



-- 
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