soarez commented on code in PR #15690:
URL: https://github.com/apache/kafka/pull/15690#discussion_r1560139446


##########
core/src/main/java/kafka/server/TierStateMachine.java:
##########
@@ -40,7 +92,72 @@ public interface TierStateMachine {
      */
     PartitionFetchState start(TopicPartition topicPartition,
                               PartitionFetchState currentFetchState,
-                              PartitionData fetchPartitionData) throws 
Exception;
+                              PartitionData fetchPartitionData) throws 
Exception {
+        OffsetAndEpoch epochAndLeaderLocalStartOffset = 
leader.fetchEarliestLocalOffset(topicPartition, 
currentFetchState.currentLeaderEpoch());
+        int epoch = epochAndLeaderLocalStartOffset.leaderEpoch();
+        long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset();
+
+        long offsetToFetch;
+        
replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).buildRemoteLogAuxStateRequestRate().mark();
+        
replicaMgr.brokerTopicStats().allTopicsStats().buildRemoteLogAuxStateRequestRate().mark();
+
+        try {
+            offsetToFetch = buildRemoteLogAuxState(topicPartition, 
currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, 
fetchPartitionData.logStartOffset());
+        } catch (RemoteStorageException e) {
+            
replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).failedBuildRemoteLogAuxStateRate().mark();
+            
replicaMgr.brokerTopicStats().allTopicsStats().failedBuildRemoteLogAuxStateRate().mark();
+            throw e;
+        }
+
+        OffsetAndEpoch fetchLatestOffsetResult = 
leader.fetchLatestOffset(topicPartition, 
currentFetchState.currentLeaderEpoch());
+        long leaderEndOffset = fetchLatestOffsetResult.offset();
+
+        long initialLag = leaderEndOffset - offsetToFetch;
+
+        return PartitionFetchState.apply(currentFetchState.topicId(), 
offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(),
+                Fetching$.MODULE$, 
replicaMgr.localLogOrException(topicPartition).latestEpoch());

Review Comment:
   Should `replicaMgr.futureLogOrException` be used instead, if `useFutureLog`?



##########
storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java:
##########
@@ -175,8 +186,38 @@ private OffsetHolder getEarliestLocalOffset(TopicPartition 
topicPartition) {
         return new 
OffsetHolder(LogFileUtils.offsetFromFileName(firstLogFile.get()), 
partitionFiles);
     }
 
