satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1059039639


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -995,11 +994,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
-  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch)
+  def latestEpoch: Option[Int] = 
leaderEpochCache.flatMap(_.latestEpoch.asScala).map(Int.unbox(_))

Review Comment:
   Avoided using Java Optional classes here as all its usages are in Scala 
classes for now. I prefer making this kind of change as part of moving 
UnifiedLog to storage module. wdyt? 
   



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.server.log.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   +1 on that. I prefer have the package name with `server` but I am fine to 
omit that to be consistent with the existing convention with other modules. 
   I will address it in the next commit of this PR. Let me know if you are fine 
with the package name as `org.apache.kafka.storage.internals.checkpoint`. 



##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -2650,7 +2647,7 @@ class PartitionTest extends AbstractPartitionTest {
     assertEquals(Some(0L), partition.leaderEpochStartOffsetOpt)
 
     val leaderLog = partition.localLogOrException
-    assertEquals(Some(EpochEntry(leaderEpoch, 0L)), 
leaderLog.leaderEpochCache.flatMap(_.latestEntry))
+    assertEquals(Some(new EpochEntry(leaderEpoch, 0L)), 
leaderLog.leaderEpochCache.flatMap(_.latestEntry.asScala))

Review Comment:
   `flatMap` has to return `Option`, that is why I converted from `Optional` to 
`Option`. Another way can be to use match case statements that makes it 
verbose. I left with Scala Option conversions for now as these will be 
converted to Java later.



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