[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

2023-02-23 Thread via GitHub


junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1116408589


##
core/src/main/java/kafka/server/TierStateMachine.java:
##
@@ -0,0 +1,58 @@
+/*
+ * 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 kafka.server;

Review Comment:
   Should this be in the storage module under 
org.apache.kafka.server.log.remote.storage package? Eventually, we could move 
ReplicaFetcherTierStateMachine to the storage module too.



##
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##
@@ -615,14 +649,29 @@ class AbstractFetcherThreadTest {
   @Test
   def testFollowerFetchMovedToTieredStore(): Unit = {
 val partition = new TopicPartition("topic", 0)
-val fetcher = new MockFetcherThread(new MockLeaderEndPoint)
 
 val replicaLog = Seq(
   mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
   mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
   mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
 val replicaState = PartitionState(replicaLog, leaderEpoch = 5, 
highWatermark = 0L, rlmEnabled = true)
+
+val mockLeaderEndpoint = new MockLeaderEndPoint
+val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) {
+  // override the start() of MockTierStateMachine to mimic 
truncateFullyAndStartAt and update the replicaState in the MockFetcherThread
+  override def start(topicPartition: TopicPartition,
+ currentFetchState: PartitionFetchState,
+ fetchPartitionData: FetchResponseData.PartitionData): 
PartitionFetchState = {
+replicaState.log.clear()
+replicaState.localLogStartOffset = 5
+replicaState.logEndOffset = 5

Review Comment:
   Got it. So, if we do that, it's the same as the callback logic you had 
earlier.
   
   Here is another way to improve this. Since AbstractFetcherThread already 
exposes the `partitionState` through `fetchState()`, we could just get rid of 
`replicaPartitionStates` in MockFetcherThread and purely rely on the 
`partitionState` that's changed on every `doWork()` call. Will that work? Since 
that's a bigger change, we could probably just take the callback code you had 
in this PR and make the bigger change in a separate PR.



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



[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

2023-02-23 Thread via GitHub


junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1116386328


##
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##
@@ -615,14 +649,29 @@ class AbstractFetcherThreadTest {
   @Test
   def testFollowerFetchMovedToTieredStore(): Unit = {
 val partition = new TopicPartition("topic", 0)
-val fetcher = new MockFetcherThread(new MockLeaderEndPoint)
 
 val replicaLog = Seq(
   mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
   mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
   mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
 val replicaState = PartitionState(replicaLog, leaderEpoch = 5, 
highWatermark = 0L, rlmEnabled = true)
+
+val mockLeaderEndpoint = new MockLeaderEndPoint
+val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) {
+  // override the start() of MockTierStateMachine to mimic 
truncateFullyAndStartAt and update the replicaState in the MockFetcherThread
+  override def start(topicPartition: TopicPartition,
+ currentFetchState: PartitionFetchState,
+ fetchPartitionData: FetchResponseData.PartitionData): 
PartitionFetchState = {
+replicaState.log.clear()
+replicaState.localLogStartOffset = 5
+replicaState.logEndOffset = 5

Review Comment:
   Instead of the customizing for this test, I am wondering if we could make 
this more general. For example, could we pass in `fetcher` to 
MockTierStateMachine and call `fetcher.truncateFullyAndStartAt() `in 
`MockTierStateMachine.start()`.



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



[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

2023-02-23 Thread via GitHub


junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1114995703


##
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##
@@ -27,9 +27,11 @@ import kafka.utils.{DelayedItem, Logging, Pool}
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.internals.PartitionStates
 import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.message.FetchResponseData.PartitionData
 import org.apache.kafka.common.message.{FetchResponseData, 
OffsetForLeaderEpochRequestData}
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Records}
+//import org.apache.kafka.common.requests.FetchRequest.PartitionData

Review Comment:
   Should we remove this import?



##
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##
@@ -633,13 +669,18 @@ class AbstractFetcherThreadTest {
   mkBatch(baseOffset = 7, leaderEpoch = 5, new SimpleRecord("h".getBytes)),
   mkBatch(baseOffset = 8, leaderEpoch = 5, new SimpleRecord("i".getBytes)))
 
-
 val leaderState = PartitionState(leaderLog, leaderEpoch = 5, highWatermark 
= 8L, rlmEnabled = true)
 // Overriding the log start offset to zero for mocking the scenario of 
segment 0-4 moved to remote store.
 leaderState.logStartOffset = 0
 fetcher.mockLeader.setLeaderState(partition, leaderState)
 
fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState)
 
+def buildRemoteLog(topicPartition: TopicPartition, leaderLogStartOffset: 
Long): Unit = {
+  fetcher.truncateFullyAndStartAt(topicPartition, 
leaderState.localLogStartOffset)
+  replicaState.logStartOffset = leaderLogStartOffset

Review Comment:
   Well, `buildRemoteLog()` does two things (1) call 
`fetcher.truncateFullyAndStartAt` and (2) set `replicaState.logStartOffset`. 
For (1), since the truncation logic is moved to TierStateMachine, it probably 
should be done in `MockTierStateMachine.start()` directly. For (2), the 
existing test doesn't need to set `replicaState.logStartOffset.` So, it seems 
it's unnecessary? If address both (1) and (2), then the callback is not needed.



##
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##
@@ -424,6 +406,13 @@ abstract class AbstractFetcherThread(name: String,
   if (onPartitionFenced(topicPartition, 
fetchPartitionData.currentLeaderEpoch))
 partitionsWithError += topicPartition
 
+case Errors.OFFSET_MOVED_TO_TIERED_STORAGE =>
+  debug(s"Received error 
${Errors.OFFSET_MOVED_TO_TIERED_STORAGE}, " +
+s"at fetch offset: ${currentFetchState.fetchOffset}, " + 
s"topic-partition: $topicPartition")
+  if (!handleOffsetsMovedToTieredStorage(topicPartition, 
currentFetchState, fetchPartitionData.currentLeaderEpoch, partitionData)) {
+partitionsWithError += topicPartition
+  }

Review Comment:
   No need for brackets for single line statements.



##
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##
@@ -794,17 +742,18 @@ abstract class AbstractFetcherThread(name: String,
*
* @param topicPartition topic partition
* @param fetchState current partition fetch state.
-   * @param leaderEpochInRequest current leader epoch sent in the fetch 
request.
-   * @param leaderLogStartOffset log-start-offset in the leader replica.
+   * @param fetchPartitionData the fetch request data for this topic partition

Review Comment:
   This is the fetch response data. Also, could we add the missing param?



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



[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

2023-02-09 Thread via GitHub


junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1101875350


##
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##
@@ -0,0 +1,265 @@
+/*
+ * 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 kafka.server;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution, and we 
only need to start the machine.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of 
the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+private static final Logger log = 
LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);

Review Comment:
   Thanks, Satish. Agreed. Since this one will be implemented as async 
eventually, there is probably no need to set LogContext. We can keep this as 
static. 



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



[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

2023-02-08 Thread via GitHub


junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1100515746


##
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##
@@ -0,0 +1,265 @@
+/*
+ * 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 kafka.server;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution, and we 
only need to start the machine.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of 
the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+private static final Logger log = 
LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);

Review Comment:
   This seems a bit weird. The log typically is an instance level object.



##
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##
@@ -0,0 +1,265 @@
+/*
+ * 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 kafka.server;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader