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


##########
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()) {

Review Comment:
   Should we move this before the computation of `targetDir`?



##########
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);
+
+        // wait until the topic partition folder disappearing from source dir 
and appearing in the target dir
+        TestUtils.waitForCondition(() -> 
localStorage.get().isTopicPartitionFileExistInDir(topicPartition, 
targetDir.get()) &&
+                    
!localStorage.get().isTopicPartitionFileExistInDir(topicPartition, 
sourceDir.get()),
+                "Failed to alter dir:" + logDirs);
+    }
+
+    @Override
+    public void describe(PrintStream output) {
+        output.print("alter di for topic partition:" + topicPartition + " in 
this broker id:" + brokerId);

Review Comment:
   alter dir?



##########
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());
+
+    }
+
+    private OffsetForLeaderEpochResponseData.EpochEndOffset 
fetchEarlierEpochEndOffset(Integer epoch,
+                                                                               
        TopicPartition partition,
+                                                                               
        Integer currentLeaderEpoch) {
+        int previousEpoch = epoch - 1;
+
+        // Find the end-offset for the epoch earlier to the given epoch from 
the leader
+        Map<TopicPartition, 
OffsetForLeaderEpochRequestData.OffsetForLeaderPartition> partitionsWithEpochs 
= new HashMap<>();
+        partitionsWithEpochs.put(partition, new 
OffsetForLeaderEpochRequestData.OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch));
+        Option<OffsetForLeaderEpochResponseData.EpochEndOffset> 
maybeEpochEndOffset = 
leader.fetchEpochEndOffsets(JavaConverters.mapAsScalaMap(partitionsWithEpochs)).get(partition);
+        if (maybeEpochEndOffset.isEmpty()) {
+            throw new KafkaException("No response received for partition: " + 
partition);
+        }
+
+        OffsetForLeaderEpochResponseData.EpochEndOffset epochEndOffset = 
maybeEpochEndOffset.get();
+        if (epochEndOffset.errorCode() != Errors.NONE.code()) {
+            throw Errors.forCode(epochEndOffset.errorCode()).exception();
+        }
+
+        return epochEndOffset;
+    }
+
+    private List<EpochEntry> readLeaderEpochCheckpoint(RemoteLogManager rlm,
+                                                       
RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, 
RemoteStorageException {
+        InputStream inputStream = 
rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, 
RemoteStorageManager.IndexType.LEADER_EPOCH);
+        try (BufferedReader bufferedReader = new BufferedReader(new 
InputStreamReader(inputStream, StandardCharsets.UTF_8))) {
+            CheckpointFile.CheckpointReadBuffer<EpochEntry> readBuffer = new 
CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, 
LeaderEpochCheckpointFile.FORMATTER);
+            return readBuffer.read();
+        }
+    }
+
+    private void buildProducerSnapshotFile(File snapshotFile,
+                                           RemoteLogSegmentMetadata 
remoteLogSegmentMetadata,
+                                           RemoteLogManager rlm) throws 
IOException, RemoteStorageException {
+        File tmpSnapshotFile = new File(snapshotFile.getAbsolutePath() + 
".tmp");

Review Comment:
   Could we directly build a `Path` object with `Path tmpSnapshotFile = 
Paths.get(snapshotFile.getAbsolutePath() + ".tmp");`?
   We seem to always use `toPath()` on `tmpSnapshotFile`.



##########
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);
+
+        // wait until the topic partition folder disappearing from source dir 
and appearing in the target dir

Review Comment:
   `disappearing` -> `disappears` and `appearing` -> `appears`



##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.tiered.storage.TieredStorageTestBuilder;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+
+public final class AlterLogDirTest extends BaseReassignReplicaTest {
+
+    /**
+     * Alter dir within broker0
+     * @return the replica-ids of the topic
+     */
+    @Override
+    protected List<Integer> replicaIds() {
+        return Collections.singletonList(broker0);
+    }
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final String topicB = "topicB";
+        final Integer p0 = 0;

Review Comment:
   All of these can be `int`.



##########
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();

Review Comment:
   I think we don't need `.intValue()`.



##########
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) {
+        File[] files = getTopicPartitionFiles(topicPartition, 
Collections.singleton(logDir));
+        return files != null && files.length > 0;
+    }
+
+    public Optional<File> getLogDirNotStoringTopicPartition(TopicPartition 
topicPartition) {

Review Comment:
   This method is unused, do we want to keep it?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1747,7 +1751,7 @@ class ReplicaManager(val config: KafkaConfig,
       val leaderLogStartOffset = log.logStartOffset
       val leaderLogEndOffset = log.logEndOffset
 
-      if (params.isFromFollower) {
+      if (params.isFromFollower || params.isFromFuture) {

Review Comment:
   Do we need to adjust the comment below too?



##########
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:
   +1
   Same for the getter method below



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