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



##########
File path: core/src/main/scala/kafka/server/AlterIsrManager.scala
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.atomic.{AtomicBoolean, AtomicLong}
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
+
+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, 
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 start(): Unit
+
+  def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean
+
+  def clearPending(topicPartition: TopicPartition): Unit
+}
+
+case class AlterIsrItem(topicPartition: TopicPartition, leaderAndIsr: 
LeaderAndIsr, callback: Either[Errors, LeaderAndIsr] => Unit)
+
+class AlterIsrManagerImpl(val controllerChannelManager: 
BrokerToControllerChannelManager,
+                          val zkClient: KafkaZkClient,
+                          val scheduler: Scheduler,
+                          val time: Time,
+                          val brokerId: Int,
+                          val brokerEpochSupplier: () => Long) extends 
AlterIsrManager with Logging with KafkaMetricsGroup {
+
+  // Used to allow only one pending ISR update per partition
+  private val unsentIsrUpdates: util.Map[TopicPartition, AlterIsrItem] = new 
ConcurrentHashMap[TopicPartition, AlterIsrItem]()
+
+  // Used to allow only one in-flight request at a time
+  private val inflightRequest: AtomicBoolean = new AtomicBoolean(false)
+
+  private val lastIsrPropagationMs = new AtomicLong(0)
+
+  override def start(): Unit = {
+    scheduler.schedule("send-alter-isr", propagateIsrChanges, 50, 50, 
TimeUnit.MILLISECONDS)

Review comment:
       Currently we impose a 2.5s delay for the old ZK based ISR propagation 
method. We could probably increase this 50ms up to a few hundred without any 
ill-effects. We still benefit from fact that we assume the maximal ISR 
immediately. How about 200ms?
   
   Longer term we can look into having a single thread invocation that sits in 
a while loop trying to consume from a LinkedBlockingQueue or maybe even a 
SynchronousQueue. But agreed we should leave this for 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.

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


Reply via email to