You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/12/08 19:23:22 UTC

[GitHub] [kafka] mumrah opened a new pull request #9713: KAFKA-10825 ZK ISR manager

mumrah opened a new pull request #9713:
URL: https://github.com/apache/kafka/pull/9713


   In an effort to consolidate the ISR write paths, this PR adapts the ZK ISR update into the new AlterIsrManager trait. This will allow us minimize divergence in the ISR update code in Partition.scala.
   
   This also removes PartitionStateStore and moves the ISR propagation code to the new ZK ISR manager class
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [kafka] mumrah commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r545964382



##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1374,47 +1314,28 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def shrinkIsrWithZk(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.toList, zkVersion)
-    val zkVersionOpt = stateStore.shrinkIsr(controllerEpoch, newLeaderAndIsr)
-    if (zkVersionOpt.isDefined) {
-      isrChangeListener.markShrink()
-    }
-    maybeUpdateIsrAndVersionWithZk(newIsr, zkVersionOpt)
-  }
-
-  private def maybeUpdateIsrAndVersionWithZk(isr: Set[Int], zkVersionOpt: Option[Int]): Unit = {
-    zkVersionOpt match {
-      case Some(newVersion) =>
-        isrState = CommittedIsr(isr)
-        zkVersion = newVersion
-        info("ISR updated to [%s] and zkVersion updated to [%d]".format(isr.mkString(","), zkVersion))
-
-      case None =>
-        info(s"Cached zkVersion $zkVersion not equal to that in zookeeper, skip updating ISR")
-        isrChangeListener.markFailed()
-    }
-  }
-
   private def sendAlterIsrRequest(proposedIsrState: IsrState): Unit = {
     val isrToSend: Set[Int] = proposedIsrState match {
       case PendingExpandIsr(isr, newInSyncReplicaId) => isr + newInSyncReplicaId
       case PendingShrinkIsr(isr, outOfSyncReplicaIds) => isr -- outOfSyncReplicaIds
       case state =>
+        isrChangeListener.markFailed()
         throw new IllegalStateException(s"Invalid state $state for `AlterIsr` request for partition $topicPartition")
     }
 
     val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion)
-    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState))
+    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState), controllerEpoch)
 
-    if (!alterIsrManager.enqueue(alterIsrItem)) {
-      isrChangeListener.markFailed()
-      throw new IllegalStateException(s"Failed to enqueue `AlterIsr` request with state " +
-        s"$newLeaderAndIsr for partition $topicPartition")
-    }
-
-    isrState = proposedIsrState
-    debug(s"Sent `AlterIsr` request to change state to $newLeaderAndIsr after transition to $proposedIsrState")
+    alterIsrManager.submit(alterIsrItem, (wasSubmitted: Boolean) => {
+      if (wasSubmitted) {
+        isrState = proposedIsrState

Review comment:
       Yea, this seems simpler than the callback thing. I've made this change in the latest commit




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r546044857



##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1374,46 +1314,29 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def shrinkIsrWithZk(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.toList, zkVersion)
-    val zkVersionOpt = stateStore.shrinkIsr(controllerEpoch, newLeaderAndIsr)
-    if (zkVersionOpt.isDefined) {
-      isrChangeListener.markShrink()
-    }
-    maybeUpdateIsrAndVersionWithZk(newIsr, zkVersionOpt)
-  }
-
-  private def maybeUpdateIsrAndVersionWithZk(isr: Set[Int], zkVersionOpt: Option[Int]): Unit = {
-    zkVersionOpt match {
-      case Some(newVersion) =>
-        isrState = CommittedIsr(isr)
-        zkVersion = newVersion
-        info("ISR updated to [%s] and zkVersion updated to [%d]".format(isr.mkString(","), zkVersion))
-
-      case None =>
-        info(s"Cached zkVersion $zkVersion not equal to that in zookeeper, skip updating ISR")
-        isrChangeListener.markFailed()
-    }
-  }
-
   private def sendAlterIsrRequest(proposedIsrState: IsrState): Unit = {
     val isrToSend: Set[Int] = proposedIsrState match {
       case PendingExpandIsr(isr, newInSyncReplicaId) => isr + newInSyncReplicaId
       case PendingShrinkIsr(isr, outOfSyncReplicaIds) => isr -- outOfSyncReplicaIds
       case state =>
+        isrChangeListener.markFailed()
         throw new IllegalStateException(s"Invalid state $state for `AlterIsr` request for partition $topicPartition")
     }
 
     val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion)
-    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState))
+    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState), controllerEpoch)
 
