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