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



##########
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": 52,
+  "type": "request",
+  "name": "AlterIsrRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "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": "[]TopicData", "versions": "0+", "fields": [
+      { "name":  "Name", "type": "string", "versions": "0+", "entityType": 
"topicName",
+        "about": "The name of the topic to alter ISRs for" },
+      { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", 
"fields": [
+        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
+          "about": "The partition index" },
+        { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": 
"brokerId",

Review comment:
       I know we've gone back and forth on including some of these fields. This 
is one I'm inclined to get rid of since we already include "BrokerId" at the 
top level and `AlterIsr` can only be sent by leaders.

##########
File path: clients/src/main/resources/common/message/AlterIsrResponse.json
##########
@@ -0,0 +1,46 @@
+// 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": 52,
+  "type": "response",
+  "name": "AlterIsrResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "about": "The duration in milliseconds for which the request was 
throttled due to a quota violation, or zero if the request did not violate any 
quota." },
+    { "name": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top level response error code" },
+    { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [
+      { "name":  "Name", "type": "string", "versions": "0+", "entityType": 
"topicName",
+        "about": "The name of the topic" },
+      { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", 
"fields": [
+        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
+          "about": "The partition index" },
+        { "name": "ErrorCode", "type": "int16", "versions": "0+",
+          "about": "The partition level error code" },
+        { "name": "Leader", "type": "int32", "versions": "0+", "entityType": 
"brokerId",

Review comment:
       nit: shall we call this `LeaderId` in line with `BrokerId` in the 
request?

##########
File path: config/log4j.properties
##########
@@ -76,8 +76,8 @@ log4j.additivity.kafka.request.logger=false
 log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender
 log4j.additivity.kafka.network.RequestChannel$=false
 
-log4j.logger.kafka.controller=TRACE, controllerAppender
-log4j.additivity.kafka.controller=false
+log4j.logger.kafka.controller=DEBUG, controllerAppender

Review comment:
       Can we revert this change? I think the trace logging is intended, if a 
bit odd.

##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -100,7 +100,9 @@ object ApiVersion {
     // Introduced StopReplicaRequest V3 containing the leader epoch for each 
partition (KIP-570)
     KAFKA_2_6_IV0,
     // Introduced feature versioning support (KIP-584)
-    KAFKA_2_7_IV0
+    KAFKA_2_7_IV0,
+    // Introduced AlterIsr (KIP-497)
+    KAFKA_2_7_IV1

Review comment:
       We probably need another version since we bumped the Fetch protocol 
version yesterday.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -617,9 +665,9 @@ class Partition(val topicPartition: TopicPartition,
         // since the replica's logStartOffset may have incremented
         val leaderLWIncremented = newLeaderLW > oldLeaderLW
 
-        // check if we need to expand ISR to include this replica
-        // if it is not in the ISR yet
-        if (!inSyncReplicaIds.contains(followerId))
+        // Check if this in-sync replica needs to be added to the ISR. We look 
at the "maximal" ISR here so we don't

Review comment:
       nit: we use "maximal" and "effective" interchangeably in this PR. Maybe 
we can choose one term and use it consistently. I do sort of like "maximal" 
since it is more suggestive of the semantics.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -239,6 +249,34 @@ class Partition(val topicPartition: TopicPartition,
 
   def isAddingReplica(replicaId: Int): Boolean = 
assignmentState.isAddingReplica(replicaId)
 
+  /**
+   * This set may include un-committed ISR members following an expansion. 
This "effective" ISR is used for advancing
+   * the high watermark as well as determining which replicas are required for 
acks=all produce requests.
+   *
+   * Only applicable as of IBP 2.7-IV1, for older versions this simply returns 
the committed ISR
+   *
+   * @return the set of replica IDs which are in-sync
+   */
+  def effectiveIsr: Set[Int] = {
+    if (useAlterIsr) {
+      pendingInSyncReplicaIds.getOrElse(inSyncReplicaIds)
+    } else {
+      inSyncReplicaIds
+    }
+  }
+
+  /**
+   * Check if we have an in-flight AlterIsr
+   */
+  def checkInFlightAlterIsr: Boolean = {

Review comment:
       nit: maybe `hasInFlightAlterIsr` so that it's clearer what the return 
value indicates?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -780,6 +826,11 @@ class Partition(val topicPartition: TopicPartition,
    * follower's log end offset may keep falling behind the HW (determined by 
the leader's log end offset) and therefore
    * will never be added to ISR.
    *
+   * With the addition of AlterIsr, we also consider newly added replicas as 
part of the ISR when advancing
+   * the HW. These replicas have not yet been committed to the ISR by the 
controller and propagated via LeaderAndIsr.

Review comment:
       nit: probably need to reword mention of `LeaderAndIsr` since the 
`AlterIsr` response is now used.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")
+            case Errors.FENCED_LEADER_EPOCH =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since we sent an old leader epoch. Giving up.")
+            case _ =>
+              pendingInSyncReplicaIds = None
+              debug(s"Controller failed to update ISR for $topicPartition due 
to $error. Retrying.")
+          }
+          case Right(leaderAndIsr: LeaderAndIsr) =>
+            // Success from controller, still need to check a few things
+            if (leaderAndIsr.leaderEpoch != leaderEpoch) {
+              debug(s"Ignoring ISR with ${leaderAndIsr} since we have a stale 
leader epoch.")
+            } else if (leaderAndIsr.zkVersion <= zkVersion) {
+              debug(s"Ignoring ISR with ${leaderAndIsr} since we have a new 
one.")
+            } else {
+              inSyncReplicaIds = leaderAndIsr.isr.toSet
+              zkVersion = leaderAndIsr.zkVersion
+              pendingInSyncReplicaIds = None
+              info("ISR updated to [%s] and version updated to 
[%d]".format(inSyncReplicaIds.mkString(","), zkVersion))

Review comment:
       nit: as long as we're updating this, can we use `$` substitutions?  Also 
can we mention that this update came from `AlterIsr`?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")
+            case Errors.FENCED_LEADER_EPOCH =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since we sent an old leader epoch. Giving up.")
+            case _ =>
+              pendingInSyncReplicaIds = None
+              debug(s"Controller failed to update ISR for $topicPartition due 
to $error. Retrying.")

Review comment:
       nit: We don't need topic partition here, but it would be nice if we 
could include the intended update.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")
+            case Errors.FENCED_LEADER_EPOCH =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since we sent an old leader epoch. Giving up.")
+            case _ =>
+              pendingInSyncReplicaIds = None

Review comment:
       Hmm... Why do we reset `pendingInSyncReplicaIds` if we are retrying? 
Unless we are guaranteed that the update failed, then I think we need to 
continue assuming the worst-case ISR. Maybe we could just could call 
`enqueueIsrUpdate` again to explicitly retry?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")
+            case Errors.FENCED_LEADER_EPOCH =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since we sent an old leader epoch. Giving up.")
+            case _ =>
+              pendingInSyncReplicaIds = None
+              debug(s"Controller failed to update ISR for $topicPartition due 
to $error. Retrying.")
+          }
+          case Right(leaderAndIsr: LeaderAndIsr) =>
+            // Success from controller, still need to check a few things
+            if (leaderAndIsr.leaderEpoch != leaderEpoch) {
+              debug(s"Ignoring ISR with ${leaderAndIsr} since we have a stale 
leader epoch.")
+            } else if (leaderAndIsr.zkVersion <= zkVersion) {
+              debug(s"Ignoring ISR with ${leaderAndIsr} since we have a new 
one.")

Review comment:
       nit: similarly, we can include current `zkVersion`

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1186,19 +1237,75 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def expandIsr(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+  private[cluster] def expandIsr(newInSyncReplica: Int): Unit = {
+    if (useAlterIsr) {
+      expandIsrWithAlterIsr(newInSyncReplica)
+    } else {
+      expandIsrWithZk(newInSyncReplica)
+    }
+  }
+
+  // This is called from maybeExpandIsr which holds the ISR write lock
+  private def expandIsrWithAlterIsr(newInSyncReplica: Int): Unit = {
+    // This is called from maybeExpandIsr which holds the ISR write lock

Review comment:
       nit: redundant comment

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")
+      case Errors.NONE =>
+        // Collect partition-level responses to pass to the callbacks
+        val partitionResponses: mutable.Map[TopicPartition, Either[Errors, 
LeaderAndIsr]] =
+          new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]()
+        debug(s"Controller successfully handled AlterIsr request")
+        data.topics.forEach(topic => {
+          topic.partitions().forEach(partition => {
+            val tp = new TopicPartition(topic.name, partition.partitionIndex)
+            if (partition.errorCode() == Errors.NONE.code()) {

Review comment:
       nit: may as well convert to `Errors` since we do so below anyway

##########
File path: core/src/main/scala/kafka/zk/KafkaZkClient.scala
##########
@@ -452,6 +452,17 @@ class KafkaZkClient private[zk] (zooKeeperClient: 
ZooKeeperClient, isSecure: Boo
     }
   }
 
+  def getBrokerEpoch(brokerId: Int): Option[Long] = {

Review comment:
       I guess we don't need this anymore.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -732,7 +778,7 @@ class Partition(val topicPartition: TopicPartition,
     leaderLogIfLocal match {
       case Some(leaderLog) =>
         // keep the current immutable replica list reference
-        val curInSyncReplicaIds = inSyncReplicaIds
+        val curInSyncReplicaIds = effectiveIsr

Review comment:
       nit: maybe we could rename `curInSyncReplicaIds` to `curEffectiveIsr` 

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -891,7 +942,7 @@ class Partition(val topicPartition: TopicPartition,
   private def needsShrinkIsr(): Boolean = {
     if (isLeader) {
       val outOfSyncReplicaIds = getOutOfSyncReplicas(replicaLagTimeMaxMs)
-      outOfSyncReplicaIds.nonEmpty
+      outOfSyncReplicaIds.nonEmpty && !checkInFlightAlterIsr

Review comment:
       nit: maybe we can check in-flight requests first (same in 
`needsExpandIsr`). Otherwise it's a little odd that `getOutOfSyncReplicas` may 
be based on the maximal ISR while we have an in-flight.

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

Review comment:
       nit: maybe we could shorten this name to just `enqueue` since the fact 
that it is an ISR update is already implied by the argument and the name of the 
trait itself.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")

Review comment:
       nit: "... doesn't know about this **topic** or partition"?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1186,19 +1237,75 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def expandIsr(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+  private[cluster] def expandIsr(newInSyncReplica: Int): Unit = {
+    if (useAlterIsr) {
+      expandIsrWithAlterIsr(newInSyncReplica)
+    } else {
+      expandIsrWithZk(newInSyncReplica)
+    }
+  }
+
+  // This is called from maybeExpandIsr which holds the ISR write lock
+  private def expandIsrWithAlterIsr(newInSyncReplica: Int): Unit = {
+    // This is called from maybeExpandIsr which holds the ISR write lock
+    if (pendingInSyncReplicaIds.isEmpty) {
+      // When expanding the ISR, we can safely assume the new replica will 
make it into the ISR since this puts us in
+      // a more constrained state for advancing the HW.
+      val newIsr = inSyncReplicaIds + newInSyncReplica
+      if (sendAlterIsrRequest(newIsr)) {
+        pendingInSyncReplicaIds = Some(newIsr)
+        debug(s"Adding new in-sync replica $newInSyncReplica. Pending ISR 
updated to [${newIsr.mkString(",")}] for $topicPartition")

Review comment:
       nit: you can take the topic partition out of this message since it is 
already included in `logIdent`. Same on line 1262 below.

##########
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:
       Still not super keen on this propagation delay. At least it would be 
nice if we did not have to wakeup the thread every 50ms when there's nothing to 
do. This is potentially something we can save for a follow-up since coming up 
with a good solution might require some experimentation and analysis.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()

Review comment:
       The use of a queue is a tad odd here. We could use `ListBuffer`? Also 
nit: use type inference.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")

Review comment:
       nit: can we include the broker epoch that was sent in this message?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,6 +1317,37 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
+  private def sendAlterIsrRequest(newIsr: Set[Int]): Boolean = {
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+    alterIsrManager.enqueueIsrUpdate(AlterIsrItem(topicPartition, 
newLeaderAndIsr, result => {
+      inWriteLock(leaderIsrUpdateLock) {
+        result match {
+          case Left(error: Errors) => error match {
+            case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since it doesn't know about this partition. Giving up.")
+            case Errors.FENCED_LEADER_EPOCH =>
+              debug(s"Controller failed to update ISR for $topicPartition 
since we sent an old leader epoch. Giving up.")
+            case _ =>
+              pendingInSyncReplicaIds = None
+              debug(s"Controller failed to update ISR for $topicPartition due 
to $error. Retrying.")
+          }
+          case Right(leaderAndIsr: LeaderAndIsr) =>
+            // Success from controller, still need to check a few things
+            if (leaderAndIsr.leaderEpoch != leaderEpoch) {
+              debug(s"Ignoring ISR with ${leaderAndIsr} since we have a stale 
leader epoch.")

Review comment:
       Maybe helpful if these messages indicate that this `leaderAndIsr` can 
from an `AlterIsr` response. Also, it may be useful to include the current 
(stale) leader epoch.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")
+      case Errors.NONE =>
+        // Collect partition-level responses to pass to the callbacks
+        val partitionResponses: mutable.Map[TopicPartition, Either[Errors, 
LeaderAndIsr]] =
+          new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]()
+        debug(s"Controller successfully handled AlterIsr request")
+        data.topics.forEach(topic => {

Review comment:
       nit: usually we write this as `forEach { topic =>`. Avoids the extra 
parenthesis.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {

Review comment:
       nit: maybe split this into two separate methods?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -323,6 +330,8 @@ class ReplicaManager(val config: KafkaConfig,
     val haltBrokerOnFailure = config.interBrokerProtocolVersion < KAFKA_1_0_IV0
     logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", 
haltBrokerOnFailure)
     logDirFailureHandler.start()
+

Review comment:
       nit: remove these lines

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")
+      case Errors.NONE =>
+        // Collect partition-level responses to pass to the callbacks
+        val partitionResponses: mutable.Map[TopicPartition, Either[Errors, 
LeaderAndIsr]] =
+          new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]()
+        debug(s"Controller successfully handled AlterIsr request")

Review comment:
       nit: this message would be more useful if we include the response. 
Perhaps it would be better to log each partition update separately?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1317,7 +1326,9 @@ class ReplicaManager(val config: KafkaConfig,
             partitionOpt.foreach { partition =>
               val currentLeaderEpoch = partition.getLeaderEpoch
               val requestLeaderEpoch = partitionState.leaderEpoch
-              if (requestLeaderEpoch > currentLeaderEpoch) {
+              val currentZkVersion = partition.getZkVersion
+              val requestZkVersion = partitionState.zkVersion
+              if (requestLeaderEpoch > currentLeaderEpoch || requestZkVersion 
> currentZkVersion) {

Review comment:
       Do we still need this change? I think we are trying to keep the current 
approach where the controller bumps the leader epoch for any 
controller-initiated change.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]

Review comment:
       nit: use type inference

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")

Review comment:
       Hmm.. Where does this exception get caught? Since it is in the response 
handler, I guess that `NetworkClient` just eats it. Perhaps we should just 
continue retrying so that the problem remains visible in the logs.

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")
+      case Errors.NONE =>
+        // Collect partition-level responses to pass to the callbacks
+        val partitionResponses: mutable.Map[TopicPartition, Either[Errors, 
LeaderAndIsr]] =
+          new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]()
+        debug(s"Controller successfully handled AlterIsr request")
+        data.topics.forEach(topic => {
+          topic.partitions().forEach(partition => {
+            val tp = new TopicPartition(topic.name, partition.partitionIndex)
+            if (partition.errorCode() == Errors.NONE.code()) {
+              val newLeaderAndIsr = new LeaderAndIsr(partition.leader(), 
partition.leaderEpoch(),
+                partition.isr().asScala.toList.map(_.toInt), 
partition.currentIsrVersion)
+              partitionResponses(tp) = Right(newLeaderAndIsr)
+            } else {
+              partitionResponses(tp) = 
Left(Errors.forCode(partition.errorCode()))
+            }
+          })
+        })
+
+        // Iterate across the items we sent rather than what we received to 
ensure we run the callback even if a
+        // partition was somehow erroneously excluded from the response. Note 
that these callbacks are run from
+        // the leaderIsrUpdateLock write lock in Partition#sendAlterIsrRequest
+        inflightAlterIsrItems.foreach(inflightAlterIsr => try {
+          if (partitionResponses.contains(inflightAlterIsr.topicPartition)) {
+            
inflightAlterIsr.callback.apply(partitionResponses(inflightAlterIsr.topicPartition))
+          } else {
+            inflightAlterIsr.callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) 
// TODO better error here?

Review comment:
       Maybe we could log a warning and let the partition remain in 
`unsentIsrUpdates` so that it is retried until we get a response?

##########
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)
+  }
+
+  override def enqueueIsrUpdate(alterIsrItem: AlterIsrItem): Boolean = {
+    unsentIsrUpdates.putIfAbsent(alterIsrItem.topicPartition, alterIsrItem) == 
null
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    unsentIsrUpdates.remove(topicPartition)
+  }
+
+  private def propagateIsrChanges(): Unit = {
+    if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, 
true)) {
+      // Copy current unsent ISRs but don't remove from the map
+      val inflightAlterIsrItems: mutable.Queue[AlterIsrItem] = new 
mutable.Queue[AlterIsrItem]()
+      unsentIsrUpdates.values().forEach(item => 
inflightAlterIsrItems.enqueue(item))
+
+      val now = time.milliseconds()
+      lastIsrPropagationMs.set(now)
+
+      buildAndSendRequest(inflightAlterIsrItems.toSeq)
+    }
+  }
+
+  def buildAndSendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val message = new AlterIsrRequestData()
+      .setBrokerId(brokerId)
+      .setBrokerEpoch(brokerEpochSupplier.apply())
+      .setTopics(new util.ArrayList())
+
+    inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach(entry => {
+      val topicPart = new AlterIsrRequestData.TopicData()
+        .setName(entry._1)
+        .setPartitions(new util.ArrayList())
+      message.topics().add(topicPart)
+      entry._2.foreach(item => {
+        topicPart.partitions().add(new AlterIsrRequestData.PartitionData()
+          .setPartitionIndex(item.topicPartition.partition)
+          .setLeaderId(item.leaderAndIsr.leader)
+          .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
+          .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+          .setCurrentIsrVersion(item.leaderAndIsr.zkVersion)
+        )
+      })
+    })
+
+    def responseHandler(response: ClientResponse): Unit = {
+      try {
+        val body: AlterIsrResponse = 
response.responseBody().asInstanceOf[AlterIsrResponse]
+        handleAlterIsrResponse(body, inflightAlterIsrItems)
+      } finally {
+        // Be sure to clear the in-flight flag to allow future requests
+        if (!inflightRequest.compareAndSet(true, false)) {
+          throw new IllegalStateException("AlterIsr response callback called 
when no requests were in flight")
+        }
+      }
+    }
+
+    debug(s"Sending AlterIsr to controller $message")
+    controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), 
responseHandler)
+  }
+
+  def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, 
inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = {
+    val data: AlterIsrResponseData = alterIsrResponse.data
+
+    Errors.forCode(data.errorCode) match {
+      case Errors.STALE_BROKER_EPOCH =>
+        warn(s"Broker had a stale broker epoch, retrying.")
+      case Errors.CLUSTER_AUTHORIZATION_FAILED =>
+        warn(s"Broker is not authorized to send AlterIsr to controller")
+        throw Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not 
authorized to send AlterIsr to controller")
+      case Errors.NONE =>
+        // Collect partition-level responses to pass to the callbacks
+        val partitionResponses: mutable.Map[TopicPartition, Either[Errors, 
LeaderAndIsr]] =
+          new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]()
+        debug(s"Controller successfully handled AlterIsr request")
+        data.topics.forEach(topic => {
+          topic.partitions().forEach(partition => {
+            val tp = new TopicPartition(topic.name, partition.partitionIndex)
+            if (partition.errorCode() == Errors.NONE.code()) {
+              val newLeaderAndIsr = new LeaderAndIsr(partition.leader(), 
partition.leaderEpoch(),

Review comment:
       nit: unneeded parenthesis

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3045,6 +3045,24 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleAlterIsrRequest(request: RequestChannel.Request): Unit = {
+    val alterIsrRequest = request.body[AlterIsrRequest]
+
+    if (!controller.isActive) {

Review comment:
       I think authorization should probably be the first thing we do.

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1210,19 +1243,66 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def expandIsr(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newIsr.toList, zkVersion)
+  private[cluster] def expandIsr(newInSyncReplica: Int): Unit = {
+    if (useAlterIsr) {
+      expandIsrWithAlterIsr(newInSyncReplica)
+    } else {
+      expandIsrWithZk(newInSyncReplica)
+    }
+  }
+
+  private def expandIsrWithAlterIsr(newInSyncReplica: Int): Unit = {
+    // This is called from maybeExpandIsr which holds the ISR write lock
+    if (pendingInSyncReplicaIds.isEmpty) {
+      // When expanding the ISR, we can safely assume the new replica will 
make it into the ISR since this puts us in
+      // a more constrained state for advancing the HW.
+      val newIsr = inSyncReplicaIds + newInSyncReplica
+      pendingInSyncReplicaIds = Some(newIsr)
+      debug(s"Adding new in-sync replica $newInSyncReplica. Pending ISR 
updated to [${newIsr.mkString(",")}] for $topicPartition")
+      alterIsr(newIsr)
+    } else {
+      debug(s"ISR update in-flight, not adding new in-sync replica 
$newInSyncReplica for $topicPartition")
+    }
+  }
+
+  private def expandIsrWithZk(newInSyncReplica: Int): Unit = {
+    val newInSyncReplicaIds = inSyncReplicaIds + newInSyncReplica
+    info(s"Expanding ISR from ${inSyncReplicaIds.mkString(",")} to 
${newInSyncReplicaIds.mkString(",")}")
+    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
newInSyncReplicaIds.toList, zkVersion)
     val zkVersionOpt = stateStore.expandIsr(controllerEpoch, newLeaderAndIsr)
-    maybeUpdateIsrAndVersion(newIsr, zkVersionOpt)
+    maybeUpdateIsrAndVersionWithZk(newInSyncReplicaIds, zkVersionOpt)
   }
 
-  private[cluster] def shrinkIsr(newIsr: Set[Int]): Unit = {
+  private[cluster] def shrinkIsr(outOfSyncReplicas: Set[Int]): Unit = {
+    if (useAlterIsr) {
+      shrinkIsrWithAlterIsr(outOfSyncReplicas)
+    } else {
+      shrinkIsrWithZk(inSyncReplicaIds -- outOfSyncReplicas)
+    }
+  }
+
+  private def shrinkIsrWithAlterIsr(outOfSyncReplicas: Set[Int]): Unit = {
+    // This is called from maybeShrinkIsr which holds the ISR write lock
+    if (pendingInSyncReplicaIds.isEmpty) {
+      // When shrinking the ISR, we cannot assume that the update will succeed 
as this could erroneously advance the HW
+      // We update pendingInSyncReplicaIds here simply to prevent any further 
ISR updates from occurring until we get
+      // the next LeaderAndIsr
+      pendingInSyncReplicaIds = Some(inSyncReplicaIds)

Review comment:
       The problem is that it is a sort of worst-case ISR and not the intended 
ISR update itself. Tough to come up with a good name to describe that. Just for 
the sake of having an alternative, what if we used case classes to represent 
the current ISR state and pending update? For example:
   
   ```scala
   sealed trait IsrStatus {
     def isr: Set[Int]
     def maximalIsr: Set[Int]
   }
   case class PendingExpand(isr: Set[Int], newInSyncReplicaId: Int) extends 
IsrStatus {
     val maximalIsr = isr + newInSyncReplicaId
   }
   case class PendingShrink(isr: Set[Int], outOfSync: Set[Int]) extends 
IsrStatus  {
     val maximalIsr = isr
   }
   case class Stable(isr: Set[Int]) extends IsrStatus {
     val maximalIsr = isr
   }
   ```
   
   Then we can get rid of `effectiveIsr`, `inSyncReplicaIds`, and 
`pendingInSyncReplicaIds`.




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