-    if (!alterIsrManager.enqueue(alterIsrItem)) {
+    val oldState = isrState
+    isrState = proposedIsrState
+
+    if (!alterIsrManager.submit(alterIsrItem)) {
+      // If the ISR manager did not accept our update, we need to revert back to previous state
+      isrState = oldState
       isrChangeListener.markFailed()
       throw new IllegalStateException(s"Failed to enqueue `AlterIsr` request with state " +
         s"$newLeaderAndIsr for partition $topicPartition")
     }
 
-    isrState = proposedIsrState
     debug(s"Sent `AlterIsr` request to change state to $newLeaderAndIsr after transition to $proposedIsrState")

Review comment:
       Should we generalize this log message as well? Also the `IllegalStateException` message above.

##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def submit(alterIsrItem: AlterIsrItem): Boolean = {
+    debug(s"Writing new ISR " + alterIsrItem.leaderAndIsr.isr + " to ZooKeeper with version " +

Review comment:
       nit: rewrite using `$` substitutions?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r544581576



##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1374,47 +1314,28 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def shrinkIsrWithZk(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.toList, zkVersion)
-    val zkVersionOpt = stateStore.shrinkIsr(controllerEpoch, newLeaderAndIsr)
-    if (zkVersionOpt.isDefined) {
-      isrChangeListener.markShrink()
-    }
-    maybeUpdateIsrAndVersionWithZk(newIsr, zkVersionOpt)
-  }
-
-  private def maybeUpdateIsrAndVersionWithZk(isr: Set[Int], zkVersionOpt: Option[Int]): Unit = {
-    zkVersionOpt match {
-      case Some(newVersion) =>
-        isrState = CommittedIsr(isr)
-        zkVersion = newVersion
-        info("ISR updated to [%s] and zkVersion updated to [%d]".format(isr.mkString(","), zkVersion))
-
-      case None =>
-        info(s"Cached zkVersion $zkVersion not equal to that in zookeeper, skip updating ISR")
-        isrChangeListener.markFailed()
-    }
-  }
-
   private def sendAlterIsrRequest(proposedIsrState: IsrState): Unit = {
     val isrToSend: Set[Int] = proposedIsrState match {
       case PendingExpandIsr(isr, newInSyncReplicaId) => isr + newInSyncReplicaId
       case PendingShrinkIsr(isr, outOfSyncReplicaIds) => isr -- outOfSyncReplicaIds
       case state =>
+        isrChangeListener.markFailed()
         throw new IllegalStateException(s"Invalid state $state for `AlterIsr` request for partition $topicPartition")
     }
 
     val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion)
-    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState))
+    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState), controllerEpoch)
 
