abbccdda commented on a change in pull request #9100:
URL: https://github.com/apache/kafka/pull/9100#discussion_r473515635



##########
File path: core/src/main/scala/kafka/server/AlterIsrManager.scala
##########
@@ -0,0 +1,149 @@
+/**
+ * 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 java.util
+import java.util.concurrent.{ScheduledFuture, TimeUnit}
+import java.util.concurrent.atomic.AtomicLong
+
+import kafka.api.LeaderAndIsr
+import kafka.metrics.KafkaMetricsGroup
+import kafka.utils.{Logging, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.TopicPartition
+import 
org.apache.kafka.common.message.AlterIsrRequestData.{AlterIsrRequestPartitions, 
AlterIsrRequestTopics}
+import org.apache.kafka.common.message.{AlterIsrRequestData, 
AlterIsrResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterIsrRequest, AlterIsrResponse}
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.mutable
+import scala.jdk.CollectionConverters._
+
+/**
+ * Handles the sending of AlterIsr requests to the controller. Updating the 
ISR is an asynchronous operation,
+ * so partitions will learn about updates through LeaderAndIsr messages sent 
from the controller
+ */
+trait AlterIsrManager {
+  def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Unit
+
+  def clearPending(topicPartition: TopicPartition): Unit
+
+  def startup(): Unit
+
+  def shutdown(): Unit
+}
+
+case class AlterIsrItem(topicPartition: TopicPartition, leaderAndIsr: 
LeaderAndIsr, callback: Errors => Unit)
+
+class AlterIsrManagerImpl(val controllerChannelManager: 
BrokerToControllerChannelManager,
+                          val zkClient: KafkaZkClient,
+                          val scheduler: Scheduler,
+                          val time: Time,
+                          val brokerId: Int) extends AlterIsrManager with 
Logging with KafkaMetricsGroup {
+
+  private val unsentIsrUpdates: mutable.Map[TopicPartition, AlterIsrItem] = 
new mutable.HashMap[TopicPartition, AlterIsrItem]()
+  private val lastIsrChangeMs = new AtomicLong(0)
+  private val lastIsrPropagationMs = new AtomicLong(0)
+
+  @volatile private var scheduledRequest: Option[ScheduledFuture[_]] = None
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Unit = {
+    unsentIsrUpdates synchronized {
+      unsentIsrUpdates(alterIsrItem.topicPartition) = alterIsrItem
+      lastIsrChangeMs.set(time.milliseconds)
+      // Rather than sending right away, we'll delay at most 50ms to allow for 
batching of ISR changes happening
+      // in fast succession
+      if (scheduledRequest.isEmpty) {
+        scheduledRequest = Some(scheduler.schedule("propagate-alter-isr", 
propagateIsrChanges, 50, -1, TimeUnit.MILLISECONDS))
+      }
+    }
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates synchronized {
+      // when we get a new LeaderAndIsr, we clear out any pending requests
+      unsentIsrUpdates.remove(topicPartition)
+    }
+  }
+
+  override def startup(): Unit = {
+    controllerChannelManager.start()

Review comment:
       Should we move the startup logic to `KafkaServer`? Note the channel is 
shared between different modules, so it makes sense to start and close inside 
the server.

##########
File path: core/src/main/scala/kafka/server/AlterIsrChannelManager.scala
##########
@@ -0,0 +1,132 @@
+package kafka.server
+
+import java.util
+import java.util.concurrent.{ScheduledFuture, TimeUnit}
+import java.util.concurrent.atomic.AtomicLong
+
+import kafka.api.LeaderAndIsr
+import kafka.metrics.KafkaMetricsGroup
+import kafka.utils.{Logging, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.TopicPartition
+import 
org.apache.kafka.common.message.AlterIsrRequestData.{AlterIsrRequestPartitions, 
AlterIsrRequestTopics}
+import org.apache.kafka.common.message.{AlterIsrRequestData, 
AlterIsrResponseData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterIsrRequest, AlterIsrResponse}
+
+import scala.collection.mutable
+import scala.jdk.CollectionConverters._
+
+/**
+ * Handles the sending of AlterIsr requests to the controller. Updating the 
ISR is an asynchronous operation,
+ * so partitions will learn about updates through LeaderAndIsr messages sent 
from the controller
+ */
+trait AlterIsrChannelManager {
+  val IsrChangePropagationBlackOut = 5000L
+  val IsrChangePropagationInterval = 60000L
+
+  def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Unit
+
+  def clearPending(topicPartition: TopicPartition): Unit
+
+  def startup(): Unit
+
+  def shutdown(): Unit
+}
+
+case class AlterIsrItem(topicPartition: TopicPartition, leaderAndIsr: 
LeaderAndIsr)
+
+class AlterIsrChannelManagerImpl(val controllerChannelManager: 
BrokerToControllerChannelManager,
+                                 val zkClient: KafkaZkClient,
+                                 val scheduler: Scheduler,
+                                 val brokerId: Int,
+                                 val brokerEpoch: Long) extends 
AlterIsrChannelManager with Logging with KafkaMetricsGroup {
+
+  private val pendingIsrUpdates: mutable.Map[TopicPartition, AlterIsrItem] = 
new mutable.HashMap[TopicPartition, AlterIsrItem]()
+  private val lastIsrChangeMs = new AtomicLong(0)
+  private val lastIsrPropagationMs = new AtomicLong(0)
+
+  @volatile private var scheduledRequest: Option[ScheduledFuture[_]] = None
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Unit = {
+    pendingIsrUpdates synchronized {
+      pendingIsrUpdates(alterIsrItem.topicPartition) = alterIsrItem
+      lastIsrChangeMs.set(System.currentTimeMillis())
+      // Rather than sending right away, we'll delay at most 50ms to allow for 
batching of ISR changes happening
+      // in fast succession
+      if (scheduledRequest.isEmpty) {
+        scheduledRequest = Some(scheduler.schedule("propagate-alter-isr", 
propagateIsrChanges, 50, -1, TimeUnit.MILLISECONDS))
+      }
+    }
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    pendingIsrUpdates synchronized {
+      // when we get a new LeaderAndIsr, we clear out any pending requests
+      pendingIsrUpdates.remove(topicPartition)
+    }
+  }
+
+  override def startup(): Unit = {
+    controllerChannelManager.start()
+  }
+
+  override def shutdown(): Unit = {
+    controllerChannelManager.shutdown()
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    val now = System.currentTimeMillis()
+    pendingIsrUpdates synchronized {
+      if (pendingIsrUpdates.nonEmpty) {
+        // Max ISRs to send?
+        val message = new AlterIsrRequestData()
+          .setBrokerId(brokerId)
+          .setBrokerEpoch(brokerEpoch)
+          .setTopics(new util.ArrayList())
+
+        
pendingIsrUpdates.values.groupBy(_.topicPartition.topic()).foreachEntry((topic, 
items) => {
+          val topicPart = new AlterIsrRequestTopics()

Review comment:
       nit: could move these operations to the `AlterIsrRequest` as helpers.

##########
File path: clients/src/main/resources/common/message/AlterIsrRequest.json
##########
@@ -0,0 +1,44 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "AlterIsrRequest",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": 
"brokerId",
+      "about": "The ID of the requesting broker" },
+    { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": 
"-1",
+      "about": "The epoch of the requesting broker" },
+    { "name": "Topics", "type": "[]AlterIsrRequestTopics", "versions": "0+", 
"fields": [
+      { "name":  "Name", "type": "string", "versions": "0+", "entityType": 
"topicName",
+        "about": "The name of the topic to alter ISRs for" },
+      { "name": "Partitions", "type": "[]AlterIsrRequestPartitions", 
"versions": "0+", "fields": [
+        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
+          "about": "The partition index" },
+        { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": 
"brokerId",
+          "about": "The ID of the leader broker" },
+        { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
+          "about": "The leader epoch of this partition" },
+        { "name": "NewIsr", "type": "[]int32", "versions": "0+",
+          "about": "The ISR for this partition"},
+        { "name": "CurrentIsrVersion", "type": "int32", "versions": "0+",
+          "about": "The expected version of ISR which is being updated"}
+      ]}
+    ]}
+  ]
+}

Review comment:
       nit: new line




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to