-    private List<String> getTopicPartitionFiles(TopicPartition topicPartition) 
{
-        File[] files = brokerStorageDirectory.listFiles((dir, name) -> 
name.equals(topicPartition.toString()));
+    public boolean isTopicPartitionFileExistInDir(TopicPartition 
topicPartition, File logDir) {

Review Comment:
   Maybe `dirContainsTopicPartition` is a better name for this method?



##########
storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java:
##########
@@ -31,31 +31,36 @@
 import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
 public final class BrokerLocalStorage {
 
     private final Integer brokerId;
-    private final File brokerStorageDirectory;
+    private final Set<File> brokerStorageDirectory;
     private final Integer storageWaitTimeoutSec;
 
     private final int storagePollPeriodSec = 1;
     private final Time time = Time.SYSTEM;
 
     public BrokerLocalStorage(Integer brokerId,
-                              String storageDirname,
+                              Set<String> storageDirname,
                               Integer storageWaitTimeoutSec) {
         this.brokerId = brokerId;
-        this.brokerStorageDirectory = new File(storageDirname);
+        this.brokerStorageDirectory = 
storageDirname.stream().map(File::new).collect(Collectors.toSet());

Review Comment:
   The names for parameter `storageDirname` and field `brokerStorageDirectory` 
should be pluralized.



##########
core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala:
##########
@@ -23,20 +23,23 @@ import org.apache.kafka.common.protocol.ApiKeys
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.{TopicPartition, Uuid}
 import org.junit.jupiter.api.Assertions._
-import org.junit.jupiter.api.Test
 import kafka.server.FetcherThreadTestUtils.{initialFetchState, mkBatch}
+import org.junit.jupiter.api.extension.ExtensionContext
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{Arguments, ArgumentsProvider, 
ArgumentsSource}
 
 import scala.collection.Map
 
-class ReplicaFetcherTierStateMachineTest {
+class TierStateMachineTest {
 
-  val truncateOnFetch = true
+  val truncateOnFetch = false

Review Comment:
   This can be removed



##########
core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala:
##########
@@ -153,12 +157,13 @@ class ReplicaFetcherTierStateMachineTest {
     assertEquals(11L, replicaState.logEndOffset)
   }
 
-  @Test
-  def testFencedOffsetResetAfterMovedToRemoteTier(): Unit = {
+  @ParameterizedTest
+  @ArgumentsSource(classOf[TierStateMachineTest.Params])
+  def testFencedOffsetResetAfterMovedToRemoteTier(truncateOnFetch: Boolean, 
useFutureLog: Boolean): Unit = {
     val partition = new TopicPartition("topic", 0)
     var isErrorHandled = false
     val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = 
truncateOnFetch, version = version)
-    val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) {
+    val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint, 
useFutureLog) {

Review Comment:
   Given:
   
   a. `MockTierStateMachine` only refers to `useFutureLog` to carry it over to 
the `TierStateMachine`;
   b. `TierStateMachine ` only refers to `useFutureLog ` in 
`buildRemoteLogAuxState`;
   c. `buildRemoteLogAuxState` is only invoked from `TierStateMachine#start`;
   d. `MockTierStateMachine` overrides `start` and does not call 
`super.start()`;
   
   It seems having `useFutureLog` as a test param is unnecessary?



##########
storage/src/test/java/org/apache/kafka/tiered/storage/actions/AlterLogDirAction.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.actions;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tiered.storage.TieredStorageTestAction;
+import org.apache.kafka.tiered.storage.TieredStorageTestContext;
+import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage;
+
+import java.io.File;
+import java.io.PrintStream;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+public final class AlterLogDirAction implements TieredStorageTestAction {
+
+    private final TopicPartition topicPartition;
+    private final int brokerId;
+
+    public AlterLogDirAction(TopicPartition topicPartition,
+                             int brokerId) {
+        this.topicPartition = topicPartition;
+        this.brokerId = brokerId;
+    }
+
+    @Override
+    public void doExecute(TieredStorageTestContext context) throws 
InterruptedException, ExecutionException {
+        Optional<BrokerLocalStorage> localStorage = 
context.localStorages().stream().filter(storage -> 
storage.getBrokerId().intValue() == brokerId).findFirst();
+        if (!localStorage.isPresent()) {
+            throw new IllegalArgumentException("cannot find local storage for 
this topic partition:" + topicPartition + " in this broker id:" + brokerId);
+        }
+
+        Optional<File> sourceDir = 
localStorage.get().getBrokerStorageDirectory().stream().filter(dir -> 
localStorage.get().isTopicPartitionFileExistInDir(topicPartition, 
dir)).findFirst();
+        Optional<File> targetDir = 
localStorage.get().getBrokerStorageDirectory().stream().filter(dir -> 
!localStorage.get().isTopicPartitionFileExistInDir(topicPartition, 
dir)).findFirst();
+        if (!sourceDir.isPresent()) {
+            throw new IllegalArgumentException("No log dir with topic 
partition:" + topicPartition + " in this broker id:" + brokerId);
+        }
+
+        if (!targetDir.isPresent()) {
+            throw new IllegalArgumentException("No log dir without topic 
partition:" + topicPartition + " in this broker id:" + brokerId);
+        }
+
+        // build alterReplicaLogDirs request content to move from sourceDir to 
targetDir
+        Map<TopicPartitionReplica, String> logDirs = 
Collections.singletonMap(new TopicPartitionReplica(topicPartition.topic(), 
topicPartition.partition(), brokerId), targetDir.get().getAbsolutePath());
+
+        context.admin().alterReplicaLogDirs(logDirs);

Review Comment:
   We could check the request error here. If there's an issue, the test will 
fail faster (not having to wait `DEFAULT_MAX_WAIT_MS` in 
`TestUtils.waitForCondition`) and the error will be easier to spot.



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