-    if (!alterIsrManager.enqueue(alterIsrItem)) {
-      isrChangeListener.markFailed()
-      throw new IllegalStateException(s"Failed to enqueue `AlterIsr` request with state " +
-        s"$newLeaderAndIsr for partition $topicPartition")
-    }
-
-    isrState = proposedIsrState
-    debug(s"Sent `AlterIsr` request to change state to $newLeaderAndIsr after transition to $proposedIsrState")
+    alterIsrManager.submit(alterIsrItem, (wasSubmitted: Boolean) => {
+      if (wasSubmitted) {
+        isrState = proposedIsrState

Review comment:
       Could we solve the problem by moving this above the call to `submit`? Then the callback logic should work even if the change is made synchronously. In the case that the request fails to be submitted, then we can reset to the previous state. What do you think?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r546044857



##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -1374,46 +1314,29 @@ class Partition(val topicPartition: TopicPartition,
     }
   }
 
-  private def shrinkIsrWithZk(newIsr: Set[Int]): Unit = {
-    val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.toList, zkVersion)
-    val zkVersionOpt = stateStore.shrinkIsr(controllerEpoch, newLeaderAndIsr)
-    if (zkVersionOpt.isDefined) {
-      isrChangeListener.markShrink()
-    }
-    maybeUpdateIsrAndVersionWithZk(newIsr, zkVersionOpt)
-  }
-
-  private def maybeUpdateIsrAndVersionWithZk(isr: Set[Int], zkVersionOpt: Option[Int]): Unit = {
-    zkVersionOpt match {
-      case Some(newVersion) =>
-        isrState = CommittedIsr(isr)
-        zkVersion = newVersion
-        info("ISR updated to [%s] and zkVersion updated to [%d]".format(isr.mkString(","), zkVersion))
-
-      case None =>
-        info(s"Cached zkVersion $zkVersion not equal to that in zookeeper, skip updating ISR")
-        isrChangeListener.markFailed()
-    }
-  }
-
   private def sendAlterIsrRequest(proposedIsrState: IsrState): Unit = {
     val isrToSend: Set[Int] = proposedIsrState match {
       case PendingExpandIsr(isr, newInSyncReplicaId) => isr + newInSyncReplicaId
       case PendingShrinkIsr(isr, outOfSyncReplicaIds) => isr -- outOfSyncReplicaIds
       case state =>
+        isrChangeListener.markFailed()
         throw new IllegalStateException(s"Invalid state $state for `AlterIsr` request for partition $topicPartition")
     }
 
     val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion)
-    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState))
+    val alterIsrItem = AlterIsrItem(topicPartition, newLeaderAndIsr, handleAlterIsrResponse(proposedIsrState), controllerEpoch)
 
