kowshik commented on code in PR #12005:
URL: https://github.com/apache/kafka/pull/12005#discussion_r848807096


##########
core/src/main/scala/kafka/server/LeaderEndPoint.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.api.{KAFKA_0_10_1_IV2, KAFKA_0_11_0_IV0, KAFKA_2_0_IV0, 
KAFKA_2_0_IV1, KAFKA_2_1_IV1, KAFKA_2_2_IV1, KAFKA_2_3_IV1, KAFKA_2_8_IV0, 
KAFKA_3_0_IV1}
+import kafka.utils.Logging
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.requests.FetchRequest
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.message.{FetchResponseData, 
OffsetForLeaderEpochRequestData}
+
+import scala.collection.Map
+
+trait LeaderEndPoint extends Logging {

Review Comment:
   Lets remove the inheritance from `Logging` here. This needs to go to the 
sub-class.



##########
core/src/main/scala/kafka/server/LeaderEndPoint.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.api.{KAFKA_0_10_1_IV2, KAFKA_0_11_0_IV0, KAFKA_2_0_IV0, 
KAFKA_2_0_IV1, KAFKA_2_1_IV1, KAFKA_2_2_IV1, KAFKA_2_3_IV1, KAFKA_2_8_IV0, 
KAFKA_3_0_IV1}
+import kafka.utils.Logging
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.requests.FetchRequest
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.message.{FetchResponseData, 
OffsetForLeaderEpochRequestData}
+
+import scala.collection.Map
+
+trait LeaderEndPoint extends Logging {
+  val brokerConfig: KafkaConfig
+  val endpoint: BlockingSend = null
+
+  type FetchData = FetchResponseData.PartitionData
+  type EpochData = OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
+
+  // Visible for testing
+  private[server] val listOffsetRequestVersion: Short =

Review Comment:
   Both `listOffsetRequestVersion` and `offsetForLeaderEpochRequestVersion` 
should go into the implementation of the trait (i.e. sub-class). It is best to 
remove it from here.



##########
core/src/main/scala/kafka/server/ReplicaFetcherThread.scala:
##########
@@ -135,10 +97,10 @@ class ReplicaFetcherThread(name: String,
       // to avoid failing the caller, especially during shutdown. We will 
attempt to close
       // leaderEndpoint after the thread terminates.
       try {
-        leaderEndpoint.initiateClose()
+        leader.initiateClose()
       } catch {
         case t: Throwable =>
-          error(s"Failed to initiate shutdown of leader endpoint 
$leaderEndpoint after initiating replica fetcher thread shutdown", t)
+          error(s"Failed to initiate shutdown of ReplicaFetcherBlockingSend 
after initiating replica fetcher thread shutdown", t)

Review Comment:
   Could the log message print the leader endpoint instead (just like earlier 
from LHS)? You could always override the `toString()` inside the `leader` 
object to be able to print a human readable version of the object here.



##########
core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala:
##########
@@ -69,6 +69,26 @@ class ReplicaAlterLogDirsThreadTest {
       initOffset = fetchOffset, currentLeaderEpoch = leaderEpoch)
   }
 
+  private def createReplicaAlterLogDirsThread(name: String,

Review Comment:
   This helper method could be eliminated, please refer to my comment below.



##########
core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala:
##########
@@ -147,7 +167,7 @@ class ReplicaAlterLogDirsThreadTest {
     mockFetchFromCurrentLog(tid1p0, fencedRequestData, config, replicaManager, 
fencedResponseData)
 
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
-    val thread = new ReplicaAlterLogDirsThread(
+    val thread = createReplicaAlterLogDirsThread(

Review Comment:
   Why not just include the new `LeaderEndpoint` parameter directly here, 
instead of introducing a helper?
   i.e.
   
   ```
   val thread = new ReplicaAlterLogDirsThread(
         "alter-logs-dirs-thread",
         new LocalLeaderEndPoint(replicaManager),
         sourceBroker = endPoint,
         brokerConfig = config,
         failedPartitions = failedPartitions,
         replicaMgr = replicaManager,
         quota = quotaManager,
         brokerTopicStats = new BrokerTopicStats)
   ```



##########
core/src/main/scala/kafka/server/LeaderEndPoint.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.api.{KAFKA_0_10_1_IV2, KAFKA_0_11_0_IV0, KAFKA_2_0_IV0, 
KAFKA_2_0_IV1, KAFKA_2_1_IV1, KAFKA_2_2_IV1, KAFKA_2_3_IV1, KAFKA_2_8_IV0, 
KAFKA_3_0_IV1}
+import kafka.utils.Logging
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.requests.FetchRequest
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.message.{FetchResponseData, 
OffsetForLeaderEpochRequestData}
+
+import scala.collection.Map
+
+trait LeaderEndPoint extends Logging {
+  val brokerConfig: KafkaConfig
+  val endpoint: BlockingSend = null
+
+  type FetchData = FetchResponseData.PartitionData
+  type EpochData = OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
+
+  // Visible for testing
+  private[server] val listOffsetRequestVersion: Short =
+    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_3_0_IV1) 7
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_8_IV0) 6
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_2_IV1) 5
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 4
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 3
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 2
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) 1
+    else 0
+
+  // Visible for testing
+  private[server] val offsetForLeaderEpochRequestVersion: Short =
+    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_8_IV0) 4
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_3_IV1) 3
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 2
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV0) 1
+    else 0
+
+  def initiateClose(): Unit = {}

Review Comment:
   All the public methods in the trait need to be documented.



##########
core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.api.Request
+import kafka.server.QuotaFactory.UnboundedQuota
+import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
+import org.apache.kafka.common.message.FetchResponseData
+import 
org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH
+import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, 
RequestUtils}
+
+import scala.collection.{Map, Seq, mutable}
+import scala.jdk.CollectionConverters._
+
+class LocalLeaderEndPoint(val replicaMgr: ReplicaManager) extends 
LeaderEndPoint {

Review Comment:
   Could we document this class?



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