-    if (!alterIsrManager.enqueue(alterIsrItem)) {
+    val oldState = isrState
+    isrState = proposedIsrState
+
+    if (!alterIsrManager.submit(alterIsrItem)) {
+      // If the ISR manager did not accept our update, we need to revert back to previous state
+      isrState = oldState
       isrChangeListener.markFailed()
       throw new IllegalStateException(s"Failed to enqueue `AlterIsr` request with state " +
         s"$newLeaderAndIsr for partition $topicPartition")
     }
 
-    isrState = proposedIsrState
     debug(s"Sent `AlterIsr` request to change state to $newLeaderAndIsr after transition to $proposedIsrState")

Review comment:
       Should we generalize this log message as well? Also the couple `IllegalStateException` messages above.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mumrah commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r539670271



##########
File path: core/src/main/scala/kafka/server/AlterIsrManager.scala
##########
@@ -46,13 +51,36 @@ trait AlterIsrManager {
   def clearPending(topicPartition: TopicPartition): Unit
 }
 
-case class AlterIsrItem(topicPartition: TopicPartition, leaderAndIsr: LeaderAndIsr, callback: Either[Errors, LeaderAndIsr] => Unit)
+case class AlterIsrItem(topicPartition: TopicPartition,
+                        leaderAndIsr: LeaderAndIsr,
+                        callback: Either[Errors, LeaderAndIsr] => Unit,
+                        controllerEpoch: Int = -1) // controllerEpoch needed for Zk impl

Review comment:
       That `-1` is left over from test refactoring. I'll remove




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mumrah commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r539707700



##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def enqueue(alterIsrItem: AlterIsrItem): Boolean = {
+    debug(s"Writing new ISR " + alterIsrItem.leaderAndIsr.isr + " to ZooKeeper with version " +
+      alterIsrItem.leaderAndIsr.zkVersion + " for partition " + alterIsrItem.topicPartition)
+
+    val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, alterIsrItem.topicPartition,
+      alterIsrItem.leaderAndIsr, alterIsrItem.controllerEpoch)
+
+    if (updateSucceeded) {
+      // Track which partitions need to be propagated to the controller
+      isrChangeSet synchronized {
+        isrChangeSet += alterIsrItem.topicPartition
+        lastIsrChangeMs.set(time.milliseconds())
+      }
+
+      // We actually need to apply the callback in another thread since Partition#sendAlterIsrRequest will write

Review comment:
       One issue is the invalid version case for ZK -- if we return None there then partition will think it's a call to DefaultAlterIsrManager... I guess we could throw from ZkIsrManager if the update fails. WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mumrah commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r539705157



##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def enqueue(alterIsrItem: AlterIsrItem): Boolean = {
+    debug(s"Writing new ISR " + alterIsrItem.leaderAndIsr.isr + " to ZooKeeper with version " +
+      alterIsrItem.leaderAndIsr.zkVersion + " for partition " + alterIsrItem.topicPartition)
+
+    val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, alterIsrItem.topicPartition,
+      alterIsrItem.leaderAndIsr, alterIsrItem.controllerEpoch)
+
+    if (updateSucceeded) {
+      // Track which partitions need to be propagated to the controller
+      isrChangeSet synchronized {
+        isrChangeSet += alterIsrItem.topicPartition
+        lastIsrChangeMs.set(time.milliseconds())
+      }
+
+      // We actually need to apply the callback in another thread since Partition#sendAlterIsrRequest will write

Review comment:
       Ok, so in this case, we would need to call the callback from `sendAlterIsrRequest` for the ZK case. I think that would probably be ok. I agree that relying on the separate thread and the lock is a little wonky.
   
   Something like this?
   ```scala
   try {
     alterIsrManager.enqueue(alterIsrItem) match {
       case Some(newVersion) => 
         isrState = proposedIsrState
         handleAlterIsrResponse(proposedState.withZkVersion(newVersion))
       case None =>
         isrState = proposedIsrState
   } catch {
      e: Throwable => 
       isrChangeListener.markFailed()
       throw e
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mumrah commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r540945473



##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def enqueue(alterIsrItem: AlterIsrItem): Boolean = {
+    debug(s"Writing new ISR " + alterIsrItem.leaderAndIsr.isr + " to ZooKeeper with version " +
+      alterIsrItem.leaderAndIsr.zkVersion + " for partition " + alterIsrItem.topicPartition)
+
+    val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, alterIsrItem.topicPartition,
+      alterIsrItem.leaderAndIsr, alterIsrItem.controllerEpoch)
+
+    if (updateSucceeded) {
+      // Track which partitions need to be propagated to the controller
+      isrChangeSet synchronized {
+        isrChangeSet += alterIsrItem.topicPartition
+        lastIsrChangeMs.set(time.milliseconds())
+      }
+
+      // We actually need to apply the callback in another thread since Partition#sendAlterIsrRequest will write

Review comment:
       I went with a slightly different approach. See the latest commits




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mumrah merged pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
mumrah merged pull request #9713:
URL: https://github.com/apache/kafka/pull/9713


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9713: KAFKA-10825 ZK ISR manager

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9713:
URL: https://github.com/apache/kafka/pull/9713#discussion_r539640590



##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def enqueue(alterIsrItem: AlterIsrItem): Boolean = {

Review comment:
       The name "enqueue" suggests an asynchronous change. Wonder if there is another name we could use. Perhaps "submit" suggests less about the implementation?

##########
File path: core/src/main/scala/kafka/cluster/Partition.scala
##########
@@ -51,40 +51,8 @@ trait IsrChangeListener {
   def markFailed(): Unit
 }
 
-trait PartitionStateStore {
-  def fetchTopicConfig(): Properties
-  def shrinkIsr(controllerEpoch: Int, leaderAndIsr: LeaderAndIsr): Option[Int]
-  def expandIsr(controllerEpoch: Int, leaderAndIsr: LeaderAndIsr): Option[Int]
-}
-
-class ZkPartitionStateStore(topicPartition: TopicPartition,
-                            zkClient: KafkaZkClient) extends PartitionStateStore {
-
-  override def fetchTopicConfig(): Properties = {
-    val adminZkClient = new AdminZkClient(zkClient)
-    adminZkClient.fetchEntityConfig(ConfigType.Topic, topicPartition.topic)
-  }
-
-  override def shrinkIsr(controllerEpoch: Int, leaderAndIsr: LeaderAndIsr): Option[Int] = {
-    val newVersionOpt = updateIsr(controllerEpoch, leaderAndIsr)
-    newVersionOpt
-  }
-
-  override def expandIsr(controllerEpoch: Int, leaderAndIsr: LeaderAndIsr): Option[Int] = {
-    val newVersionOpt = updateIsr(controllerEpoch, leaderAndIsr)
-    newVersionOpt
-  }
-
-  private def updateIsr(controllerEpoch: Int, leaderAndIsr: LeaderAndIsr): Option[Int] = {
-    val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topicPartition,
-      leaderAndIsr, controllerEpoch)
-
-    if (updateSucceeded) {
-      Some(newVersion)
-    } else {
-      None
-    }
-  }
+trait TopicConfigProvider {
+  def get(): Properties

Review comment:
       nit: I wonder if it might be better to use a verb like `fetch` which suggests some overhead involved

##########
File path: core/src/main/scala/kafka/server/AlterIsrManager.scala
##########
@@ -46,13 +51,36 @@ trait AlterIsrManager {
   def clearPending(topicPartition: TopicPartition): Unit
 }
 
-case class AlterIsrItem(topicPartition: TopicPartition, leaderAndIsr: LeaderAndIsr, callback: Either[Errors, LeaderAndIsr] => Unit)
+case class AlterIsrItem(topicPartition: TopicPartition,
+                        leaderAndIsr: LeaderAndIsr,
+                        callback: Either[Errors, LeaderAndIsr] => Unit,
+                        controllerEpoch: Int = -1) // controllerEpoch needed for Zk impl

Review comment:
       We don't have to do it here, but I think the controller epoch is no longer needed. This was previously used as a sort of poor man's controller fencing, but now the controller has a stronger mechanism relying on conditional zk updates.
   
   By the way, do we need the default value? Especially the fact that it is a sentinel seems dangerous.

##########
File path: core/src/main/scala/kafka/server/ZkIsrManager.scala
##########
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.utils.{Logging, ReplicationUtils, Scheduler}
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.Time
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import scala.collection.mutable
+
+/**
+ * @param checkIntervalMs How often to check for ISR
+ * @param maxDelayMs  Maximum time that an ISR change may be delayed before sending the notification
+ * @param lingerMs  Maximum time to await additional changes before sending the notification
+ */
+case class IsrChangePropagationConfig(checkIntervalMs: Long, maxDelayMs: Long, lingerMs: Long)
+
+object ZkIsrManager {
+  // This field is mutable to allow overriding change notification behavior in test cases
+  @volatile var DefaultIsrPropagationConfig: IsrChangePropagationConfig = IsrChangePropagationConfig(
+    checkIntervalMs = 2500,
+    lingerMs = 5000,
+    maxDelayMs = 60000,
+  )
+}
+
+class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) extends AlterIsrManager with Logging {
+
+  private val isrChangeNotificationConfig = ZkIsrManager.DefaultIsrPropagationConfig
+  // Visible for testing
+  private[server] val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
+  private val lastIsrChangeMs = new AtomicLong(time.milliseconds())
+  private val lastIsrPropagationMs = new AtomicLong(time.milliseconds())
+
+  override def start(): Unit = {
+    scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _,
+      period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS)
+  }
+
+  override def clearPending(topicPartition: TopicPartition): Unit = {
+    // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to
+    // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK
+    // has already happened, so we may as well send the notification to the controller.
+  }
+
+  override def enqueue(alterIsrItem: AlterIsrItem): Boolean = {
+    debug(s"Writing new ISR " + alterIsrItem.leaderAndIsr.isr + " to ZooKeeper with version " +
+      alterIsrItem.leaderAndIsr.zkVersion + " for partition " + alterIsrItem.topicPartition)
+
+    val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, alterIsrItem.topicPartition,
+      alterIsrItem.leaderAndIsr, alterIsrItem.controllerEpoch)
+
+    if (updateSucceeded) {
+      // Track which partitions need to be propagated to the controller
+      isrChangeSet synchronized {
+        isrChangeSet += alterIsrItem.topicPartition
+        lastIsrChangeMs.set(time.milliseconds())
+      }
+
+      // We actually need to apply the callback in another thread since Partition#sendAlterIsrRequest will write

Review comment:
       Hmm.. It feels a tad brittle to rely on locking assumptions like this. Perhaps we could let `enqueue` return a different response in the case that the change was applied immediately? For example, we could return `Option[Int]` to indicate that new version if the change was applied. Note that we can probably discard the boolean return type and let `enqueue` raise an illegal state exception directly since that is what `Partition` is already doing.




----------------------------------------------------------------
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:
users@infra.apache.org